diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java index 50a3ffb157961..cb6d3700afe3a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java @@ -90,6 +90,11 @@ public interface ManagedLedgerMXBean { */ long getAddEntryErrors(); + /** + * @return the number of entries read from the managed ledger (from cache or BK) + */ + long getEntriesReadTotalCount(); + /** * @return the number of readEntries requests that succeeded */ diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java index 58c83961d619f..92f3d892b532d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java @@ -25,25 +25,46 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.locks.StampedLock; +import lombok.Value; +import lombok.experimental.UtilityClass; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang3.tuple.Pair; /** * Contains cursors for a ManagedLedger. - * - *

The goal is to always know the slowest consumer and hence decide which is the oldest ledger we need to keep. - * - *

This data structure maintains a heap and a map of cursors. The map is used to relate a cursor name with + *

+ * The goal is to always know the slowest consumer and hence decide which is the oldest ledger we need to keep. + *

+ * This data structure maintains a heap and a map of cursors. The map is used to relate a cursor name with * an entry index in the heap. The heap data structure sorts cursors in a binary tree which is represented * in a single array. More details about heap implementations: - * https://en.wikipedia.org/wiki/Heap_(data_structure)#Implementation - * - *

The heap is updated and kept sorted when a cursor is updated. + * here + *

+ * The heap is updated and kept sorted when a cursor is updated. * */ public class ManagedCursorContainer implements Iterable { + /** + * This field is incremented everytime the cursor information is updated. + */ + private long version; + + @Value + public static class CursorInfo { + ManagedCursor cursor; + PositionImpl position; + + /** + * Cursor info's version. + *

+ * Use {@link DataVersion#compareVersions(long, long)} to compare between two versions, + * since it rolls over to 0 once reaching Long.MAX_VALUE + */ + long version; + } + private static class Item { final ManagedCursor cursor; PositionImpl position; @@ -56,10 +77,66 @@ private static class Item { } } - public ManagedCursorContainer() { + /** + * Utility class to manage a data version, which rolls over to 0 when reaching Long.MAX_VALUE. + */ + @UtilityClass + public class DataVersion { + + /** + * Compares two data versions, which either rolls overs to 0 when reaching Long.MAX_VALUE. + *

+ * Use {@link DataVersion#getNextVersion(long)} to increment the versions. The assumptions + * are that metric versions are compared with close time proximity one to another, hence, + * they are expected not close to each other in terms of distance, hence we don't + * expect the distance ever to exceed Long.MAX_VALUE / 2, otherwise we wouldn't be able + * to know which one is a later version in case the furthest rolls over to beyond 0. We + * assume the shortest distance between them dictates that. + *

+ * @param v1 First version to compare + * @param v2 Second version to compare + * @return the value {@code 0} if {@code v1 == v2}; + * a value less than {@code 0} if {@code v1 < v2}; and + * a value greater than {@code 0} if {@code v1 > v2} + */ + public static int compareVersions(long v1, long v2) { + if (v1 == v2) { + return 0; + } + + // 0-------v1--------v2--------MAX_LONG + if (v2 > v1) { + long distance = v2 - v1; + long wrapAroundDistance = (Long.MAX_VALUE - v2) + v1; + if (distance < wrapAroundDistance) { + return -1; + } else { + return 1; + } + + // 0-------v2--------v1--------MAX_LONG + } else { + long distance = v1 - v2; + long wrapAroundDistance = (Long.MAX_VALUE - v1) + v2; + if (distance < wrapAroundDistance) { + return 1; // v1 is bigger + } else { + return -1; // v2 is bigger + } + } + } + public static long getNextVersion(long existingVersion) { + if (existingVersion == Long.MAX_VALUE) { + return 0; + } else { + return existingVersion + 1; + } + } } + public ManagedCursorContainer() {} + // Used to keep track of slowest cursor. private final ArrayList heap = new ArrayList<>(); @@ -94,6 +171,7 @@ public void add(ManagedCursor cursor, Position position) { if (cursor.isDurable()) { durableCursorCount++; } + version = DataVersion.getNextVersion(version); } finally { rwLock.unlockWrite(stamp); } @@ -129,6 +207,7 @@ public boolean removeCursor(String name) { if (item.cursor.isDurable()) { durableCursorCount--; } + version = DataVersion.getNextVersion(version); return true; } else { return false; @@ -162,6 +241,7 @@ public Pair cursorUpdated(ManagedCursor cursor, Posi PositionImpl previousSlowestConsumer = heap.get(0).position; item.position = (PositionImpl) newPosition; + version = DataVersion.getNextVersion(version); if (heap.size() == 1) { return Pair.of(previousSlowestConsumer, item.position); @@ -204,6 +284,24 @@ public ManagedCursor getSlowestReader() { } } + /** + * @return Returns the CursorInfo for the cursor with the oldest position, + * or null if there aren't any tracked cursors + */ + public CursorInfo getCursorWithOldestPosition() { + long stamp = rwLock.readLock(); + try { + if (heap.isEmpty()) { + return null; + } else { + Item item = heap.get(0); + return new CursorInfo(item.cursor, item.position, version); + } + } finally { + rwLock.unlockRead(stamp); + } + } + /** * Check whether there are any cursors. * @return true is there are no cursors and false if there are 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 9fb570eee491e..3451167affbb6 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 @@ -540,7 +540,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac log.error("[{}] Error opening metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc)); // Rewind to oldest entry available - initialize(getRollbackPosition(info), Collections.emptyMap(), Collections.emptyMap(), callback); + initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); return; } else if (rc != BKException.Code.OK) { log.warn("[{}] Error opening metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, 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 91edf6ad46648..4cc883f140a46 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 @@ -59,6 +59,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReference; @@ -242,6 +243,9 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { protected volatile long lastAddEntryTimeMs = 0; private long inactiveLedgerRollOverTimeMs = 0; + /** A signal that may trigger all the subsequent OpAddEntry of current ledger to be failed due to timeout. **/ + protected volatile AtomicBoolean currentLedgerTimeoutTriggered; + protected static final int DEFAULT_LEDGER_DELETE_RETRIES = 3; protected static final int DEFAULT_LEDGER_DELETE_BACKOFF_TIME_SEC = 60; private static final String MIGRATION_STATE_PROPERTY = "migrated"; @@ -325,7 +329,7 @@ public enum PositionBound { /** * This variable is used for testing the tests. - * {@link ManagedLedgerTest#testManagedLedgerWithPlacementPolicyInCustomMetadata()} + * ManagedLedgerTest#testManagedLedgerWithPlacementPolicyInCustomMetadata() */ @VisibleForTesting Map createdLedgerCustomMetadata; @@ -534,6 +538,7 @@ public void operationFailed(MetaStoreException e) { STATE_UPDATER.set(this, State.LedgerOpened); updateLastLedgerCreatedTimeAndScheduleRolloverTask(); currentLedger = lh; + currentLedgerTimeoutTriggered = new AtomicBoolean(); lastConfirmedEntry = new PositionImpl(lh.getId(), -1); // bypass empty ledgers, find last ledger with Message if possible. @@ -776,7 +781,8 @@ public void asyncAddEntry(ByteBuf buffer, AddEntryCallback callback, Object ctx) // Jump to specific thread to avoid contention from writers writing from different threads executor.execute(() -> { - OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, callback, ctx); + OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, callback, ctx, + currentLedgerTimeoutTriggered); internalAsyncAddEntry(addOperation); }); } @@ -792,7 +798,8 @@ public void asyncAddEntry(ByteBuf buffer, int numberOfMessages, AddEntryCallback // Jump to specific thread to avoid contention from writers writing from different threads executor.execute(() -> { - OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, numberOfMessages, callback, ctx); + OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, numberOfMessages, callback, ctx, + currentLedgerTimeoutTriggered); internalAsyncAddEntry(addOperation); }); } @@ -844,6 +851,7 @@ protected synchronized void internalAsyncAddEntry(OpAddEntry addOperation) { // Write into lastLedger addOperation.setLedger(currentLedger); + addOperation.setTimeoutTriggered(currentLedgerTimeoutTriggered); ++currentLedgerEntries; currentLedgerSize += addOperation.data.readableBytes(); @@ -1587,6 +1595,7 @@ public void operationComplete(Void v, Stat stat) { LedgerHandle originalCurrentLedger = currentLedger; ledgers.put(lh.getId(), newLedger); currentLedger = lh; + currentLedgerTimeoutTriggered = new AtomicBoolean(); currentLedgerEntries = 0; currentLedgerSize = 0; updateLedgersIdsComplete(originalCurrentLedger); @@ -1670,9 +1679,11 @@ void createNewOpAddEntryForNewLedger() { if (existsOp != null) { // If op is used by another ledger handle, we need to close it and create a new one if (existsOp.ledger != null) { - existsOp.close(); - existsOp = OpAddEntry.createNoRetainBuffer(existsOp.ml, existsOp.data, - existsOp.getNumberOfMessages(), existsOp.callback, existsOp.ctx); + existsOp = existsOp.duplicateAndClose(currentLedgerTimeoutTriggered); + } else { + // This scenario should not happen. + log.warn("[{}] An OpAddEntry's ledger is empty.", name); + existsOp.setTimeoutTriggered(currentLedgerTimeoutTriggered); } existsOp.setLedger(currentLedger); pendingAddEntries.add(existsOp); @@ -2135,6 +2146,7 @@ private void internalReadFromLedger(ReadHandle ledger, OpReadEntry opReadEntry) } protected void asyncReadEntry(ReadHandle ledger, PositionImpl position, ReadEntryCallback callback, Object ctx) { + mbean.addEntriesRead(1); if (config.getReadEntryTimeoutSeconds() > 0) { // set readOpCount to uniquely validate if ReadEntryCallbackWrapper is already recycled long readOpCount = READ_OP_COUNT_UPDATER.incrementAndGet(this); @@ -4210,13 +4222,14 @@ private void checkAddTimeout() { } OpAddEntry opAddEntry = pendingAddEntries.peek(); if (opAddEntry != null) { - final long finalAddOpCount = opAddEntry.addOpCount; boolean isTimedOut = opAddEntry.lastInitTime != -1 && TimeUnit.NANOSECONDS.toSeconds(System.nanoTime() - opAddEntry.lastInitTime) >= timeoutSec; if (isTimedOut) { - log.error("Failed to add entry for ledger {} in time-out {} sec", - (opAddEntry.ledger != null ? opAddEntry.ledger.getId() : -1), timeoutSec); - opAddEntry.handleAddTimeoutFailure(opAddEntry.ledger, finalAddOpCount); + log.warn("[{}] Failed to add entry {}:{} in time-out {} sec", this.name, + opAddEntry.ledger != null ? opAddEntry.ledger.getId() : -1, + opAddEntry.entryId, timeoutSec); + currentLedgerTimeoutTriggered.set(true); + opAddEntry.handleAddFailure(opAddEntry.ledger); } } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java index e057dee99538e..3935828ff3d80 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java @@ -41,6 +41,7 @@ public class ManagedLedgerMBeanImpl implements ManagedLedgerMXBean { private final Rate readEntriesOpsFailed = new Rate(); private final Rate readEntriesOpsCacheMisses = new Rate(); private final Rate markDeleteOps = new Rate(); + private final Rate entriesRead = new Rate(); private final LongAdder dataLedgerOpenOp = new LongAdder(); private final LongAdder dataLedgerCloseOp = new LongAdder(); @@ -80,6 +81,7 @@ public void refreshStats(long period, TimeUnit unit) { ledgerAddEntryLatencyStatsUsec.refresh(); ledgerSwitchLatencyStatsUsec.refresh(); entryStats.refresh(); + entriesRead.calculateRate(seconds); } public void addAddEntrySample(long size) { @@ -120,6 +122,10 @@ public void addReadEntriesSample(int count, long totalSize) { readEntriesOps.recordMultipleEvents(count, totalSize); } + public void addEntriesRead(int count) { + entriesRead.recordEvent(count); + } + public void startDataLedgerOpenOp() { dataLedgerOpenOp.increment(); } @@ -189,6 +195,11 @@ public String getName() { return managedLedger.getName(); } + @Override + public long getEntriesReadTotalCount() { + return entriesRead.getTotalCount(); + } + @Override public double getAddEntryMessagesRate() { return addEntryOps.getRate(); @@ -333,5 +344,4 @@ public PendingBookieOpsStats getPendingBookieOpsStats() { result.cursorLedgerDeleteOp = cursorLedgerDeleteOp.longValue(); return result; } - } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java index ae2beafb64374..acbb0da5a4e74 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java @@ -24,8 +24,10 @@ import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCountUtil; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import lombok.Setter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; @@ -45,7 +47,7 @@ public class OpAddEntry implements AddCallback, CloseCallback, Runnable { protected ManagedLedgerImpl ml; LedgerHandle ledger; - private long entryId; + long entryId; private int numberOfMessages; @SuppressWarnings("unused") @@ -68,6 +70,9 @@ public class OpAddEntry implements AddCallback, CloseCallback, Runnable { AtomicReferenceFieldUpdater.newUpdater(OpAddEntry.class, OpAddEntry.State.class, "state"); volatile State state; + @Setter + private AtomicBoolean timeoutTriggered; + enum State { OPEN, INITIATED, @@ -76,8 +81,8 @@ enum State { } public static OpAddEntry createNoRetainBuffer(ManagedLedgerImpl ml, ByteBuf data, AddEntryCallback callback, - Object ctx) { - OpAddEntry op = createOpAddEntryNoRetainBuffer(ml, data, callback, ctx); + Object ctx, AtomicBoolean timeoutTriggered) { + OpAddEntry op = createOpAddEntryNoRetainBuffer(ml, data, callback, ctx, timeoutTriggered); if (log.isDebugEnabled()) { log.debug("Created new OpAddEntry {}", op); } @@ -85,8 +90,9 @@ public static OpAddEntry createNoRetainBuffer(ManagedLedgerImpl ml, ByteBuf data } public static OpAddEntry createNoRetainBuffer(ManagedLedgerImpl ml, ByteBuf data, int numberOfMessages, - AddEntryCallback callback, Object ctx) { - OpAddEntry op = createOpAddEntryNoRetainBuffer(ml, data, callback, ctx); + AddEntryCallback callback, Object ctx, + AtomicBoolean timeoutTriggered) { + OpAddEntry op = createOpAddEntryNoRetainBuffer(ml, data, callback, ctx, timeoutTriggered); op.numberOfMessages = numberOfMessages; if (log.isDebugEnabled()) { log.debug("Created new OpAddEntry {}", op); @@ -95,7 +101,8 @@ public static OpAddEntry createNoRetainBuffer(ManagedLedgerImpl ml, ByteBuf data } private static OpAddEntry createOpAddEntryNoRetainBuffer(ManagedLedgerImpl ml, ByteBuf data, - AddEntryCallback callback, Object ctx) { + AddEntryCallback callback, Object ctx, + AtomicBoolean timeoutTriggered) { OpAddEntry op = RECYCLER.get(); op.ml = ml; op.ledger = null; @@ -109,6 +116,7 @@ private static OpAddEntry createOpAddEntryNoRetainBuffer(ManagedLedgerImpl ml, B op.startTime = System.nanoTime(); op.state = State.OPEN; op.payloadProcessorHandle = null; + op.timeoutTriggered = timeoutTriggered; ml.mbean.addAddEntrySample(op.dataLength); return op; } @@ -176,7 +184,9 @@ public void addComplete(int rc, final LedgerHandle lh, long entryId, Object ctx) if (!STATE_UPDATER.compareAndSet(OpAddEntry.this, State.INITIATED, State.COMPLETED)) { log.warn("[{}] The add op is terminal legacy callback for entry {}-{} adding.", ml.getName(), lh.getId(), entryId); - OpAddEntry.this.recycle(); + // Since there is a thread is coping this object, do not recycle this object to avoid other problems. + // For example: we recycled this object, other thread get a null "opAddEntry.{variable_name}". + // Recycling is not mandatory, JVM GC will collect it. return; } @@ -200,7 +210,7 @@ public void addComplete(int rc, final LedgerHandle lh, long entryId, Object ctx) lh == null ? -1 : lh.getId(), entryId, dataLength, rc); } - if (rc != BKException.Code.OK) { + if (rc != BKException.Code.OK || timeoutTriggered.get()) { handleAddFailure(lh); } else { // Trigger addComplete callback in a thread hashed on the managed ledger name @@ -307,13 +317,6 @@ private boolean checkAndCompleteOp(Object ctx) { return false; } - void handleAddTimeoutFailure(final LedgerHandle ledger, Object ctx) { - if (checkAndCompleteOp(ctx)) { - this.close(); - this.handleAddFailure(ledger); - } - } - /** * It handles add failure on the given ledger. it can be triggered when add-entry fails or times out. * @@ -333,8 +336,11 @@ void handleAddFailure(final LedgerHandle lh) { }); } - void close() { + OpAddEntry duplicateAndClose(AtomicBoolean timeoutTriggered) { STATE_UPDATER.set(OpAddEntry.this, State.CLOSED); + OpAddEntry duplicate = + OpAddEntry.createNoRetainBuffer(ml, data, getNumberOfMessages(), callback, ctx, timeoutTriggered); + return duplicate; } public State getState() { @@ -389,6 +395,7 @@ public void recycle() { startTime = -1; lastInitTime = -1; payloadProcessorHandle = null; + timeoutTriggered = null; recyclerHandle.recycle(this); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java index 8b2742d958783..ec5b006c4745b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Supplier; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.AsyncCallback; @@ -54,6 +55,8 @@ public ShadowManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper book String name, final Supplier> mlOwnershipChecker) { super(factory, bookKeeper, store, config, scheduledExecutor, name, mlOwnershipChecker); this.sourceMLName = config.getShadowSourceName(); + // ShadowManagedLedgerImpl does not implement add entry timeout yet, so this variable will always be false. + this.currentLedgerTimeoutTriggered = new AtomicBoolean(false); } /** diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index 04d99d3bdf480..f0b3efe39d6b7 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger.impl; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; @@ -46,7 +47,6 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursorMXBean; import org.apache.bookkeeper.mledger.ManagedLedger; -import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.testng.annotations.Test; @@ -105,7 +105,7 @@ public boolean isDurable() { } @Override - public List readEntries(int numberOfEntriesToRead) throws ManagedLedgerException { + public List readEntries(int numberOfEntriesToRead) { return new ArrayList(); } @@ -137,14 +137,14 @@ public long getNumberOfEntriesInBacklog(boolean isPrecise) { } @Override - public void markDelete(Position position) throws ManagedLedgerException { + public void markDelete(Position position) { markDelete(position, Collections.emptyMap()); } @Override - public void markDelete(Position position, Map properties) throws ManagedLedgerException { + public void markDelete(Position position, Map properties) { this.position = position; - container.cursorUpdated(this, (PositionImpl) position); + container.cursorUpdated(this, position); } @Override @@ -209,7 +209,7 @@ public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { } @Override - public void delete(Position position) throws InterruptedException, ManagedLedgerException { + public void delete(Position position) { } @Override @@ -217,7 +217,7 @@ public void asyncDelete(Position position, DeleteCallback callback, Object ctx) } @Override - public void delete(Iterable positions) throws InterruptedException, ManagedLedgerException { + public void delete(Iterable positions) { } @Override @@ -225,7 +225,7 @@ public void asyncDelete(Iterable position, DeleteCallback callback, Ob } @Override - public void clearBacklog() throws InterruptedException, ManagedLedgerException { + public void clearBacklog() { } @Override @@ -233,8 +233,7 @@ public void asyncClearBacklog(ClearBacklogCallback callback, Object ctx) { } @Override - public void skipEntries(int numEntriesToSkip, IndividualDeletedEntries deletedEntries) - throws InterruptedException, ManagedLedgerException { + public void skipEntries(int numEntriesToSkip, IndividualDeletedEntries deletedEntries) { } @Override @@ -243,13 +242,12 @@ public void asyncSkipEntries(int numEntriesToSkip, IndividualDeletedEntries dele } @Override - public Position findNewestMatching(Predicate condition) - throws InterruptedException, ManagedLedgerException { + public Position findNewestMatching(Predicate condition) { return null; } @Override - public Position findNewestMatching(FindPositionConstraint constraint, Predicate condition) throws InterruptedException, ManagedLedgerException { + public Position findNewestMatching(FindPositionConstraint constraint, Predicate condition) { return null; } @@ -270,7 +268,7 @@ public void asyncResetCursor(final Position position, boolean forceReset, } @Override - public void resetCursor(final Position position) throws ManagedLedgerException, InterruptedException { + public void resetCursor(final Position position) { } @@ -284,8 +282,7 @@ public void setAlwaysInactive() { } @Override - public List replayEntries(Set positions) - throws InterruptedException, ManagedLedgerException { + public List replayEntries(Set positions) { return null; } @@ -300,8 +297,7 @@ public Set asyncReplayEntries(Set positi } @Override - public List readEntriesOrWait(int numberOfEntriesToRead) - throws InterruptedException, ManagedLedgerException { + public List readEntriesOrWait(int numberOfEntriesToRead) { return null; } @@ -322,8 +318,7 @@ public boolean cancelPendingReadRequest() { } @Override - public Entry getNthEntry(int N, IndividualDeletedEntries deletedEntries) - throws InterruptedException, ManagedLedgerException { + public Entry getNthEntry(int N, IndividualDeletedEntries deletedEntries) { return null; } @@ -399,13 +394,8 @@ public ManagedCursorMXBean getStats() { return null; } - public void asyncReadEntriesOrWait(int maxEntries, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx) { - } - @Override - public List readEntriesOrWait(int maxEntries, long maxSizeBytes) - throws InterruptedException, ManagedLedgerException { + public List readEntriesOrWait(int maxEntries, long maxSizeBytes) { return null; } @@ -421,7 +411,7 @@ public boolean isClosed() { } @Test - public void testSlowestReadPositionForActiveCursors() throws Exception { + public void testSlowestReadPositionForActiveCursors() { ManagedCursorContainer container = new ManagedCursorContainer(); assertNull(container.getSlowestReaderPosition()); @@ -466,14 +456,20 @@ public void simple() throws Exception { ManagedCursor cursor1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); container.add(cursor1, cursor1.getMarkDeletedPosition()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), + cursor1, new PositionImpl(5, 5)); ManagedCursor cursor2 = new MockManagedCursor(container, "test2", new PositionImpl(2, 2)); container.add(cursor2, cursor2.getMarkDeletedPosition()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 2)); + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), + cursor2, new PositionImpl(2, 2)); ManagedCursor cursor3 = new MockManagedCursor(container, "test3", new PositionImpl(2, 0)); container.add(cursor3, cursor3.getMarkDeletedPosition()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 0)); + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), + cursor3, new PositionImpl(2, 0)); assertEquals(container.toString(), "[test1=5:5, test2=2:2, test3=2:0]"); @@ -487,6 +483,8 @@ public void simple() throws Exception { cursor3.markDelete(new PositionImpl(3, 0)); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 2)); + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), + cursor2, new PositionImpl(2, 2)); cursor2.markDelete(new PositionImpl(10, 5)); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(3, 0)); @@ -498,6 +496,8 @@ public void simple() throws Exception { container.removeCursor(cursor5.getName()); container.removeCursor(cursor1.getName()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(4, 0)); + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), + cursor4, new PositionImpl(4, 0)); assertTrue(container.hasDurableCursors()); @@ -514,7 +514,7 @@ public void simple() throws Exception { } @Test - public void updatingCursorOutsideContainer() throws Exception { + public void updatingCursorOutsideContainer() { ManagedCursorContainer container = new ManagedCursorContainer(); ManagedCursor cursor1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); @@ -533,10 +533,19 @@ public void updatingCursorOutsideContainer() throws Exception { container.cursorUpdated(cursor2, cursor2.position); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), + cursor1, new PositionImpl(5, 5)); + } + + private void assertEqualsCursorAndPosition(ManagedCursorContainer.CursorInfo cursorInfo, + ManagedCursor expectedCursor, + PositionImpl expectedPosition) { + assertThat(cursorInfo.getCursor().getName()).isEqualTo(expectedCursor.getName()); + assertThat(cursorInfo.getPosition()).isEqualTo(expectedPosition); } @Test - public void removingCursor() throws Exception { + public void removingCursor() { ManagedCursorContainer container = new ManagedCursorContainer(); ManagedCursor cursor1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); @@ -607,7 +616,7 @@ public void ordering() throws Exception { } @Test - public void orderingWithUpdates() throws Exception { + public void orderingWithUpdates() { ManagedCursorContainer container = new ManagedCursorContainer(); MockManagedCursor c1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); @@ -672,7 +681,7 @@ public void orderingWithUpdates() throws Exception { } @Test - public void orderingWithUpdatesAndReset() throws Exception { + public void orderingWithUpdatesAndReset() { ManagedCursorContainer container = new ManagedCursorContainer(); MockManagedCursor c1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); @@ -735,4 +744,56 @@ public void orderingWithUpdatesAndReset() throws Exception { assertFalse(container.hasDurableCursors()); } + + @Test + public void testDataVersion() { + assertThat(ManagedCursorContainer.DataVersion.compareVersions(1L, 3L)).isNegative(); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(3L, 1L)).isPositive(); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(3L, 3L)).isZero(); + + long v1 = Long.MAX_VALUE - 1; + long v2 = ManagedCursorContainer.DataVersion.getNextVersion(v1); + + assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isNegative(); + + v2 = ManagedCursorContainer.DataVersion.getNextVersion(v2); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isNegative(); + + v1 = ManagedCursorContainer.DataVersion.getNextVersion(v1); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isNegative(); + + v1 = ManagedCursorContainer.DataVersion.getNextVersion(v1); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isZero(); + + v1 = ManagedCursorContainer.DataVersion.getNextVersion(v1); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isPositive(); + } + + @Test + public void testVersions() { + ManagedCursorContainer container = new ManagedCursorContainer(); + + MockManagedCursor c1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); + MockManagedCursor c2 = new MockManagedCursor(container, "test2", new PositionImpl(5, 1)); + + container.add(c1, c1.getMarkDeletedPosition()); + long version = container.getCursorWithOldestPosition().getVersion(); + + container.add(c2, c2.getMarkDeletedPosition()); + long newVersion = container.getCursorWithOldestPosition().getVersion(); + // newVersion > version + assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); + version = newVersion; + + container.cursorUpdated(c2, new PositionImpl(5, 8)); + newVersion = container.getCursorWithOldestPosition().getVersion(); + // newVersion > version + assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); + version = newVersion; + + container.removeCursor("test2"); + newVersion = container.getCursorWithOldestPosition().getVersion(); + // newVersion > version + assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); + } } 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 7a34947a6b7cc..eca9c28a5ef6b 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 @@ -146,6 +146,7 @@ import org.apache.pulsar.metadata.impl.FaultInjectionMetadataStore; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; +import org.eclipse.jetty.util.BlockingArrayQueue; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.DataProvider; @@ -3184,6 +3185,55 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { ledger.close(); } + @Test + public void testAddEntryResponseTimeout() throws Exception { + // Create ML with feature Add Entry Timeout Check. + final ManagedLedgerConfig config = new ManagedLedgerConfig().setAddEntryTimeoutSeconds(2); + final ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("ml1", config); + final ManagedCursor cursor = ledger.openCursor("c1"); + final CollectCtxAddEntryCallback collectCtxAddEntryCallback = new CollectCtxAddEntryCallback(); + + // Insert a response delay. + bkc.addEntryResponseDelay(8, TimeUnit.SECONDS); + + // Add two entries. + final byte[] msg1 = new byte[]{1}; + final byte[] msg2 = new byte[]{2}; + int ctx1 = 1; + int ctx2 = 2; + ledger.asyncAddEntry(msg1, collectCtxAddEntryCallback, ctx1); + ledger.asyncAddEntry(msg2, collectCtxAddEntryCallback, ctx2); + // Verify all write requests are completed. + Awaitility.await().untilAsserted(() -> { + assertEquals(collectCtxAddEntryCallback.addCompleteCtxList, Arrays.asList(1, 2)); + }); + Entry entry1 = cursor.readEntries(1).get(0); + assertEquals(entry1.getData(), msg1); + entry1.release(); + Entry entry2 = cursor.readEntries(1).get(0); + assertEquals(entry2.getData(), msg2); + entry2.release(); + + // cleanup. + factory.delete(ledger.name); + } + + private static class CollectCtxAddEntryCallback implements AddEntryCallback { + + public List addCompleteCtxList = new BlockingArrayQueue<>(); + public List addFailedCtxList = new BlockingArrayQueue<>(); + + @Override + public void addComplete(Position position, ByteBuf entryData, Object ctx) { + addCompleteCtxList.add(ctx); + } + + @Override + public void addFailed(ManagedLedgerException exception, Object ctx) { + addFailedCtxList.add(ctx); + } + } + /** * It verifies that if bk-client doesn't complete the add-entry in given time out then broker is resilient enough * to create new ledger and add entry successfully. @@ -3259,7 +3309,8 @@ public void avoidUseSameOpAddEntryBetweenDifferentLedger() throws Exception { List oldOps = new ArrayList<>(); for (int i = 0; i < 10; i++) { - OpAddEntry op = OpAddEntry.createNoRetainBuffer(ledger, ByteBufAllocator.DEFAULT.buffer(128).retain(), null, null); + OpAddEntry op = OpAddEntry.createNoRetainBuffer(ledger, + ByteBufAllocator.DEFAULT.buffer(128).retain(), null, null, new AtomicBoolean()); if (i > 4) { op.setLedger(mock(LedgerHandle.class)); } diff --git a/pom.xml b/pom.xml index f3d824d02aa3b..536238a04e9bb 100644 --- a/pom.xml +++ b/pom.xml @@ -259,6 +259,7 @@ flexible messaging model and an intuitive client API. 1.18.3 2.2 + 5.4.0 3.3.0 @@ -1452,6 +1453,12 @@ flexible messaging model and an intuitive client API. oshi-core-java11 ${oshi.version} + + io.rest-assured + rest-assured + ${restassured.version} + test + diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java index c2b4b90073391..bc1f25c5af933 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java @@ -22,7 +22,6 @@ import lombok.Data; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.ClassLoaderSwitcher; import org.apache.pulsar.common.configuration.PulsarConfiguration; import org.apache.pulsar.common.nar.NarClassLoader; import org.eclipse.jetty.servlet.ServletHolder; @@ -40,29 +39,45 @@ public class AdditionalServletWithClassLoader implements AdditionalServlet { @Override public void loadConfig(PulsarConfiguration pulsarConfiguration) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); servlet.loadConfig(pulsarConfiguration); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public String getBasePath() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return servlet.getBasePath(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public ServletHolder getServletHolder() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return servlet.getServletHolder(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void close() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); servlet.close(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } try { classLoader.close(); diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 76d902a253aeb..b33ebab872cfe 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -431,6 +431,12 @@ javax.activation + + io.rest-assured + rest-assured + test + + diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java index 6d49dd81da13d..48577fc701486 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java @@ -125,6 +125,9 @@ public AllocatorStats getAllocatorStats(@PathParam("allocator") String allocator @GET @Path("/bookieops") @ApiOperation(value = "Get pending bookie client op stats by namespace", + notes = "Returns a nested map structure which Swagger does not fully support for display. " + + "Structure: Map>." + + " Please refer to this structure for details.", response = PendingBookieOpsStats.class, // https://github.com/swagger-api/swagger-core/issues/449 // nested containers are not supported diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index 61b354610ac20..eeb65590bec8a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -219,7 +219,7 @@ public void updateDynamicConfiguration(@Suspended AsyncResponse asyncResponse, @ApiOperation(value = "Delete dynamic ServiceConfiguration into metadata only." + " This operation requires Pulsar super-user privileges.") - @ApiResponses(value = { @ApiResponse(code = 204, message = "Service configuration updated successfully"), + @ApiResponses(value = { @ApiResponse(code = 204, message = "Service configuration delete successfully"), @ApiResponse(code = 403, message = "You don't have admin permission to update service-configuration"), @ApiResponse(code = 412, message = "Invalid dynamic-config value"), @ApiResponse(code = 500, message = "Internal server error") }) @@ -240,7 +240,8 @@ public void deleteDynamicConfiguration( @GET @Path("/configuration/values") - @ApiOperation(value = "Get value of all dynamic configurations' value overridden on local config") + @ApiOperation(value = "Get value of all dynamic configurations' value overridden on local config", + response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "You don't have admin permission to view configuration"), @ApiResponse(code = 404, message = "Configuration not found"), @@ -258,7 +259,8 @@ public void getAllDynamicConfigurations(@Suspended AsyncResponse asyncResponse) @GET @Path("/configuration") - @ApiOperation(value = "Get all updatable dynamic configurations's name") + @ApiOperation(value = "Get all updatable dynamic configurations's name", + response = String.class, responseContainer = "List") @ApiResponses(value = { @ApiResponse(code = 403, message = "You don't have admin permission to get configuration")}) public void getDynamicConfigurationName(@Suspended AsyncResponse asyncResponse) { @@ -273,7 +275,8 @@ public void getDynamicConfigurationName(@Suspended AsyncResponse asyncResponse) @GET @Path("/configuration/runtime") - @ApiOperation(value = "Get all runtime configurations. This operation requires Pulsar super-user privileges.") + @ApiOperation(value = "Get all runtime configurations. This operation requires Pulsar super-user privileges.", + response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) public void getRuntimeConfiguration(@Suspended AsyncResponse asyncResponse) { validateSuperUserAccessAsync() @@ -330,7 +333,7 @@ public void getInternalConfigurationData(@Suspended AsyncResponse asyncResponse) @Path("/backlog-quota-check") @ApiOperation(value = "An REST endpoint to trigger backlogQuotaCheck") @ApiResponses(value = { - @ApiResponse(code = 200, message = "Everything is OK"), + @ApiResponse(code = 204, message = "Everything is OK"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 500, message = "Internal server error")}) public void backlogQuotaCheck(@Suspended AsyncResponse asyncResponse) { @@ -368,15 +371,15 @@ public void isReady(@Suspended AsyncResponse asyncResponse) { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Cluster doesn't exist"), @ApiResponse(code = 500, message = "Internal server error")}) - @ApiParam(value = "Topic Version") public void healthCheck(@Suspended AsyncResponse asyncResponse, + @ApiParam(value = "Topic Version") @QueryParam("topicVersion") TopicVersion topicVersion) { validateSuperUserAccessAsync() .thenAccept(__ -> checkDeadlockedThreads()) .thenCompose(__ -> internalRunHealthCheck(topicVersion)) .thenAccept(__ -> { LOG.info("[{}] Successfully run health check.", clientAppId()); - asyncResponse.resume("ok"); + asyncResponse.resume(Response.ok("ok").build()); }).exceptionally(ex -> { LOG.error("[{}] Fail to run health check.", clientAppId(), ex); resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -541,7 +544,7 @@ private CompletableFuture internalDeleteDynamicConfigurationOnMetadataAsyn @Path("/version") @ApiOperation(value = "Get version of current broker") @ApiResponses(value = { - @ApiResponse(code = 200, message = "Everything is OK"), + @ApiResponse(code = 200, message = "The Pulsar version", response = String.class), @ApiResponse(code = 500, message = "Internal server error")}) public String version() throws Exception { return PulsarVersion.getVersion(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java index 5d4ed54c33466..7c3d0290c54f0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java @@ -132,7 +132,7 @@ public void getCluster(@Suspended AsyncResponse asyncResponse, notes = "This operation requires Pulsar superuser privileges, and the name cannot contain the '/' characters." ) @ApiResponses(value = { - @ApiResponse(code = 204, message = "Cluster has been created."), + @ApiResponse(code = 200, message = "Cluster has been created."), @ApiResponse(code = 400, message = "Bad request parameter."), @ApiResponse(code = 403, message = "You don't have admin permission to create the cluster."), @ApiResponse(code = 409, message = "Cluster already exists."), @@ -198,7 +198,7 @@ public void createCluster( value = "Update the configuration for a cluster.", notes = "This operation requires Pulsar superuser privileges.") @ApiResponses(value = { - @ApiResponse(code = 204, message = "Cluster has been updated."), + @ApiResponse(code = 200, message = "Cluster has been updated."), @ApiResponse(code = 400, message = "Bad request parameter."), @ApiResponse(code = 403, message = "Don't have admin permission or policies are read-only."), @ApiResponse(code = 404, message = "Cluster doesn't exist."), @@ -253,7 +253,7 @@ public void updateCluster( value = "Update the configuration for a cluster migration.", notes = "This operation requires Pulsar superuser privileges.") @ApiResponses(value = { - @ApiResponse(code = 204, message = "Cluster has been updated."), + @ApiResponse(code = 200, message = "Cluster has been updated."), @ApiResponse(code = 400, message = "Cluster url must not be empty."), @ApiResponse(code = 403, message = "Don't have admin permission or policies are read-only."), @ApiResponse(code = 404, message = "Cluster doesn't exist."), @@ -652,6 +652,7 @@ public void getBrokerWithNamespaceIsolationPolicy( notes = "This operation requires Pulsar superuser privileges." ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Set namespace isolation policy successfully."), @ApiResponse(code = 400, message = "Namespace isolation policy data is invalid."), @ApiResponse(code = 403, message = "Don't have admin permission or policies are read-only."), @ApiResponse(code = 404, message = "Namespace isolation policy doesn't exist."), @@ -762,6 +763,7 @@ private CompletableFuture filterAndUnloadMatchedNamespaceAsync(NamespaceIs notes = "This operation requires Pulsar superuser privileges." ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Delete namespace isolation policy successfully."), @ApiResponse(code = 403, message = "Don't have admin permission or policies are read only."), @ApiResponse(code = 404, message = "Namespace isolation policy doesn't exist."), @ApiResponse(code = 412, message = "Cluster doesn't exist."), @@ -809,6 +811,7 @@ public void deleteNamespaceIsolationPolicy( notes = "This operation requires Pulsar superuser privileges." ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Set the failure domain of the cluster successfully."), @ApiResponse(code = 403, message = "Don't have admin permission."), @ApiResponse(code = 404, message = "Failure domain doesn't exist."), @ApiResponse(code = 409, message = "Broker already exists in another domain."), @@ -944,6 +947,7 @@ public void getDomain( notes = "This operation requires Pulsar superuser privileges." ) @ApiResponses(value = { + @ApiResponse(code = 200, message = "Delete the failure domain of the cluster successfully"), @ApiResponse(code = 403, message = "Don't have admin permission or policy is read only"), @ApiResponse(code = 404, message = "FailureDomain doesn't exist"), @ApiResponse(code = 412, message = "Cluster doesn't exist"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java index 4350316e2f011..42971ae231c05 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java @@ -39,7 +39,6 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.StreamingOutput; import org.apache.pulsar.broker.admin.AdminResource; -import org.apache.pulsar.client.api.Message; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.FunctionDefinition; import org.apache.pulsar.common.functions.FunctionState; @@ -486,7 +485,7 @@ public List listFunctions( @POST @ApiOperation( value = "Triggers a Pulsar Function with a user-specified value or file data", - response = Message.class + response = String.class ) @ApiResponses(value = { @ApiResponse(code = 400, message = "Invalid request"), @@ -541,6 +540,7 @@ public FunctionState getFunctionState( value = "Put the state associated with a Pulsar Function" ) @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @@ -557,8 +557,9 @@ public void putFunctionState(final @PathParam("tenant") String tenant, } @POST - @ApiOperation(value = "Restart an instance of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Restart an instance of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this function"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @@ -578,8 +579,9 @@ public void restartFunction( } @POST - @ApiOperation(value = "Restart all instances of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Restart all instances of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -597,8 +599,9 @@ public void restartFunction( } @POST - @ApiOperation(value = "Stop an instance of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Stop an instance of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -617,8 +620,9 @@ public void stopFunction( } @POST - @ApiOperation(value = "Stop all instances of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Stop all instances of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -636,8 +640,9 @@ public void stopFunction( } @POST - @ApiOperation(value = "Start an instance of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Start an instance of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -656,8 +661,9 @@ public void startFunction( } @POST - @ApiOperation(value = "Start all instances of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Start all instances of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -718,7 +724,8 @@ public StreamingOutput downloadFunction( @GET @ApiOperation( value = "Fetches a list of supported Pulsar IO connectors currently running in cluster mode", - response = List.class + response = ConnectorDefinition.class, + responseContainer = "List" ) @ApiResponses(value = { @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @@ -739,6 +746,7 @@ public List getConnectorsList() throws IOException { value = "Reload the built-in Functions" ) @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 401, message = "This operation requires super-user access"), @ApiResponse(code = 503, message = "Function worker service is now initializing. Please try again later."), @ApiResponse(code = 500, message = "Internal server error") @@ -768,6 +776,7 @@ public List getBuiltinFunction() { @PUT @ApiOperation(value = "Updates a Pulsar Function on the worker leader", hidden = true) @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 403, message = "The requester doesn't have super-user permissions"), @ApiResponse(code = 404, message = "The function does not exist"), @ApiResponse(code = 400, message = "Invalid request"), 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 8c5851c9d82fb..f20897634d685 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 @@ -35,6 +35,7 @@ import java.util.Base64; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -97,6 +98,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; @@ -463,7 +465,14 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean if (!policies.isPresent()) { return CompletableFuture.completedFuture(null); } - final Set replicationClusters = policies.get().replication_clusters; + // Combine namespace level policies and topic level policies. + Set replicationClusters = policies.get().replication_clusters; + TopicPolicies topicPolicies = + pulsarService.getTopicPoliciesService().getTopicPoliciesIfExists(topicName); + if (topicPolicies != null) { + replicationClusters = new HashSet<>(topicPolicies.getReplicationClusters()); + } + // Do check replicated clusters. if (replicationClusters.size() == 0) { return CompletableFuture.completedFuture(null); } @@ -479,6 +488,7 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean // The replication clusters just has the current cluster itself. return CompletableFuture.completedFuture(null); } + // Do sync operation to other clusters. List> futures = replicationClusters.stream() .map(replicationCluster -> admin.clusters().getClusterAsync(replicationCluster) .thenCompose(clusterData -> pulsarService.getBrokerService() @@ -2845,7 +2855,7 @@ public void readEntryFailed(ManagedLedgerException exception, @Override public void readEntryComplete(Entry entry, Object ctx) { try { - results.complete(generateResponseWithEntry(entry)); + results.complete(generateResponseWithEntry(entry, (PersistentTopic) topic)); } catch (IOException exception) { throw new RestException(exception); } finally { @@ -2959,57 +2969,59 @@ private CompletableFuture findMessageIdByPublishTime(long timestamp, protected CompletableFuture internalPeekNthMessageAsync(String subName, int messagePosition, boolean authoritative) { - CompletableFuture ret; - // If the topic name is a partition name, no need to get partition topic metadata again - if (!topicName.isPartitioned()) { - ret = getPartitionedTopicMetadataAsync(topicName, authoritative, false) - .thenCompose(topicMetadata -> { - if (topicMetadata.partitions > 0) { - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Peek messages on a partitioned topic is not allowed"); - } - return CompletableFuture.completedFuture(null); - }); - } else { - ret = CompletableFuture.completedFuture(null); - } - return ret.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES, subName)) - .thenCompose(__ -> getTopicReferenceAsync(topicName)) - .thenCompose(topic -> { - CompletableFuture entry; - if (!(topic instanceof PersistentTopic)) { - log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), - topicName, subName); - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Peek messages on a non-persistent topic is not allowed"); - } else { - if (subName.startsWith(((PersistentTopic) topic).getReplicatorPrefix())) { - PersistentReplicator repl = getReplicatorReference(subName, (PersistentTopic) topic); - entry = repl.peekNthMessage(messagePosition); - } else { - PersistentSubscription sub = - (PersistentSubscription) getSubscriptionReference(subName, (PersistentTopic) topic); - entry = sub.peekNthMessage(messagePosition); - } - } - return entry; - }).thenCompose(entry -> { - try { - Response response = generateResponseWithEntry(entry); - return CompletableFuture.completedFuture(response); - } catch (NullPointerException npe) { - throw new RestException(Status.NOT_FOUND, "Message not found"); - } catch (Exception exception) { - log.error("[{}] Failed to peek message at position {} from {} {}", clientAppId(), - messagePosition, topicName, subName, exception); - throw new RestException(exception); - } finally { - if (entry != null) { - entry.release(); - } - } - }); + CompletableFuture ret = validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES, subName); + return ret.thenCompose(__ -> { + // If the topic name is a partition name, no need to get partition topic metadata again + if (!topicName.isPartitioned()) { + return getPartitionedTopicMetadataAsync(topicName, authoritative, false) + .thenCompose(topicMetadata -> { + if (topicMetadata.partitions > 0) { + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Peek messages on a partitioned topic is not allowed"); + } + return CompletableFuture.completedFuture(null); + }); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> getTopicReferenceAsync(topicName)) + .thenCompose(topic -> { + CompletableFuture entry; + if (!(topic instanceof PersistentTopic)) { + log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), + topicName, subName); + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Peek messages on a non-persistent topic is not allowed"); + } else { + if (subName.startsWith(((PersistentTopic) topic).getReplicatorPrefix())) { + PersistentReplicator repl = getReplicatorReference(subName, (PersistentTopic) topic); + entry = repl.peekNthMessage(messagePosition); + } else { + PersistentSubscription sub = + (PersistentSubscription) getSubscriptionReference(subName, (PersistentTopic) topic); + entry = sub.peekNthMessage(messagePosition); + } + } + return entry.thenApply(e -> Pair.of(e, (PersistentTopic) topic)); + }).thenCompose(entryTopicPair -> { + Entry entry = entryTopicPair.getLeft(); + PersistentTopic persistentTopic = entryTopicPair.getRight(); + try { + Response response = generateResponseWithEntry(entry, persistentTopic); + return CompletableFuture.completedFuture(response); + } catch (NullPointerException npe) { + throw new RestException(Status.NOT_FOUND, "Message not found"); + } catch (Exception exception) { + log.error("[{}] Failed to peek message at position {} from {} {}", clientAppId(), + messagePosition, topicName, subName, exception); + throw new RestException(exception); + } finally { + if (entry != null) { + entry.release(); + } + } + }); } protected CompletableFuture internalExamineMessageAsync(String initialPosition, long messagePosition, @@ -3073,7 +3085,7 @@ public String toString() { PersistentTopicsBase.this.topicName); } }, null); - return future; + return future.thenApply(entry -> Pair.of(entry, (PersistentTopic) topic)); } catch (ManagedLedgerException exception) { log.error("[{}] Failed to examine message at position {} from {} due to {}", clientAppId(), messagePosition, @@ -3081,9 +3093,11 @@ public String toString() { throw new RestException(exception); } - }).thenApply(entry -> { + }).thenApply(entryTopicPair -> { + Entry entry = entryTopicPair.getLeft(); + PersistentTopic persistentTopic = entryTopicPair.getRight(); try { - return generateResponseWithEntry(entry); + return generateResponseWithEntry(entry, persistentTopic); } catch (IOException exception) { throw new RestException(exception); } finally { @@ -3094,7 +3108,7 @@ public String toString() { }); } - private Response generateResponseWithEntry(Entry entry) throws IOException { + private Response generateResponseWithEntry(Entry entry, PersistentTopic persistentTopic) throws IOException { checkNotNull(entry); PositionImpl pos = (PositionImpl) entry.getPosition(); ByteBuf metadataAndPayload = entry.getDataBuffer(); @@ -3212,6 +3226,14 @@ private Response generateResponseWithEntry(Entry entry) throws IOException { if (metadata.hasNullPartitionKey()) { responseBuilder.header("X-Pulsar-null-partition-key", metadata.isNullPartitionKey()); } + if (metadata.hasTxnidMostBits() && metadata.hasTxnidLeastBits()) { + TxnID txnID = new TxnID(metadata.getTxnidMostBits(), metadata.getTxnidLeastBits()); + boolean isTxnAborted = persistentTopic.isTxnAborted(txnID, (PositionImpl) entry.getPosition()); + responseBuilder.header("X-Pulsar-txn-aborted", isTxnAborted); + } + boolean isTxnUncommitted = ((PositionImpl) entry.getPosition()) + .compareTo(persistentTopic.getMaxReadPosition()) > 0; + responseBuilder.header("X-Pulsar-txn-uncommitted", isTxnUncommitted); // Decode if needed CompressionCodec codec = CompressionCodecProvider.getCompressionCodec(metadata.getCompression()); @@ -3416,11 +3438,12 @@ protected CompletableFuture internalSetReplicationClusters(List cl Set replicationClusters = Sets.newHashSet(clusterIds); return validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) - .thenCompose(__ -> { + .thenAccept(__ -> { if (CollectionUtils.isEmpty(clusterIds)) { throw new RestException(Status.PRECONDITION_FAILED, "ClusterIds should not be null or empty"); } - Set clusters = clusters(); + }).thenCompose(__ -> clustersAsync()) + .thenCompose(clusters -> { List> futures = new ArrayList<>(replicationClusters.size()); for (String clusterId : replicationClusters) { if (!clusters.contains(clusterId)) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java index 80ad72d6f9aa9..0a76fe27e0a35 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java @@ -389,8 +389,9 @@ public List listSinks(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Restart an instance of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Restart an instance of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this sink"), @ApiResponse(code = 400, message = "Invalid restart request"), @ApiResponse(code = 401, message = "The client is not authorized to perform this operation"), @@ -415,8 +416,9 @@ public void restartSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Restart all instances of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Restart all instances of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid restart request"), @ApiResponse(code = 401, message = "The client is not authorized to perform this operation"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @@ -436,8 +438,9 @@ public void restartSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Stop an instance of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Stop an instance of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid stop request"), @ApiResponse(code = 404, message = "The Pulsar Sink instance does not exist"), @ApiResponse(code = 500, message = @@ -460,8 +463,9 @@ public void stopSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Stop all instances of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Stop all instances of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid stop request"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @ApiResponse(code = 500, message = @@ -481,8 +485,9 @@ public void stopSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Start an instance of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Start an instance of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid start request"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @ApiResponse(code = 500, message = @@ -505,8 +510,9 @@ public void startSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Start all instances of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Start all instances of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid start request"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @ApiResponse(code = 500, message = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java index 4af0afc0d6ec5..0d037dd42362f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java @@ -323,7 +323,7 @@ public SourceStatus getSourceStatus( @ApiOperation( value = "Lists all Pulsar Sources currently deployed in a given namespace", response = String.class, - responseContainer = "Collection" + responseContainer = "List" ) @ApiResponses(value = { @ApiResponse(code = 400, message = "Invalid request"), @@ -342,8 +342,9 @@ public List listSources( } @POST - @ApiOperation(value = "Restart an instance of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Restart an instance of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this source"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @@ -365,8 +366,9 @@ public void restartSource( } @POST - @ApiOperation(value = "Restart all instances of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Restart all instances of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -386,8 +388,9 @@ public void restartSource( } @POST - @ApiOperation(value = "Stop instance of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Stop instance of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -407,8 +410,9 @@ public void stopSource( } @POST - @ApiOperation(value = "Stop all instances of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Stop all instances of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -428,8 +432,9 @@ public void stopSource( } @POST - @ApiOperation(value = "Start an instance of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Start an instance of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -449,8 +454,9 @@ public void startSource( } @POST - @ApiOperation(value = "Start all instances of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Start all instances of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java index b93f3e3c6ebcc..cb60414edeb48 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java @@ -103,7 +103,9 @@ public void getTenantAdmin(@Suspended final AsyncResponse asyncResponse, @PUT @Path("/{tenant}") @ApiOperation(value = "Create a new tenant.", notes = "This operation requires Pulsar super-user privileges.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 409, message = "Tenant already exists"), @ApiResponse(code = 412, message = "Tenant name is not valid"), @ApiResponse(code = 412, message = "Clusters can not be empty"), @@ -155,7 +157,9 @@ public void createTenant(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}") @ApiOperation(value = "Update the admins for a tenant.", notes = "This operation requires Pulsar super-user privileges.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 404, message = "Tenant does not exist"), @ApiResponse(code = 409, message = "Tenant already exists"), @ApiResponse(code = 412, message = "Clusters can not be empty"), @@ -190,7 +194,9 @@ public void updateTenant(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}") @ApiOperation(value = "Delete a tenant and all namespaces and topics under it.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 404, message = "Tenant does not exist"), @ApiResponse(code = 405, message = "Broker doesn't allow forced deletion of tenants"), @ApiResponse(code = 409, message = "The tenant still has active namespaces")}) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java index a50bc7515ff6f..ba02a22cbc2c8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java @@ -20,6 +20,7 @@ import io.swagger.annotations.Api; import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; import java.util.ArrayList; @@ -124,7 +125,10 @@ public void getBookieRackInfo(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/racks-info/{bookie}") @ApiOperation(value = "Removed the rack placement information for a specific bookie in the cluster") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") + }) public void deleteBookieRackInfo(@Suspended final AsyncResponse asyncResponse, @PathParam("bookie") String bookieAddress) throws Exception { validateSuperUserAccess(); @@ -153,11 +157,17 @@ public void deleteBookieRackInfo(@Suspended final AsyncResponse asyncResponse, @Path("/racks-info/{bookie}") @ApiOperation(value = "Updates the rack placement information for a specific bookie in the cluster (note." + " bookie address format:`address:port`)") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")} + ) public void updateBookieRackInfo(@Suspended final AsyncResponse asyncResponse, + @ApiParam(value = "The bookie address", required = true) @PathParam("bookie") String bookieAddress, + @ApiParam(value = "The group", required = true) @QueryParam("group") String group, - BookieInfo bookieInfo) throws Exception { + @ApiParam(value = "The bookie info", required = true) + BookieInfo bookieInfo) throws Exception { validateSuperUserAccess(); if (group == null) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java index aba6cb1a0aba4..6f280e8d197f8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java @@ -61,7 +61,12 @@ public StreamingOutput getTopics2() throws Exception { + "sum of all of the resource usage percent is called broker-resource-availability" + "

THIS API IS ONLY FOR USE BY TESTING FOR CONFIRMING NAMESPACE ALLOCATION ALGORITHM", response = ResourceUnit.class, responseContainer = "Map") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Returns broker resource availability as Map>." + + "Since `ResourceUnit` is an interface, its specific content is not determinable via class " + + "reflection. Refer to the source code or interface tests for detailed type definitions.", + response = Map.class), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 409, message = "Load-manager doesn't support operation") }) public Map> getBrokerResourceAvailability(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java index 317ff49e5423c..259195056e326 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java @@ -24,6 +24,8 @@ import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; +import io.swagger.annotations.Example; +import io.swagger.annotations.ExampleProperty; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; import java.util.HashSet; @@ -59,9 +61,11 @@ import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; +import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.EntryFilters; import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.OffloadPolicies; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.Policies; @@ -73,6 +77,12 @@ import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; +import org.apache.pulsar.common.policies.data.TopicHashPositions; +import org.apache.pulsar.common.policies.data.impl.AutoSubscriptionCreationOverrideImpl; +import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; +import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; +import org.apache.pulsar.common.policies.data.impl.BookieAffinityGroupDataImpl; +import org.apache.pulsar.common.policies.data.impl.BundlesDataImpl; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -152,7 +162,9 @@ public void getPolicies(@Suspended AsyncResponse response, @PUT @Path("/{tenant}/{namespace}") @ApiOperation(value = "Creates a new namespace with the specified policies") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster doesn't exist"), @ApiResponse(code = 409, message = "Namespace already exists"), @ApiResponse(code = 412, message = "Namespace name is not valid") }) @@ -180,6 +192,7 @@ public void createNamespace(@Suspended AsyncResponse response, @Path("/{tenant}/{namespace}") @ApiOperation(value = "Delete a namespace and all the topics under it.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @@ -208,6 +221,7 @@ public void deleteNamespace(@Suspended final AsyncResponse asyncResponse, @PathP @Path("/{tenant}/{namespace}/{bundle}") @ApiOperation(value = "Delete a namespace bundle and all the topics under it.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @@ -231,7 +245,10 @@ public void deleteNamespaceBundle(@Suspended AsyncResponse response, @PathParam( @GET @Path("/{tenant}/{namespace}/permissions") - @ApiOperation(value = "Retrieve the permissions for a namespace.") + @ApiOperation(value = "Retrieve the permissions for a namespace.", + notes = "Returns a nested map structure which Swagger does not fully support for display. " + + "Structure: Map>. Please refer to this structure for details.", + response = AuthAction.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Namespace is not empty") }) @@ -251,7 +268,10 @@ public void getPermissions(@Suspended AsyncResponse response, @GET @Path("/{tenant}/{namespace}/permissions/subscription") - @ApiOperation(value = "Retrieve the permissions for a subscription.") + @ApiOperation(value = "Retrieve the permissions for a subscription.", + notes = "Returns a nested map structure which Swagger does not fully support for display. " + + "Structure: Map>. Please refer to this structure for details.", + response = String.class, responseContainer = "Map") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Namespace is not empty")}) @@ -273,7 +293,9 @@ public void getPermissionOnSubscription(@Suspended AsyncResponse response, @POST @Path("/{tenant}/{namespace}/permissions/{role}") @ApiOperation(value = "Grant a new permission to a role on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 501, message = "Authorization is not enabled")}) @@ -297,7 +319,9 @@ public void grantPermissionOnNamespace(@Suspended AsyncResponse asyncResponse, @Path("/{property}/{namespace}/permissions/subscription/{subscription}") @ApiOperation(hidden = true, value = "Grant a new permission to roles for a subscription." + "[Tenant admin is allowed to perform this operation]") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 501, message = "Authorization is not enabled") }) @@ -321,7 +345,9 @@ public void grantPermissionOnSubscription(@Suspended AsyncResponse asyncResponse @DELETE @Path("/{tenant}/{namespace}/permissions/{role}") @ApiOperation(value = "Revoke all permissions to a role on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void revokePermissionsOnNamespace(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -340,7 +366,9 @@ public void revokePermissionsOnNamespace(@Suspended AsyncResponse asyncResponse, @DELETE @Path("/{property}/{namespace}/permissions/{subscription}/{role}") @ApiOperation(hidden = true, value = "Revoke subscription admin-api access permission for a role.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) public void revokePermissionOnSubscription(@Suspended AsyncResponse asyncResponse, @PathParam("property") String property, @@ -360,7 +388,7 @@ public void revokePermissionOnSubscription(@Suspended AsyncResponse asyncRespons @GET @Path("/{tenant}/{namespace}/replication") @ApiOperation(value = "Get the replication clusters for a namespace.", - response = String.class, responseContainer = "List") + response = String.class, responseContainer = "Set") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Namespace is not global")}) @@ -381,7 +409,9 @@ public void getNamespaceReplicationClusters(@Suspended AsyncResponse asyncRespon @POST @Path("/{tenant}/{namespace}/replication") @ApiOperation(value = "Set the replication clusters for a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Peer-cluster can't be part of replication-cluster"), @ApiResponse(code = 412, message = "Namespace is not global or invalid cluster ids") }) @@ -422,7 +452,9 @@ public void getNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @Path @POST @Path("/{tenant}/{namespace}/messageTTL") @ApiOperation(value = "Set message TTL in seconds for namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid TTL") }) public void setNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -442,7 +474,9 @@ public void setNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @Path @DELETE @Path("/{tenant}/{namespace}/messageTTL") @ApiOperation(value = "Remove message TTL in seconds for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid TTL")}) public void removeNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @@ -460,7 +494,7 @@ public void removeNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/subscriptionExpirationTime") - @ApiOperation(value = "Get the subscription expiration time for the namespace") + @ApiOperation(value = "Get the subscription expiration time for the namespace", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void getSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse, @@ -482,7 +516,9 @@ public void getSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse @POST @Path("/{tenant}/{namespace}/subscriptionExpirationTime") @ApiOperation(value = "Set subscription expiration time in minutes for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid expiration time")}) public void setSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse, @@ -505,7 +541,9 @@ public void setSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse @DELETE @Path("/{tenant}/{namespace}/subscriptionExpirationTime") @ApiOperation(value = "Remove subscription expiration time for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist")}) public void removeSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -523,7 +561,7 @@ public void removeSubscriptionExpirationTime(@Suspended AsyncResponse asyncRespo @GET @Path("/{tenant}/{namespace}/deduplication") - @ApiOperation(value = "Get broker side deduplication for all topics in a namespace") + @ApiOperation(value = "Get broker side deduplication for all topics in a namespace", response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void getDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -541,7 +579,9 @@ public void getDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam( @POST @Path("/{tenant}/{namespace}/deduplication") @ApiOperation(value = "Enable or disable broker side deduplication for all topics in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void modifyDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -561,7 +601,9 @@ public void modifyDeduplication(@Suspended AsyncResponse asyncResponse, @PathPar @DELETE @Path("/{tenant}/{namespace}/deduplication") @ApiOperation(value = "Remove broker side deduplication for all topics in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void removeDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -578,7 +620,7 @@ public void removeDeduplication(@Suspended AsyncResponse asyncResponse, @PathPar @GET @Path("/{tenant}/{namespace}/autoTopicCreation") - @ApiOperation(value = "Get autoTopicCreation info in a namespace") + @ApiOperation(value = "Get autoTopicCreation info in a namespace", response = AutoTopicCreationOverrideImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getAutoTopicCreation(@Suspended AsyncResponse asyncResponse, @@ -597,7 +639,9 @@ public void getAutoTopicCreation(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/autoTopicCreation") @ApiOperation(value = "Override broker's allowAutoTopicCreation setting for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 406, message = "The number of partitions should be less than or" + " equal to maxNumPartitionsPerPartitionedTopic"), @@ -633,7 +677,9 @@ public void setAutoTopicCreation( @DELETE @Path("/{tenant}/{namespace}/autoTopicCreation") @ApiOperation(value = "Remove override of broker's allowAutoTopicCreation in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void removeAutoTopicCreation(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -661,7 +707,9 @@ public void removeAutoTopicCreation(@Suspended final AsyncResponse asyncResponse @POST @Path("/{tenant}/{namespace}/autoSubscriptionCreation") @ApiOperation(value = "Override broker's allowAutoSubscriptionCreation setting for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 400, message = "Invalid autoSubscriptionCreation override")}) public void setAutoSubscriptionCreation( @@ -691,7 +739,8 @@ public void setAutoSubscriptionCreation( @GET @Path("/{tenant}/{namespace}/autoSubscriptionCreation") - @ApiOperation(value = "Get autoSubscriptionCreation info in a namespace") + @ApiOperation(value = "Get autoSubscriptionCreation info in a namespace", + response = AutoSubscriptionCreationOverrideImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getAutoSubscriptionCreation(@Suspended final AsyncResponse asyncResponse, @@ -710,7 +759,9 @@ public void getAutoSubscriptionCreation(@Suspended final AsyncResponse asyncResp @DELETE @Path("/{tenant}/{namespace}/autoSubscriptionCreation") @ApiOperation(value = "Remove override of broker's allowAutoSubscriptionCreation in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void removeAutoSubscriptionCreation(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -736,7 +787,7 @@ public void removeAutoSubscriptionCreation(@Suspended final AsyncResponse asyncR @GET @Path("/{tenant}/{namespace}/bundles") - @ApiOperation(value = "Get the bundles split data.") + @ApiOperation(value = "Get the bundles split data.", response = BundlesDataImpl.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Namespace is not setup to split in bundles") }) @@ -768,6 +819,7 @@ public void getBundlesData(@Suspended final AsyncResponse asyncResponse, + " since it wouldresult in non-persistent message loss and" + " unexpected connection closure to the clients.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), @@ -800,6 +852,7 @@ public void unloadNamespace(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{bundle}/unload") @ApiOperation(value = "Unload a namespace bundle") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 403, message = "Don't have admin permission") }) @@ -829,6 +882,7 @@ public void unloadNamespaceBundle(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{bundle}/split") @ApiOperation(value = "Split a namespace bundle") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 403, message = "Don't have admin permission") }) @@ -865,7 +919,7 @@ public void splitNamespaceBundle( @GET @Path("/{tenant}/{namespace}/{bundle}/topicHashPositions") - @ApiOperation(value = "Get hash positions for topics") + @ApiOperation(value = "Get hash positions for topics", response = TopicHashPositions.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) @@ -891,7 +945,9 @@ public void getTopicHashPositions( @POST @Path("/{property}/{namespace}/publishRate") @ApiOperation(hidden = true, value = "Set publish-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void setPublishRate(@Suspended AsyncResponse asyncResponse, @PathParam("property") String property, @PathParam("namespace") String namespace, @ApiParam(value = "Publish rate for all topics of the specified namespace") PublishRate publishRate) { @@ -907,7 +963,9 @@ public void setPublishRate(@Suspended AsyncResponse asyncResponse, @PathParam("p @DELETE @Path("/{property}/{namespace}/publishRate") @ApiOperation(hidden = true, value = "Set publish-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void removePublishRate(@Suspended AsyncResponse asyncResponse, @PathParam("property") String property, @PathParam("namespace") String namespace) { validateNamespaceName(property, namespace); @@ -925,7 +983,8 @@ public void removePublishRate(@Suspended AsyncResponse asyncResponse, @PathParam @Path("/{property}/{namespace}/publishRate") @ApiOperation(hidden = true, value = "Get publish-rate configured for the namespace, null means publish-rate not configured, " - + "-1 means msg-publish-rate or byte-publish-rate not configured in publish-rate yet") + + "-1 means msg-publish-rate or byte-publish-rate not configured in publish-rate yet", + response = PublishRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void getPublishRate(@Suspended AsyncResponse asyncResponse, @@ -944,7 +1003,9 @@ public void getPublishRate(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/dispatchRate") @ApiOperation(value = "Set dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void setDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @ApiParam(value = "Dispatch rate for all topics of the specified namespace") @@ -963,7 +1024,9 @@ public void setDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam(" @DELETE @Path("/{tenant}/{namespace}/dispatchRate") @ApiOperation(value = "Delete dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void deleteDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { validateNamespaceName(tenant, namespace); @@ -980,7 +1043,8 @@ public void deleteDispatchRate(@Suspended AsyncResponse asyncResponse, @PathPara @GET @Path("/{tenant}/{namespace}/dispatchRate") @ApiOperation(value = "Get dispatch-rate configured for the namespace, null means dispatch-rate not configured, " - + "-1 means msg-dispatch-rate or byte-dispatch-rate not configured in dispatch-rate yet") + + "-1 means msg-dispatch-rate or byte-dispatch-rate not configured in dispatch-rate yet", + response = DispatchRate.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -997,7 +1061,9 @@ public void getDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam(" @POST @Path("/{tenant}/{namespace}/subscriptionDispatchRate") @ApiOperation(value = "Set Subscription dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void setSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1019,7 +1085,7 @@ public void setSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/subscriptionDispatchRate") @ApiOperation(value = "Get subscription dispatch-rate configured for the namespace, null means subscription " + "dispatch-rate not configured, -1 means msg-dispatch-rate or byte-dispatch-rate not configured " - + "in dispatch-rate yet") + + "in dispatch-rate yet", response = DispatchRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void getSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @@ -1039,7 +1105,9 @@ public void getSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/subscriptionDispatchRate") @ApiOperation(value = "Delete Subscription dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void deleteSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1057,7 +1125,9 @@ public void deleteSubscriptionDispatchRate(@Suspended AsyncResponse asyncRespons @DELETE @Path("/{tenant}/{namespace}/subscribeRate") @ApiOperation(value = "Delete subscribe-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void deleteSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { validateNamespaceName(tenant, namespace); @@ -1074,7 +1144,9 @@ public void deleteSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathPar @POST @Path("/{tenant}/{namespace}/subscribeRate") @ApiOperation(value = "Set subscribe-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void setSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @ApiParam(value = "Subscribe rate for all topics of the specified namespace") @@ -1092,7 +1164,7 @@ public void setSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam( @GET @Path("/{tenant}/{namespace}/subscribeRate") - @ApiOperation(value = "Get subscribe-rate configured for the namespace") + @ApiOperation(value = "Get subscribe-rate configured for the namespace", response = SubscribeRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void getSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -1110,7 +1182,9 @@ public void getSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam( @DELETE @Path("/{tenant}/{namespace}/replicatorDispatchRate") @ApiOperation(value = "Remove replicator dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void removeReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1121,7 +1195,9 @@ public void removeReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/replicatorDispatchRate") @ApiOperation(value = "Set replicator dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void setReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1135,7 +1211,7 @@ public void setReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/replicatorDispatchRate") @ApiOperation(value = "Get replicator dispatch-rate configured for the namespace, null means replicator " + "dispatch-rate not configured, -1 means msg-dispatch-rate or byte-dispatch-rate not configured " - + "in dispatch-rate yet") + + "in dispatch-rate yet", response = DispatchRateImpl.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getReplicatorDispatchRate(@Suspended final AsyncResponse asyncResponse, @@ -1147,7 +1223,8 @@ public void getReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @GET @Path("/{tenant}/{namespace}/backlogQuotaMap") - @ApiOperation(value = "Get backlog quota map on a namespace.") + @ApiOperation(value = "Get backlog quota map on a namespace.", + response = BacklogQuotaImpl.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getBacklogQuotaMap( @@ -1161,7 +1238,9 @@ public void getBacklogQuotaMap( @POST @Path("/{tenant}/{namespace}/backlogQuota") @ApiOperation(value = " Set a backlog quota for all the topics on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, @@ -1179,7 +1258,9 @@ public void setBacklogQuota( @DELETE @Path("/{tenant}/{namespace}/backlogQuota") @ApiOperation(value = "Remove a backlog quota policy from a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeBacklogQuota( @@ -1192,7 +1273,7 @@ public void removeBacklogQuota( @GET @Path("/{tenant}/{namespace}/retention") - @ApiOperation(value = "Get retention config on a namespace.") + @ApiOperation(value = "Get retention config on a namespace.", response = RetentionPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getRetention(@Suspended final AsyncResponse asyncResponse, @@ -1213,7 +1294,9 @@ public void getRetention(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/retention") @ApiOperation(value = " Set retention configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") }) @@ -1226,7 +1309,9 @@ public void setRetention(@PathParam("tenant") String tenant, @PathParam("namespa @DELETE @Path("/{tenant}/{namespace}/retention") @ApiOperation(value = " Remove retention configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") }) @@ -1239,7 +1324,9 @@ public void removeRetention(@PathParam("tenant") String tenant, @PathParam("name @POST @Path("/{tenant}/{namespace}/persistence") @ApiOperation(value = "Set the persistence configuration for all the topics on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 400, message = "Invalid persistence policies")}) @@ -1261,7 +1348,9 @@ public void setPersistence(@Suspended final AsyncResponse asyncResponse, @PathPa @DELETE @Path("/{tenant}/{namespace}/persistence") @ApiOperation(value = "Delete the persistence configuration for all topics on a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void deletePersistence(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { validateNamespaceName(tenant, namespace); @@ -1279,6 +1368,7 @@ public void deletePersistence(@Suspended final AsyncResponse asyncResponse, @Pat @Path("/{tenant}/{namespace}/persistence/bookieAffinity") @ApiOperation(value = "Set the bookie-affinity-group to namespace-persistent policy.") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @@ -1292,7 +1382,8 @@ public void setBookieAffinityGroup(@PathParam("tenant") String tenant, @PathPara @GET @Path("/{property}/{namespace}/persistence/bookieAffinity") - @ApiOperation(value = "Get the bookie-affinity-group from namespace-local policy.") + @ApiOperation(value = "Get the bookie-affinity-group from namespace-local policy.", + response = BookieAffinityGroupDataImpl.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1307,7 +1398,9 @@ public BookieAffinityGroupData getBookieAffinityGroup(@PathParam("property") Str @DELETE @Path("/{property}/{namespace}/persistence/bookieAffinity") @ApiOperation(value = "Delete the bookie-affinity-group from namespace-local policy.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void deleteBookieAffinityGroup(@PathParam("property") String property, @@ -1318,7 +1411,7 @@ public void deleteBookieAffinityGroup(@PathParam("property") String property, @GET @Path("/{tenant}/{namespace}/persistence") - @ApiOperation(value = "Get the persistence configuration for a namespace.") + @ApiOperation(value = "Get the persistence configuration for a namespace.", response = PersistencePolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -1342,6 +1435,7 @@ public void getPersistence( @Path("/{tenant}/{namespace}/clearBacklog") @ApiOperation(value = "Clear backlog for all topics on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void clearNamespaceBacklog(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -1361,6 +1455,7 @@ public void clearNamespaceBacklog(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{bundle}/clearBacklog") @ApiOperation(value = "Clear backlog for all topics on a namespace bundle.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) @@ -1375,6 +1470,7 @@ public void clearNamespaceBundleBacklog(@PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}/clearBacklog/{subscription}") @ApiOperation(value = "Clear backlog for a given subscription on all topics on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void clearNamespaceBacklogForSubscription(@Suspended final AsyncResponse asyncResponse, @@ -1395,6 +1491,7 @@ public void clearNamespaceBacklogForSubscription(@Suspended final AsyncResponse @Path("/{tenant}/{namespace}/{bundle}/clearBacklog/{subscription}") @ApiOperation(value = "Clear backlog for a given subscription on all topics on a namespace bundle.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) @@ -1410,6 +1507,7 @@ public void clearNamespaceBundleBacklogForSubscription(@PathParam("tenant") Stri @Path("/{tenant}/{namespace}/unsubscribe/{subscription}") @ApiOperation(value = "Unsubscribes the given subscription on all topics on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespacen"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void unsubscribeNamespace(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -1430,6 +1528,7 @@ public void unsubscribeNamespace(@Suspended final AsyncResponse asyncResponse, @ @Path("/{tenant}/{namespace}/{bundle}/unsubscribe/{subscription}") @ApiOperation(value = "Unsubscribes the given subscription on all topics on a namespace bundle.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void unsubscribeNamespaceBundle(@PathParam("tenant") String tenant, @@ -1443,7 +1542,9 @@ public void unsubscribeNamespaceBundle(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/subscriptionAuthMode") @ApiOperation(value = " Set a subscription auth mode for all the topics on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSubscriptionAuthMode(@PathParam("tenant") String tenant, @@ -1456,7 +1557,7 @@ public void setSubscriptionAuthMode(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/subscriptionAuthMode") - @ApiOperation(value = "Get subscription auth mode in a namespace") + @ApiOperation(value = "Get subscription auth mode in a namespace", response = SubscriptionAuthMode.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getSubscriptionAuthMode( @@ -1478,7 +1579,9 @@ public void getSubscriptionAuthMode( @POST @Path("/{tenant}/{namespace}/encryptionRequired") @ApiOperation(value = "Message encryption is required or not for all topics in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), }) public void modifyEncryptionRequired( @@ -1492,7 +1595,7 @@ public void modifyEncryptionRequired( @GET @Path("/{tenant}/{namespace}/encryptionRequired") - @ApiOperation(value = "Get message encryption required status in a namespace") + @ApiOperation(value = "Get message encryption required status in a namespace", response = Boolean.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getEncryptionRequired(@Suspended AsyncResponse asyncResponse, @@ -1512,7 +1615,8 @@ public void getEncryptionRequired(@Suspended AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/delayedDelivery") - @ApiOperation(value = "Get delayed delivery messages config on a namespace.") + @ApiOperation(value = "Get delayed delivery messages config on a namespace.", + response = DelayedDeliveryPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), }) @@ -1534,7 +1638,9 @@ public void getDelayedDeliveryPolicies(@Suspended final AsyncResponse asyncRespo @POST @Path("/{tenant}/{namespace}/delayedDelivery") @ApiOperation(value = "Set delayed delivery messages config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), }) public void setDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1547,7 +1653,9 @@ public void setDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/delayedDelivery") @ApiOperation(value = "Delete delayed delivery messages config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), }) public void removeDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1557,7 +1665,7 @@ public void removeDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/inactiveTopicPolicies") - @ApiOperation(value = "Get inactive topic policies config on a namespace.") + @ApiOperation(value = "Get inactive topic policies config on a namespace.", response = InactiveTopicPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), }) @@ -1579,7 +1687,9 @@ public void getInactiveTopicPolicies(@Suspended final AsyncResponse asyncRespons @DELETE @Path("/{tenant}/{namespace}/inactiveTopicPolicies") @ApiOperation(value = "Remove inactive topic policies from a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void removeInactiveTopicPolicies(@PathParam("tenant") String tenant, @@ -1591,7 +1701,9 @@ public void removeInactiveTopicPolicies(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/inactiveTopicPolicies") @ApiOperation(value = "Set inactive topic policies config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), }) public void setInactiveTopicPolicies(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1603,7 +1715,7 @@ public void setInactiveTopicPolicies(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/maxProducersPerTopic") - @ApiOperation(value = "Get maxProducersPerTopic config on a namespace.") + @ApiOperation(value = "Get maxProducersPerTopic config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxProducersPerTopic( @@ -1625,7 +1737,9 @@ public void getMaxProducersPerTopic( @POST @Path("/{tenant}/{namespace}/maxProducersPerTopic") @ApiOperation(value = " Set maxProducersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxProducersPerTopic value is not valid") }) @@ -1638,7 +1752,9 @@ public void setMaxProducersPerTopic(@PathParam("tenant") String tenant, @PathPar @DELETE @Path("/{tenant}/{namespace}/maxProducersPerTopic") @ApiOperation(value = "Remove maxProducersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeMaxProducersPerTopic(@PathParam("tenant") String tenant, @@ -1649,7 +1765,7 @@ public void removeMaxProducersPerTopic(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/deduplicationSnapshotInterval") - @ApiOperation(value = "Get deduplicationSnapshotInterval config on a namespace.") + @ApiOperation(value = "Get deduplicationSnapshotInterval config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getDeduplicationSnapshotInterval( @@ -1671,7 +1787,9 @@ public void getDeduplicationSnapshotInterval( @POST @Path("/{tenant}/{namespace}/deduplicationSnapshotInterval") @ApiOperation(value = "Set deduplicationSnapshotInterval config on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void setDeduplicationSnapshotInterval(@PathParam("tenant") String tenant , @PathParam("namespace") String namespace @@ -1683,7 +1801,7 @@ public void setDeduplicationSnapshotInterval(@PathParam("tenant") String tenant @GET @Path("/{tenant}/{namespace}/maxConsumersPerTopic") - @ApiOperation(value = "Get maxConsumersPerTopic config on a namespace.") + @ApiOperation(value = "Get maxConsumersPerTopic config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxConsumersPerTopic( @@ -1705,7 +1823,9 @@ public void getMaxConsumersPerTopic( @POST @Path("/{tenant}/{namespace}/maxConsumersPerTopic") @ApiOperation(value = " Set maxConsumersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxConsumersPerTopic value is not valid") }) @@ -1718,7 +1838,9 @@ public void setMaxConsumersPerTopic(@PathParam("tenant") String tenant, @PathPar @DELETE @Path("/{tenant}/{namespace}/maxConsumersPerTopic") @ApiOperation(value = "Remove maxConsumersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeMaxConsumersPerTopic(@PathParam("tenant") String tenant, @@ -1729,7 +1851,7 @@ public void removeMaxConsumersPerTopic(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/maxConsumersPerSubscription") - @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.") + @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxConsumersPerSubscription( @@ -1751,7 +1873,9 @@ public void getMaxConsumersPerSubscription( @POST @Path("/{tenant}/{namespace}/maxConsumersPerSubscription") @ApiOperation(value = " Set maxConsumersPerSubscription configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxConsumersPerSubscription value is not valid")}) @@ -1767,7 +1891,9 @@ public void setMaxConsumersPerSubscription(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/maxConsumersPerSubscription") @ApiOperation(value = " Set maxConsumersPerSubscription configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxConsumersPerSubscription value is not valid")}) @@ -1779,7 +1905,7 @@ public void removeMaxConsumersPerSubscription(@PathParam("tenant") String tenant @GET @Path("/{tenant}/{namespace}/maxUnackedMessagesPerConsumer") - @ApiOperation(value = "Get maxUnackedMessagesPerConsumer config on a namespace.") + @ApiOperation(value = "Get maxUnackedMessagesPerConsumer config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxUnackedMessagesPerConsumer(@Suspended final AsyncResponse asyncResponse, @@ -1800,7 +1926,9 @@ public void getMaxUnackedMessagesPerConsumer(@Suspended final AsyncResponse asyn @POST @Path("/{tenant}/{namespace}/maxUnackedMessagesPerConsumer") @ApiOperation(value = " Set maxConsumersPerTopic configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxUnackedMessagesPerConsumer value is not valid")}) @@ -1816,7 +1944,9 @@ public void setMaxUnackedMessagesPerConsumer(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/maxUnackedMessagesPerConsumer") @ApiOperation(value = "Remove maxUnackedMessagesPerConsumer config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void removeMaxUnackedmessagesPerConsumer(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1826,7 +1956,7 @@ public void removeMaxUnackedmessagesPerConsumer(@PathParam("tenant") String tena @GET @Path("/{tenant}/{namespace}/maxUnackedMessagesPerSubscription") - @ApiOperation(value = "Get maxUnackedMessagesPerSubscription config on a namespace.") + @ApiOperation(value = "Get maxUnackedMessagesPerSubscription config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxUnackedmessagesPerSubscription( @@ -1848,7 +1978,9 @@ public void getMaxUnackedmessagesPerSubscription( @POST @Path("/{tenant}/{namespace}/maxUnackedMessagesPerSubscription") @ApiOperation(value = " Set maxUnackedMessagesPerSubscription configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxUnackedMessagesPerSubscription value is not valid")}) @@ -1863,7 +1995,9 @@ public void setMaxUnackedMessagesPerSubscription( @DELETE @Path("/{tenant}/{namespace}/maxUnackedMessagesPerSubscription") @ApiOperation(value = "Remove maxUnackedMessagesPerSubscription config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void removeMaxUnackedmessagesPerSubscription(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1873,7 +2007,7 @@ public void removeMaxUnackedmessagesPerSubscription(@PathParam("tenant") String @GET @Path("/{tenant}/{namespace}/maxSubscriptionsPerTopic") - @ApiOperation(value = "Get maxSubscriptionsPerTopic config on a namespace.") + @ApiOperation(value = "Get maxSubscriptionsPerTopic config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResponse, @@ -1894,7 +2028,9 @@ public void getMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/maxSubscriptionsPerTopic") @ApiOperation(value = " Set maxSubscriptionsPerTopic configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxUnackedMessagesPerSubscription value is not valid")}) @@ -1909,7 +2045,9 @@ public void setMaxSubscriptionsPerTopic( @DELETE @Path("/{tenant}/{namespace}/maxSubscriptionsPerTopic") @ApiOperation(value = "Remove maxSubscriptionsPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeMaxSubscriptionsPerTopic(@PathParam("tenant") String tenant, @@ -1921,7 +2059,9 @@ public void removeMaxSubscriptionsPerTopic(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/antiAffinity") @ApiOperation(value = "Set anti-affinity group for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid antiAffinityGroup")}) public void setNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @@ -1935,7 +2075,7 @@ public void setNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/antiAffinity") - @ApiOperation(value = "Get anti-affinity group of a namespace.") + @ApiOperation(value = "Get anti-affinity group of a namespace.", response = String.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public String getNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @@ -1947,7 +2087,9 @@ public String getNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/antiAffinity") @ApiOperation(value = "Remove anti-affinity group of a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @@ -1959,7 +2101,8 @@ public void removeNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @GET @Path("{cluster}/antiAffinity/{group}") @ApiOperation(value = "Get all namespaces that are grouped by given anti-affinity group in a given cluster." - + " api can be only accessed by admin of any of the existing tenant") + + " api can be only accessed by admin of any of the existing tenant", + response = String.class, responseContainer = "List") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 412, message = "Cluster not exist/Anti-affinity group can't be empty.")}) public List getAntiAffinityNamespaces(@PathParam("cluster") String cluster, @@ -1985,7 +2128,7 @@ private Policies getDefaultPolicesIfNull(Policies policies) { @Path("/{tenant}/{namespace}/compactionThreshold") @ApiOperation(value = "Maximum number of uncompacted bytes in topics before compaction is triggered.", notes = "The backlog size is compared to the threshold periodically. " - + "A threshold of 0 disabled automatic compaction") + + "A threshold of 0 disabled automatic compaction", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getCompactionThreshold( @@ -2009,7 +2152,9 @@ public void getCompactionThreshold( @ApiOperation(value = "Set maximum number of uncompacted bytes in a topic before compaction is triggered.", notes = "The backlog size is compared to the threshold periodically. " + "A threshold of 0 disabled automatic compaction") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "compactionThreshold value is not valid")}) @@ -2027,7 +2172,9 @@ public void setCompactionThreshold(@PathParam("tenant") String tenant, @ApiOperation(value = "Delete maximum number of uncompacted bytes in a topic before compaction is triggered.", notes = "The backlog size is compared to the threshold periodically. " + "A threshold of 0 disabled automatic compaction") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void deleteCompactionThreshold(@PathParam("tenant") String tenant, @@ -2040,7 +2187,7 @@ public void deleteCompactionThreshold(@PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}/offloadThreshold") @ApiOperation(value = "Maximum number of bytes stored on the pulsar cluster for a topic," + " before the broker will start offloading to longterm storage", - notes = "A negative value disables automatic offloading") + notes = "A negative value disables automatic offloading", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getOffloadThreshold( @@ -2071,7 +2218,9 @@ public void getOffloadThreshold( + " before the broker will start offloading to longterm storage", notes = "-1 will revert to using the cluster default." + " A negative value disables automatic offloading. ") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "offloadThreshold value is not valid")}) @@ -2089,7 +2238,7 @@ public void setOffloadThreshold(@PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}/offloadThresholdInSeconds") @ApiOperation(value = "Maximum number of bytes stored on the pulsar cluster for a topic," + " before the broker will start offloading to longterm storage", - notes = "A negative value disables automatic offloading") + notes = "A negative value disables automatic offloading", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getOffloadThresholdInSeconds( @@ -2119,7 +2268,9 @@ public void getOffloadThresholdInSeconds( @ApiOperation(value = "Set maximum number of seconds stored on the pulsar cluster for a topic," + " before the broker will start offloading to longterm storage", notes = "A negative value disables automatic offloading") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "offloadThresholdInSeconds value is not valid") }) @@ -2143,7 +2294,7 @@ public void setOffloadThresholdInSeconds( + " from the Pulsar cluster's local storage (i.e. BookKeeper)", notes = "A negative value denotes that deletion has been completely disabled." + " 'null' denotes that the topics in the namespace will fall back to the" - + " broker default for deletion lag.") + + " broker default for deletion lag.", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getOffloadDeletionLag( @@ -2173,7 +2324,9 @@ public void getOffloadDeletionLag( @ApiOperation(value = "Set number of milliseconds to wait before deleting a ledger segment which has been offloaded" + " from the Pulsar cluster's local storage (i.e. BookKeeper)", notes = "A negative value disables the deletion completely.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "offloadDeletionLagMs value is not valid")}) @@ -2192,6 +2345,7 @@ public void setOffloadDeletionLag(@PathParam("tenant") String tenant, @ApiOperation(value = "Clear the namespace configured offload deletion lag. The topics in the namespace" + " will fallback to using the default configured deletion lag for the broker") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void clearOffloadDeletionLag(@PathParam("tenant") String tenant, @@ -2205,7 +2359,8 @@ public void clearOffloadDeletionLag(@PathParam("tenant") String tenant, @ApiOperation(value = "The strategy used to check the compatibility of new schemas," + " provided by producers, before automatically updating the schema", notes = "The value AutoUpdateDisabled prevents producers from updating the schema. " - + " If set to AutoUpdateDisabled, schemas must be updated through the REST api") + + " If set to AutoUpdateDisabled, schemas must be updated through the REST api", + response = SchemaAutoUpdateCompatibilityStrategy.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2222,7 +2377,9 @@ public SchemaAutoUpdateCompatibilityStrategy getSchemaAutoUpdateCompatibilityStr + " provided by producers, before automatically updating the schema", notes = "The value AutoUpdateDisabled prevents producers from updating the schema. " + " If set to AutoUpdateDisabled, schemas must be updated through the REST api") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSchemaAutoUpdateCompatibilityStrategy( @@ -2236,7 +2393,8 @@ public void setSchemaAutoUpdateCompatibilityStrategy( @GET @Path("/{tenant}/{namespace}/schemaCompatibilityStrategy") - @ApiOperation(value = "The strategy of the namespace schema compatibility ") + @ApiOperation(value = "The strategy of the namespace schema compatibility ", + response = SchemaCompatibilityStrategy.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2260,7 +2418,9 @@ public void getSchemaCompatibilityStrategy( @PUT @Path("/{tenant}/{namespace}/schemaCompatibilityStrategy") @ApiOperation(value = "Update the strategy used to check the compatibility of new schema") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSchemaCompatibilityStrategy( @@ -2274,7 +2434,7 @@ public void setSchemaCompatibilityStrategy( @GET @Path("/{tenant}/{namespace}/isAllowAutoUpdateSchema") - @ApiOperation(value = "The flag of whether allow auto update schema") + @ApiOperation(value = "The flag of whether allow auto update schema", response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2304,7 +2464,9 @@ public void getIsAllowAutoUpdateSchema( @POST @Path("/{tenant}/{namespace}/isAllowAutoUpdateSchema") @ApiOperation(value = "Update flag of whether allow auto update schema") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setIsAllowAutoUpdateSchema( @@ -2318,7 +2480,8 @@ public void setIsAllowAutoUpdateSchema( @GET @Path("/{tenant}/{namespace}/subscriptionTypesEnabled") - @ApiOperation(value = "The set of whether allow subscription types") + @ApiOperation(value = "The set of whether allow subscription types", + response = SubscriptionType.class, responseContainer = "Set") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2346,7 +2509,9 @@ public void getSubscriptionTypesEnabled( @POST @Path("/{tenant}/{namespace}/subscriptionTypesEnabled") @ApiOperation(value = "Update set of whether allow share sub type") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSubscriptionTypesEnabled( @@ -2377,7 +2542,8 @@ public void removeSubscriptionTypesEnabled(@PathParam("tenant") String tenant, notes = "If the flag is set to true, when a producer without a schema attempts to produce to a topic" + " with schema in this namespace, the producer will be failed to connect. PLEASE be" + " carefully on using this, since non-java clients don't support schema.if you enable" - + " this setting, it will cause non-java clients failed to produce.") + + " this setting, it will cause non-java clients failed to produce.", + response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getSchemaValidtionEnforced( @@ -2412,7 +2578,9 @@ public void getSchemaValidtionEnforced( + " with schema in this namespace, the producer will be failed to connect. PLEASE be" + " carefully on using this, since non-java clients don't support schema.if you enable" + " this setting, it will cause non-java clients failed to produce.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or Namespace doesn't exist"), @ApiResponse(code = 412, message = "schemaValidationEnforced value is not valid")}) public void setSchemaValidationEnforced(@PathParam("tenant") String tenant, @@ -2427,8 +2595,9 @@ public void setSchemaValidationEnforced(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/offloadPolicies") - @ApiOperation(value = " Set offload configuration on a namespace.") + @ApiOperation(value = "Set offload configuration on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @@ -2452,6 +2621,7 @@ public void setOffloadPolicies(@PathParam("tenant") String tenant, @PathParam("n @Path("/{tenant}/{namespace}/removeOffloadPolicies") @ApiOperation(value = " Set offload configuration on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @@ -2471,7 +2641,7 @@ public void removeOffloadPolicies(@PathParam("tenant") String tenant, @PathParam @GET @Path("/{tenant}/{namespace}/offloadPolicies") - @ApiOperation(value = "Get offload configuration on a namespace.") + @ApiOperation(value = "Get offload configuration on a namespace.", response = OffloadPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) @@ -2493,7 +2663,7 @@ public void getOffloadPolicies( @GET @Path("/{tenant}/{namespace}/maxTopicsPerNamespace") - @ApiOperation(value = "Get maxTopicsPerNamespace config on a namespace.") + @ApiOperation(value = "Get maxTopicsPerNamespace config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace does not exist") }) public void getMaxTopicsPerNamespace(@Suspended final AsyncResponse asyncResponse, @@ -2518,7 +2688,9 @@ public void getMaxTopicsPerNamespace(@Suspended final AsyncResponse asyncRespons @POST @Path("/{tenant}/{namespace}/maxTopicsPerNamespace") @ApiOperation(value = "Set maxTopicsPerNamespace config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void setMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -2531,7 +2703,9 @@ public void setMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/maxTopicsPerNamespace") @ApiOperation(value = "Remove maxTopicsPerNamespace config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void removeMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -2542,7 +2716,9 @@ public void removeMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @PUT @Path("/{tenant}/{namespace}/property/{key}/{value}") @ApiOperation(value = "Put a key value pair property on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void setProperty( @Suspended final AsyncResponse asyncResponse, @@ -2556,7 +2732,7 @@ public void setProperty( @GET @Path("/{tenant}/{namespace}/property/{key}") - @ApiOperation(value = "Get property value for a given key on a namespace.") + @ApiOperation(value = "Get property value for a given key on a namespace.", response = String.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void getProperty( @@ -2571,7 +2747,9 @@ public void getProperty( @DELETE @Path("/{tenant}/{namespace}/property/{key}") @ApiOperation(value = "Remove property value for a given key on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void removeProperty( @Suspended final AsyncResponse asyncResponse, @@ -2585,7 +2763,9 @@ public void removeProperty( @PUT @Path("/{tenant}/{namespace}/properties") @ApiOperation(value = "Put key value pairs property on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void setProperties( @Suspended final AsyncResponse asyncResponse, @@ -2599,7 +2779,8 @@ public void setProperties( @GET @Path("/{tenant}/{namespace}/properties") - @ApiOperation(value = "Get key value pair properties for a given namespace.") + @ApiOperation(value = "Get key value pair properties for a given namespace.", + response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void getProperties( @@ -2613,7 +2794,9 @@ public void getProperties( @DELETE @Path("/{tenant}/{namespace}/properties") @ApiOperation(value = "Clear properties on a given namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void clearProperties( @Suspended final AsyncResponse asyncResponse, @@ -2625,7 +2808,7 @@ public void clearProperties( @GET @Path("/{tenant}/{namespace}/resourcegroup") - @ApiOperation(value = "Get the resource group attached to the namespace") + @ApiOperation(value = "Get the resource group attached to the namespace", response = String.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void getNamespaceResourceGroup( @@ -2647,7 +2830,9 @@ public void getNamespaceResourceGroup( @POST @Path("/{tenant}/{namespace}/resourcegroup/{resourcegroup}") @ApiOperation(value = "Set resourcegroup for a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid resourcegroup") }) public void setNamespaceResourceGroup(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -2659,7 +2844,9 @@ public void setNamespaceResourceGroup(@PathParam("tenant") String tenant, @PathP @DELETE @Path("/{tenant}/{namespace}/resourcegroup") @ApiOperation(value = "Delete resourcegroup for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid resourcegroup")}) public void removeNamespaceResourceGroup(@PathParam("tenant") String tenant, @@ -2671,7 +2858,13 @@ public void removeNamespaceResourceGroup(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/scanOffloadedLedgers") @ApiOperation(value = "Trigger the scan of offloaded Ledgers on the LedgerOffloader for the given namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Successful get of offloaded ledger data", response = String.class, + examples = @Example(value = { @ExampleProperty(mediaType = "application/json", + value = "{\"objects\":[{\"key1\":\"value1\",\"key2\":\"value2\"}]," + + "\"total\":100,\"errors\":5,\"unknown\":3}") + })), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public Response scanOffloadedLedgers(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -2720,7 +2913,7 @@ public void finished(int total, int errors, int unknown) throws Exception { @GET @Path("/{tenant}/{namespace}/entryFilters") - @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.") + @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.", response = EntryFilters.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getEntryFiltersPerTopic( @@ -2743,6 +2936,7 @@ public void getEntryFiltersPerTopic( @Path("/{tenant}/{namespace}/entryFilters") @ApiOperation(value = "Set entry filters for namespace") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 400, message = "Specified entry filters are not valid"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") @@ -2764,7 +2958,9 @@ public void setEntryFiltersPerTopic(@Suspended AsyncResponse asyncResponse, @Pat @DELETE @Path("/{tenant}/{namespace}/entryFilters") @ApiOperation(value = "Remove entry filters for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid TTL")}) public void removeNamespaceEntryFilters(@Suspended AsyncResponse asyncResponse, @@ -2780,7 +2976,5 @@ public void removeNamespaceEntryFilters(@Suspended AsyncResponse asyncResponse, }); } - - private static final Logger log = LoggerFactory.getLogger(Namespaces.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index f8fe30ccecb2c..cae7c651ce791 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -52,8 +52,10 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.EntryFilters; import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.policies.data.TopicStats; @@ -74,7 +76,7 @@ public class NonPersistentTopics extends PersistentTopics { @GET @Path("/{tenant}/{namespace}/{topic}/partitions") - @ApiOperation(value = "Get partitioned topic metadata.") + @ApiOperation(value = "Get partitioned topic metadata.", response = PartitionedTopicMetadata.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to manage resources on this tenant"), @@ -102,7 +104,7 @@ public void getPartitionedMetadata( @GET @Path("{tenant}/{namespace}/{topic}/internalStats") - @ApiOperation(value = "Get the internal stats for the topic.") + @ApiOperation(value = "Get the internal stats for the topic.", response = PersistentTopicInternalStats.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to manage resources on this tenant"), @@ -145,6 +147,7 @@ public void getInternalStats( @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to manage resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -306,6 +309,7 @@ public void getPartitionedStats( @Path("/{tenant}/{namespace}/{topic}/unload") @ApiOperation(value = "Unload a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "This operation requires super-user access"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -491,6 +495,7 @@ public void getListFromBundle( @ApiOperation(value = "Truncate a topic.", notes = "NonPersistentTopic does not support truncate.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 412, message = "NonPersistentTopic does not support truncate.") }) public void truncateTopic( @@ -514,7 +519,7 @@ protected void validateAdminOperationOnTopic(TopicName topicName, boolean author @GET @Path("/{tenant}/{namespace}/{topic}/entryFilters") - @ApiOperation(value = "Get entry filters for a topic.") + @ApiOperation(value = "Get entry filters for a topic.", response = EntryFilters.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @@ -542,7 +547,9 @@ public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Set entry filters for specified topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -570,7 +577,9 @@ public void setEntryFilters(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Remove entry filters for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 94fb1f53ac710..e619769d81ea0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -49,7 +49,10 @@ import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; +import org.apache.pulsar.client.admin.OffloadProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.ResetCursorData; @@ -162,7 +165,10 @@ public void getPartitionedTopicList( @ApiOperation(value = "Get permissions on a topic.", notes = "Retrieve the effective permissions for a topic." + " These permissions are defined by the permissions set at the" - + "namespace level combined (union) with any eventual specific permission set on the topic.") + + "namespace level combined (union) with any eventual specific permission set on the topic." + + "Returns a nested map structure which Swagger does not fully support for display. " + + "Structure: Map>. Please refer to this structure for details.", + response = AuthAction.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -195,6 +201,7 @@ public void getPermissionsOnTopic( @Path("/{tenant}/{namespace}/{topic}/permissions/{role}") @ApiOperation(value = "Grant a new permission to a role on a single topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -232,6 +239,7 @@ public void grantPermissionsOnTopic( + "level, but rather at the namespace level," + " this operation will return an error (HTTP status code 412).") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -263,6 +271,7 @@ public void revokePermissionsOnTopic( @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -305,6 +314,7 @@ public void createPartitionedTopic( @ApiOperation(value = "Create a non-partitioned topic.", notes = "This is the only REST endpoint from which non-partitioned topics could be created.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), @@ -344,7 +354,7 @@ public void createNonPartitionedTopic( @GET @Path("/{tenant}/{namespace}/{topic}/offloadPolicies") - @ApiOperation(value = "Get offload policies on a topic.") + @ApiOperation(value = "Get offload policies on a topic.", response = OffloadPoliciesImpl.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 500, message = "Internal server error"), }) @@ -370,7 +380,9 @@ public void getOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/offloadPolicies") @ApiOperation(value = "Set offload policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -394,7 +406,9 @@ public void setOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/offloadPolicies") @ApiOperation(value = "Delete offload policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void removeOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -441,7 +455,9 @@ public void getMaxUnackedMessagesOnConsumer(@Suspended final AsyncResponse async @POST @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnConsumer") @ApiOperation(value = "Set max unacked messages per consumer config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setMaxUnackedMessagesOnConsumer( @Suspended final AsyncResponse asyncResponse, @@ -467,7 +483,9 @@ public void setMaxUnackedMessagesOnConsumer( @DELETE @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnConsumer") @ApiOperation(value = "Delete max unacked messages per consumer config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteMaxUnackedMessagesOnConsumer(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -517,7 +535,9 @@ public void getDeduplicationSnapshotInterval(@Suspended final AsyncResponse asyn @POST @Path("/{tenant}/{namespace}/{topic}/deduplicationSnapshotInterval") @ApiOperation(value = "Set deduplicationSnapshotInterval config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setDeduplicationSnapshotInterval( @Suspended final AsyncResponse asyncResponse, @@ -543,7 +563,9 @@ public void setDeduplicationSnapshotInterval( @DELETE @Path("/{tenant}/{namespace}/{topic}/deduplicationSnapshotInterval") @ApiOperation(value = "Delete deduplicationSnapshotInterval config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteDeduplicationSnapshotInterval(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -590,7 +612,9 @@ public void getInactiveTopicPolicies(@Suspended final AsyncResponse asyncRespons @POST @Path("/{tenant}/{namespace}/{topic}/inactiveTopicPolicies") @ApiOperation(value = "Set inactive topic policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setInactiveTopicPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -615,7 +639,9 @@ public void setInactiveTopicPolicies(@Suspended final AsyncResponse asyncRespons @DELETE @Path("/{tenant}/{namespace}/{topic}/inactiveTopicPolicies") @ApiOperation(value = "Delete inactive topic policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteInactiveTopicPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -663,7 +689,9 @@ public void getMaxUnackedMessagesOnSubscription(@Suspended final AsyncResponse a @POST @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnSubscription") @ApiOperation(value = "Set max unacked messages per subscription config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setMaxUnackedMessagesOnSubscription( @Suspended final AsyncResponse asyncResponse, @@ -691,7 +719,9 @@ public void setMaxUnackedMessagesOnSubscription( @DELETE @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnSubscription") @ApiOperation(value = "Delete max unacked messages per subscription config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteMaxUnackedMessagesOnSubscription(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -739,7 +769,9 @@ public void getDelayedDeliveryPolicies(@Suspended final AsyncResponse asyncRespo @POST @Path("/{tenant}/{namespace}/{topic}/delayedDelivery") @ApiOperation(value = "Set delayed delivery messages config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setDelayedDeliveryPolicies( @Suspended final AsyncResponse asyncResponse, @@ -768,7 +800,9 @@ public void setDelayedDeliveryPolicies( @DELETE @Path("/{tenant}/{namespace}/{topic}/delayedDelivery") @ApiOperation(value = "Set delayed delivery messages config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteDelayedDeliveryPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -851,6 +885,7 @@ public void updatePartitionedTopic( @Path("/{tenant}/{namespace}/{topic}/createMissedPartitions") @ApiOperation(value = "Create missed partitions of an existing partitioned topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @@ -951,6 +986,7 @@ public void getProperties( @Path("/{tenant}/{namespace}/{topic}/properties") @ApiOperation(value = "Update the properties on the given topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -987,6 +1023,7 @@ public void updateProperties( @Path("/{tenant}/{namespace}/{topic}/properties") @ApiOperation(value = "Remove the key in properties on the given topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1023,6 +1060,7 @@ public void removeProperties( @ApiOperation(value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1063,6 +1101,7 @@ public void deletePartitionedTopic( @Path("/{tenant}/{namespace}/{topic}/unload") @ApiOperation(value = "Unload a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic does not exist"), @@ -1097,6 +1136,7 @@ public void unloadTopic( + "subscription or producer connected to the it. " + "Force delete ignores connected clients and deletes topic by explicitly closing them.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1360,6 +1400,7 @@ public void getPartitionedStatsInternal( + " there are any active consumers attached to it. " + "Force delete ignores connected consumers and deletes subscription by explicitly closing them.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1412,6 +1453,7 @@ public void deleteSubscription( @ApiOperation(value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1447,6 +1489,7 @@ public void skipAllMessages( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/skip/{numMessages}") @ApiOperation(value = "Skipping messages on a topic subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1483,6 +1526,7 @@ public void skipMessages( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/expireMessages/{expireTimeInSeconds}") @ApiOperation(value = "Expiry messages on a topic subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1520,6 +1564,7 @@ public void expireTopicMessages( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/expireMessages") @ApiOperation(value = "Expiry messages on a topic subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1559,6 +1604,7 @@ public void expireTopicMessages( @Path("/{tenant}/{namespace}/{topic}/all_subscription/expireMessages/{expireTimeInSeconds}") @ApiOperation(value = "Expiry messages on all subscriptions of topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1595,6 +1641,7 @@ public void expireMessagesForAllSubscriptions( @ApiOperation(value = "Create a subscription on the topic.", notes = "Creates a subscription on the topic at the specified message id") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 400, message = "Create subscription on non persistent topic is not supported"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -1650,6 +1697,7 @@ public void createSubscription( @ApiOperation(value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before resetting cursor.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1700,6 +1748,7 @@ public void resetCursor( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/properties") @ApiOperation(value = "Replace all the properties on the given subscription") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1735,7 +1784,8 @@ public void updateSubscriptionProperties( @GET @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/properties") - @ApiOperation(value = "Return all the properties on the given subscription") + @ApiOperation(value = "Return all the properties on the given subscription", + response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -1773,6 +1823,7 @@ public void getSubscriptionProperties( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/analyzeBacklog") @ApiOperation(value = "Analyse a subscription, by scanning all the unprocessed messages") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1819,6 +1870,7 @@ public void analyzeSubscriptionBacklog( @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before resetting cursor.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1857,6 +1909,13 @@ public void resetCursorOnPosition( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/position/{messagePosition}") @ApiOperation(value = "Peek nth message on a topic subscription.") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Successfully retrieved the message. The response is a binary byte stream " + + "containing the message data. Clients need to parse this binary stream based" + + " on the message metadata provided in the response headers.", + response = byte[].class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1899,6 +1958,13 @@ public void peekNthMessage( @ApiOperation(value = "Examine a specific message on a topic by position relative to the earliest or the latest message.") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Successfully retrieved the message. The response is a binary byte stream " + + "containing the message data. Clients need to parse this binary stream based" + + " on the message metadata provided in the response headers.", + response = byte[].class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic, the message position does not exist"), @@ -1941,6 +2007,13 @@ public void examineMessage( @Path("/{tenant}/{namespace}/{topic}/ledger/{ledgerId}/entry/{entryId}") @ApiOperation(value = "Get message by its messageId.") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Successfully retrieved the message. The response is a binary byte stream " + + "containing the message data. Clients need to parse this binary stream based" + + " on the message metadata provided in the response headers.", + response = byte[].class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1981,7 +2054,8 @@ public void getMessageById( @GET @Path("/{tenant}/{namespace}/{topic}/messageid/{timestamp}") - @ApiOperation(value = "Get message ID published at or just after this absolute timestamp (in ms).") + @ApiOperation(value = "Get message ID published at or just after this absolute timestamp (in ms).", + response = MessageIdAdv.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -2111,7 +2185,9 @@ public void getBacklogQuotaMap( @POST @Path("/{tenant}/{namespace}/{topic}/backlogQuota") @ApiOperation(value = "Set a backlog quota for a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 405, @@ -2140,7 +2216,9 @@ public void setBacklogQuota( @DELETE @Path("/{tenant}/{namespace}/{topic}/backlogQuota") @ApiOperation(value = "Remove a backlog quota policy from a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2202,7 +2280,9 @@ public void getReplicationClusters(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/replication") @ApiOperation(value = "Set the replication clusters for a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 405, @@ -2228,7 +2308,9 @@ public void setReplicationClusters( @DELETE @Path("/{tenant}/{namespace}/{topic}/replication") @ApiOperation(value = "Remove the replication clusters from a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2287,7 +2369,9 @@ public void getMessageTTL(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/messageTTL") @ApiOperation(value = "Set message TTL in seconds for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Not authenticate to perform the request or policy is read only"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = @@ -2317,6 +2401,7 @@ public void setMessageTTL(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{topic}/messageTTL") @ApiOperation(value = "Remove message TTL in seconds for a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Not authenticate to perform the request or policy is read only"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @@ -2371,7 +2456,9 @@ public void getDeduplication(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/deduplicationEnabled") @ApiOperation(value = "Set deduplication enabled on a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry")}) @@ -2399,7 +2486,9 @@ public void setDeduplication( @DELETE @Path("/{tenant}/{namespace}/{topic}/deduplicationEnabled") @ApiOperation(value = "Remove deduplication configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2452,7 +2541,9 @@ public void getRetention(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/retention") @ApiOperation(value = "Set retention configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2490,7 +2581,9 @@ public void setRetention(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/retention") @ApiOperation(value = "Remove retention configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2552,7 +2645,9 @@ public void getPersistence(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/persistence") @ApiOperation(value = "Set configuration of persistence policies for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2592,7 +2687,9 @@ public void setPersistence(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/persistence") @ApiOperation(value = "Remove configuration of persistence policies for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2651,7 +2748,9 @@ public void getMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/{topic}/maxSubscriptionsPerTopic") @ApiOperation(value = "Set maxSubscriptionsPerTopic config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2684,7 +2783,9 @@ public void setMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResp @DELETE @Path("/{tenant}/{namespace}/{topic}/maxSubscriptionsPerTopic") @ApiOperation(value = "Remove maxSubscriptionsPerTopic config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2741,7 +2842,9 @@ public void getReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @POST @Path("/{tenant}/{namespace}/{topic}/replicatorDispatchRate") @ApiOperation(value = "Set replicatorDispatchRate config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2774,7 +2877,9 @@ public void setReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @DELETE @Path("/{tenant}/{namespace}/{topic}/replicatorDispatchRate") @ApiOperation(value = "Remove replicatorDispatchRate config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2831,7 +2936,9 @@ public void getMaxProducers(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/maxProducers") @ApiOperation(value = "Set maxProducers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2866,7 +2973,9 @@ public void setMaxProducers(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/maxProducers") @ApiOperation(value = "Remove maxProducers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2925,7 +3034,9 @@ public void getMaxConsumers(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/maxConsumers") @ApiOperation(value = "Set maxConsumers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2960,7 +3071,9 @@ public void setMaxConsumers(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/maxConsumers") @ApiOperation(value = "Remove maxConsumers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -3020,7 +3133,9 @@ public void getMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/maxMessageSize") @ApiOperation(value = "Set maxMessageSize config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -3057,7 +3172,9 @@ public void setMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/maxMessageSize") @ApiOperation(value = "Remove maxMessageSize config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -3092,6 +3209,12 @@ public void removeMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @ApiOperation(value = "Terminate a topic. A topic that is terminated will not accept any more " + "messages to be published and will let consumer to drain existing messages in backlog") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Operation terminated successfully. The response includes the 'lastMessageId'," + + " which is the identifier of the last message processed.", + response = MessageIdAdv.class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -3129,6 +3252,7 @@ public void terminate( @ApiOperation(value = "Terminate all partitioned topic. A topic that is terminated will not accept any more " + "messages to be published and will let consumer to drain existing messages in backlog") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -3155,6 +3279,7 @@ public void terminatePartitionedTopic(@Suspended final AsyncResponse asyncRespon @Path("/{tenant}/{namespace}/{topic}/compaction") @ApiOperation(value = "Trigger a compaction operation on a topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -3187,7 +3312,8 @@ public void compact( @GET @Path("/{tenant}/{namespace}/{topic}/compaction") - @ApiOperation(value = "Get the status of a compaction operation for a topic.") + @ApiOperation(value = "Get the status of a compaction operation for a topic.", + response = LongRunningProcessStatus.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3225,6 +3351,7 @@ public void compactionStatus( @Path("/{tenant}/{namespace}/{topic}/offload") @ApiOperation(value = "Offload a prefix of a topic to long term storage") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 400, message = "Message ID is null"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3262,7 +3389,7 @@ public void triggerOffload( @GET @Path("/{tenant}/{namespace}/{topic}/offload") - @ApiOperation(value = "Offload a prefix of a topic to long term storage") + @ApiOperation(value = "Offload a prefix of a topic to long term storage", response = OffloadProcessStatus.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3295,7 +3422,7 @@ public void offloadStatus( @GET @Path("/{tenant}/{namespace}/{topic}/lastMessageId") - @ApiOperation(value = "Return the last commit message id of topic") + @ApiOperation(value = "Return the last commit message id of topic", response = MessageIdAdv.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3328,6 +3455,7 @@ public void getLastMessageId( @Path("/{tenant}/{namespace}/{topic}/trim") @ApiOperation(value = " Trim a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -3364,7 +3492,7 @@ public void trimTopic( @GET @Path("/{tenant}/{namespace}/{topic}/dispatchRate") - @ApiOperation(value = "Get dispatch rate configuration for specified topic.") + @ApiOperation(value = "Get dispatch rate configuration for specified topic.", response = DispatchRateImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, @@ -3392,7 +3520,9 @@ public void getDispatchRate(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/dispatchRate") @ApiOperation(value = "Set message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3430,7 +3560,9 @@ public void setDispatchRate(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/dispatchRate") @ApiOperation(value = "Remove message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3493,7 +3625,9 @@ public void getSubscriptionDispatchRate(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/{topic}/subscriptionDispatchRate") @ApiOperation(value = "Set subscription message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3533,7 +3667,9 @@ public void setSubscriptionDispatchRate( @DELETE @Path("/{tenant}/{namespace}/{topic}/subscriptionDispatchRate") @ApiOperation(value = "Remove subscription message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3565,7 +3701,8 @@ public void removeSubscriptionDispatchRate(@Suspended final AsyncResponse asyncR @GET @Path("/{tenant}/{namespace}/{topic}/{subName}/dispatchRate") - @ApiOperation(value = "Get message dispatch rate configuration for specified subscription.") + @ApiOperation(value = "Get message dispatch rate configuration for specified subscription.", + response = DispatchRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, @@ -3595,7 +3732,9 @@ public void getSubscriptionLevelDispatchRate(@Suspended final AsyncResponse asyn @POST @Path("/{tenant}/{namespace}/{topic}/{subName}/dispatchRate") @ApiOperation(value = "Set message dispatch rate configuration for specified subscription.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3636,7 +3775,9 @@ public void setSubscriptionLevelDispatchRate( @DELETE @Path("/{tenant}/{namespace}/{topic}/{subName}/dispatchRate") @ApiOperation(value = "Remove message dispatch rate configuration for specified subscription.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3697,7 +3838,9 @@ public void getCompactionThreshold(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/compactionThreshold") @ApiOperation(value = "Set compaction threshold configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3735,7 +3878,9 @@ public void setCompactionThreshold(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/compactionThreshold") @ApiOperation(value = "Remove compaction threshold configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3798,7 +3943,9 @@ public void getMaxConsumersPerSubscription(@Suspended final AsyncResponse asyncR @POST @Path("/{tenant}/{namespace}/{topic}/maxConsumersPerSubscription") @ApiOperation(value = "Set max consumers per subscription configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3837,7 +3984,9 @@ public void setMaxConsumersPerSubscription( @DELETE @Path("/{tenant}/{namespace}/{topic}/maxConsumersPerSubscription") @ApiOperation(value = "Remove max consumers per subscription configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3898,7 +4047,9 @@ public void getPublishRate(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/publishRate") @ApiOperation(value = "Set message publish rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3937,7 +4088,9 @@ public void setPublishRate(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/publishRate") @ApiOperation(value = "Remove message publish rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4004,7 +4157,9 @@ public void getSubscriptionTypesEnabled(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/{topic}/subscriptionTypesEnabled") @ApiOperation(value = "Set is enable sub types for specified topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4043,7 +4198,9 @@ public void setSubscriptionTypesEnabled(@Suspended final AsyncResponse asyncResp @DELETE @Path("/{tenant}/{namespace}/{topic}/subscriptionTypesEnabled") @ApiOperation(value = "Remove subscription types enabled for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -4101,7 +4258,9 @@ public void getSubscribeRate(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/subscribeRate") @ApiOperation(value = "Set subscribe rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4141,7 +4300,9 @@ public void setSubscribeRate( @DELETE @Path("/{tenant}/{namespace}/{topic}/subscribeRate") @ApiOperation(value = "Remove subscribe rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4180,6 +4341,7 @@ public void removeSubscribeRate(@Suspended final AsyncResponse asyncResponse, notes = "The truncate operation will move all cursors to the end of the topic " + "and delete all inactive ledgers.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -4215,6 +4377,7 @@ public void truncateTopic( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/replicatedSubscriptionStatus") @ApiOperation(value = "Enable or disable a replicated subscription on a topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or " + "subscriber is not authorized to access this operation"), @@ -4311,6 +4474,7 @@ public void getSchemaCompatibilityStrategy( @Path("/{tenant}/{namespace}/{topic}/schemaCompatibilityStrategy") @ApiOperation(value = "Set schema compatibility strategy on a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 405, message = "Operation not allowed on persistent topic"), @@ -4351,6 +4515,7 @@ public void setSchemaCompatibilityStrategy( @Path("/{tenant}/{namespace}/{topic}/schemaCompatibilityStrategy") @ApiOperation(value = "Remove schema compatibility strategy on a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 405, message = "Operation not allowed on persistent topic"), @@ -4389,7 +4554,7 @@ public void removeSchemaCompatibilityStrategy( @GET @Path("/{tenant}/{namespace}/{topic}/schemaValidationEnforced") - @ApiOperation(value = "Get schema validation enforced flag for topic.") + @ApiOperation(value = "Get schema validation enforced flag for topic.", response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @@ -4417,7 +4582,9 @@ public void getSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/schemaValidationEnforced") @ApiOperation(value = "Set schema validation enforced flag on topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or Namespace doesn't exist"), @ApiResponse(code = 412, message = "schemaValidationEnforced value is not valid")}) public void setSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @@ -4444,7 +4611,7 @@ public void setSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/{topic}/entryFilters") - @ApiOperation(value = "Get entry filters for a topic.") + @ApiOperation(value = "Get entry filters for a topic.", response = EntryFilters.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @@ -4472,7 +4639,9 @@ public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Set entry filters for specified topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4500,7 +4669,9 @@ public void setEntryFilters(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Remove entry filters for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4534,7 +4705,8 @@ public void removeEntryFilters(@Suspended final AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/{topic}/shadowTopics") - @ApiOperation(value = "Get the shadow topic list for a topic") + @ApiOperation(value = "Get the shadow topic list for a topic", + response = String.class, responseContainer = "List") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = @@ -4561,7 +4733,9 @@ public void getShadowTopics( @PUT @Path("/{tenant}/{namespace}/{topic}/shadowTopics") @ApiOperation(value = "Set shadow topic list for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, enable the topic level policy and retry"), @@ -4588,7 +4762,9 @@ public void setShadowTopics( @DELETE @Path("/{tenant}/{namespace}/{topic}/shadowTopics") @ApiOperation(value = "Delete shadow topics for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, enable the topic level policy and retry"), @@ -4614,7 +4790,9 @@ public void deleteShadowTopics( @POST @Path("/{tenant}/{namespace}/{topic}/autoSubscriptionCreation") @ApiOperation(value = "Override namespace's allowAutoSubscriptionCreation setting for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, enable the topic level policy and retry"), @@ -4641,7 +4819,8 @@ public void setAutoSubscriptionCreation( @GET @Path("/{tenant}/{namespace}/{topic}/autoSubscriptionCreation") - @ApiOperation(value = "Get autoSubscriptionCreation info in a topic") + @ApiOperation(value = "Get autoSubscriptionCreation info in a topic", + response = AutoSubscriptionCreationOverrideImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic does not exist"), @ApiResponse(code = 405, @@ -4668,7 +4847,9 @@ public void getAutoSubscriptionCreation( @DELETE @Path("/{tenant}/{namespace}/{topic}/autoSubscriptionCreation") @ApiOperation(value = "Remove autoSubscriptionCreation ina a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic does not exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java index 52fd03b18ed0b..58f593e20ce3b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java @@ -60,7 +60,9 @@ public ResourceGroup getResourceGroup(@PathParam("resourcegroup") String resourc @PUT @Path("/{resourcegroup}") @ApiOperation(value = "Creates a new resourcegroup with the specified rate limiters") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "cluster doesn't exist")}) public void createOrUpdateResourceGroup(@PathParam("resourcegroup") String name, @ApiParam(value = "Rate limiters for the resourcegroup") @@ -72,6 +74,7 @@ public void createOrUpdateResourceGroup(@PathParam("resourcegroup") String name, @Path("/{resourcegroup}") @ApiOperation(value = "Delete a resourcegroup.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "ResourceGroup doesn't exist"), @ApiResponse(code = 409, message = "ResourceGroup is in use")}) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java index 58ccc1c10288c..d2884e8ea6f7e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java @@ -75,7 +75,7 @@ public void setDefaultResourceQuota( @GET @Path("/{tenant}/{namespace}/{bundle}") - @ApiOperation(value = "Get resource quota of a namespace bundle.") + @ApiOperation(value = "Get resource quota of a namespace bundle.", response = ResourceQuota.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -103,6 +103,7 @@ public void getNamespaceBundleResourceQuota( @Path("/{tenant}/{namespace}/{bundle}") @ApiOperation(value = "Set resource quota on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -133,6 +134,7 @@ public void setNamespaceBundleResourceQuota( @Path("/{tenant}/{namespace}/{bundle}") @ApiOperation(value = "Remove resource quota for a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 409, message = "Concurrent modification") }) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java index 3813790e4f428..7178b565719ca 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java @@ -87,7 +87,9 @@ public WorkerInfo getClusterLeader() { @GET @ApiOperation( value = "Fetches information about which Pulsar Functions are assigned to which Pulsar clusters", - response = Map.class + response = Map.class, + notes = "Returns a nested map structure which Swagger does not fully support for display." + + "Structure: Map>. Please refer to this structure for details." ) @ApiResponses(value = { @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @@ -102,7 +104,8 @@ public Map> getAssignments() { @GET @ApiOperation( value = "Fetches a list of supported Pulsar IO connectors currently running in cluster mode", - response = List.class + response = ConnectorDefinition.class, + responseContainer = "List" ) @ApiResponses(value = { @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @@ -120,6 +123,7 @@ public List getConnectorsList() throws IOException { value = "Triggers a rebalance of functions to workers" ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 408, message = "Request timeout") @@ -134,6 +138,7 @@ public void rebalance() { value = "Drains the specified worker, i.e., moves its work-assignments to other workers" ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 408, message = "Request timeout"), @@ -150,6 +155,7 @@ public void drainAtLeader(@QueryParam("workerId") String workerId) { value = "Drains this worker, i.e., moves its work-assignments to other workers" ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 408, message = "Request timeout"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java index 15e7b69554dc7..4ca7e3948ff5a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java @@ -80,7 +80,7 @@ public void getMeta( ) @ApiResponses( value = { - @ApiResponse(code = 200, message = "Update the metadata of the specified package successfully."), + @ApiResponse(code = 204, message = "Update the metadata of the specified package successfully."), @ApiResponse(code = 404, message = "The specified package is not existent."), @ApiResponse(code = 412, message = "The package name is illegal."), @ApiResponse(code = 500, message = "Internal server error."), @@ -113,7 +113,7 @@ public void updateMeta( ) @ApiResponses( value = { - @ApiResponse(code = 200, message = "Upload the specified package successfully."), + @ApiResponse(code = 204, message = "Upload the specified package successfully."), @ApiResponse(code = 412, message = "The package name is illegal."), @ApiResponse(code = 500, message = "Internal server error."), @ApiResponse(code = 503, message = "Package Management Service is not enabled in the broker.") @@ -169,7 +169,7 @@ public StreamingOutput download( @Path("/{type}/{tenant}/{namespace}/{packageName}/{version}") @ApiResponses( value = { - @ApiResponse(code = 200, message = "Delete the specified package successfully."), + @ApiResponse(code = 204, message = "Delete the specified package successfully."), @ApiResponse(code = 404, message = "The specified package is not existent."), @ApiResponse(code = 412, message = "The package name is illegal."), @ApiResponse(code = 500, message = "Internal server error."), @@ -218,7 +218,8 @@ public void listPackageVersion( @Path("/{type}/{tenant}/{namespace}") @ApiOperation( value = "Get all the specified type packages in a namespace.", - response = PackageMetadata.class + response = PackageMetadata.class, + responseContainer = "List" ) @ApiResponses( value = { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java index b2a3ae8cdb935..fdfbacb65d121 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java @@ -43,6 +43,16 @@ import org.apache.pulsar.broker.admin.impl.TransactionsBase; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.common.policies.data.TransactionBufferStats; +import org.apache.pulsar.common.policies.data.TransactionCoordinatorInfo; +import org.apache.pulsar.common.policies.data.TransactionCoordinatorInternalStats; +import org.apache.pulsar.common.policies.data.TransactionCoordinatorStats; +import org.apache.pulsar.common.policies.data.TransactionInBufferStats; +import org.apache.pulsar.common.policies.data.TransactionInPendingAckStats; +import org.apache.pulsar.common.policies.data.TransactionMetadata; +import org.apache.pulsar.common.policies.data.TransactionPendingAckInternalStats; +import org.apache.pulsar.common.policies.data.TransactionPendingAckStats; +import org.apache.pulsar.common.stats.PositionInPendingAckStats; import org.apache.pulsar.common.util.FutureUtil; import org.jetbrains.annotations.Nullable; @@ -55,7 +65,8 @@ public class Transactions extends TransactionsBase { @GET @Path("/coordinators") - @ApiOperation(value = "List transaction coordinators.") + @ApiOperation(value = "List transaction coordinators.", + response = TransactionCoordinatorInfo.class, responseContainer = "List") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 503, message = "This Broker is not " + "configured with transactionCoordinatorEnabled=true.")}) @@ -66,7 +77,7 @@ public void listCoordinators(@Suspended final AsyncResponse asyncResponse) { @GET @Path("/coordinatorStats") - @ApiOperation(value = "Get transaction coordinator stats.") + @ApiOperation(value = "Get transaction coordinator stats.", response = TransactionCoordinatorStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 503, message = "This Broker is not " + "configured with transactionCoordinatorEnabled=true."), @@ -82,7 +93,7 @@ public void getCoordinatorStats(@Suspended final AsyncResponse asyncResponse, @GET @Path("/transactionInBufferStats/{tenant}/{namespace}/{topic}/{mostSigBits}/{leastSigBits}") - @ApiOperation(value = "Get transaction state in transaction buffer.") + @ApiOperation(value = "Get transaction state in transaction buffer.", response = TransactionInBufferStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -119,7 +130,7 @@ public void getTransactionInBufferStats(@Suspended final AsyncResponse asyncResp @GET @Path("/transactionInPendingAckStats/{tenant}/{namespace}/{topic}/{subName}/{mostSigBits}/{leastSigBits}") - @ApiOperation(value = "Get transaction state in pending ack.") + @ApiOperation(value = "Get transaction state in pending ack.", response = TransactionInPendingAckStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -157,7 +168,7 @@ public void getTransactionInPendingAckStats(@Suspended final AsyncResponse async @GET @Path("/transactionBufferStats/{tenant}/{namespace}/{topic}") - @ApiOperation(value = "Get transaction buffer stats in topic.") + @ApiOperation(value = "Get transaction buffer stats in topic.", response = TransactionBufferStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -195,7 +206,7 @@ public void getTransactionBufferStats(@Suspended final AsyncResponse asyncRespon @GET @Path("/pendingAckStats/{tenant}/{namespace}/{topic}/{subName}") - @ApiOperation(value = "Get transaction pending ack stats in topic.") + @ApiOperation(value = "Get transaction pending ack stats in topic.", response = TransactionPendingAckStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic or subName doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -231,7 +242,7 @@ public void getPendingAckStats(@Suspended final AsyncResponse asyncResponse, @GET @Path("/transactionMetadata/{mostSigBits}/{leastSigBits}") - @ApiOperation(value = "Get transaction metadata") + @ApiOperation(value = "Get transaction metadata", response = TransactionMetadata.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or coordinator or transaction doesn't exist"), @@ -252,7 +263,7 @@ public void getTransactionMetadata(@Suspended final AsyncResponse asyncResponse, @GET @Path("/slowTransactions/{timeout}") - @ApiOperation(value = "Get slow transactions.") + @ApiOperation(value = "Get slow transactions.", response = TransactionMetadata.class, responseContainer = "Map") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or coordinator or transaction doesn't exist"), @@ -272,7 +283,7 @@ public void getSlowTransactions(@Suspended final AsyncResponse asyncResponse, @GET @Path("/coordinatorInternalStats/{coordinatorId}") - @ApiOperation(value = "Get coordinator internal stats.") + @ApiOperation(value = "Get coordinator internal stats.", response = TransactionCoordinatorInternalStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 503, message = "This Broker is not " + "configured with transactionCoordinatorEnabled=true."), @@ -290,7 +301,8 @@ public void getCoordinatorInternalStats(@Suspended final AsyncResponse asyncResp @GET @Path("/pendingAckInternalStats/{tenant}/{namespace}/{topic}/{subName}") - @ApiOperation(value = "Get transaction pending ack internal stats.") + @ApiOperation(value = "Get transaction pending ack internal stats.", + response = TransactionPendingAckInternalStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or subscription name doesn't exist"), @@ -379,6 +391,7 @@ public void getTransactionBufferInternalStats(@Suspended final AsyncResponse asy @POST @Path("/transactionCoordinator/replicas") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 503, message = "This Broker is not configured " + "with transactionCoordinatorEnabled=true."), @ApiResponse(code = 406, message = "The number of replicas should be more than " @@ -401,7 +414,7 @@ public void scaleTransactionCoordinators(@Suspended final AsyncResponse asyncRes @GET @Path("/positionStatsInPendingAck/{tenant}/{namespace}/{topic}/{subName}/{ledgerId}/{entryId}") - @ApiOperation(value = "Get position stats in pending ack.") + @ApiOperation(value = "Get position stats in pending ack.", response = PositionInPendingAckStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or subscription name doesn't exist"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java index faee5799289d0..3997e214f4316 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java @@ -29,7 +29,6 @@ import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; -import org.apache.pulsar.broker.ClassLoaderSwitcher; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Producer; @@ -51,16 +50,20 @@ public class BrokerInterceptorWithClassLoader implements BrokerInterceptor { private final BrokerInterceptor interceptor; - private final NarClassLoader classLoader; + private final NarClassLoader narClassLoader; @Override public void beforeSendMessage(Subscription subscription, Entry entry, long[] ackSet, MessageMetadata msgMetadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.beforeSendMessage( subscription, entry, ackSet, msgMetadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -70,25 +73,37 @@ public void beforeSendMessage(Subscription subscription, long[] ackSet, MessageMetadata msgMetadata, Consumer consumer) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.beforeSendMessage( subscription, entry, ackSet, msgMetadata, consumer); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onMessagePublish(Producer producer, ByteBuf headersAndPayload, Topic.PublishContext publishContext) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onMessagePublish(producer, headersAndPayload, publishContext); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void producerCreated(ServerCnx cnx, Producer producer, Map metadata){ - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.producerCreated(cnx, producer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -96,8 +111,12 @@ public void producerCreated(ServerCnx cnx, Producer producer, public void producerClosed(ServerCnx cnx, Producer producer, Map metadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.producerClosed(cnx, producer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -105,9 +124,12 @@ public void producerClosed(ServerCnx cnx, public void consumerCreated(ServerCnx cnx, Consumer consumer, Map metadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { - this.interceptor.consumerCreated( - cnx, consumer, metadata); + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); + this.interceptor.consumerCreated(cnx, consumer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -115,8 +137,12 @@ public void consumerCreated(ServerCnx cnx, public void consumerClosed(ServerCnx cnx, Consumer consumer, Map metadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.consumerClosed(cnx, consumer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -124,87 +150,140 @@ public void consumerClosed(ServerCnx cnx, @Override public void messageProduced(ServerCnx cnx, Producer producer, long startTimeNs, long ledgerId, long entryId, Topic.PublishContext publishContext) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.messageProduced(cnx, producer, startTimeNs, ledgerId, entryId, publishContext); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void messageDispatched(ServerCnx cnx, Consumer consumer, long ledgerId, long entryId, ByteBuf headersAndPayload) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.messageDispatched(cnx, consumer, ledgerId, entryId, headersAndPayload); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void messageAcked(ServerCnx cnx, Consumer consumer, CommandAck ackCmd) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.messageAcked(cnx, consumer, ackCmd); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void txnOpened(long tcId, String txnID) { - this.interceptor.txnOpened(tcId, txnID); + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); + this.interceptor.txnOpened(tcId, txnID); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); + } } @Override public void txnEnded(String txnID, long txnAction) { - this.interceptor.txnEnded(txnID, txnAction); + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); + this.interceptor.txnEnded(txnID, txnAction); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); + } } @Override public void onConnectionCreated(ServerCnx cnx) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onConnectionCreated(cnx); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onPulsarCommand(BaseCommand command, ServerCnx cnx) throws InterceptException { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onPulsarCommand(command, cnx); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onConnectionClosed(ServerCnx cnx) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onConnectionClosed(cnx); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onWebserviceRequest(ServletRequest request) throws IOException, ServletException, InterceptException { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onWebserviceRequest(request); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onWebserviceResponse(ServletRequest request, ServletResponse response) throws IOException, ServletException { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onWebserviceResponse(request, response); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void initialize(PulsarService pulsarService) throws Exception { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.initialize(pulsarService); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void close() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); interceptor.close(); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } + try { - classLoader.close(); + narClassLoader.close(); } catch (IOException e) { log.warn("Failed to close the broker interceptor class loader", e); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 26ee45b7444f3..855e8ccfcaaf3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -195,13 +195,14 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { /** * Get all the bundles that are owned by this broker. */ - public Set getOwnedServiceUnits() { + public CompletableFuture> getOwnedServiceUnitsAsync() { if (!started) { log.warn("Failed to get owned service units, load manager is not started."); - return Collections.emptySet(); + return CompletableFuture.completedFuture(Collections.emptySet()); } - Set> entrySet = serviceUnitStateChannel.getOwnershipEntrySet(); + String brokerId = brokerRegistry.getBrokerId(); + Set> entrySet = serviceUnitStateChannel.getOwnershipEntrySet(); Set ownedServiceUnits = entrySet.stream() .filter(entry -> { var stateData = entry.getValue(); @@ -214,34 +215,26 @@ public Set getOwnedServiceUnits() { }).collect(Collectors.toSet()); // Add heartbeat and SLA monitor namespace bundle. NamespaceName heartbeatNamespace = NamespaceService.getHeartbeatNamespace(brokerId, pulsar.getConfiguration()); - try { - NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundle(heartbeatNamespace); - ownedServiceUnits.add(fullBundle); - } catch (Exception e) { - log.warn("Failed to get heartbeat namespace bundle.", e); - } NamespaceName heartbeatNamespaceV2 = NamespaceService .getHeartbeatNamespaceV2(brokerId, pulsar.getConfiguration()); - try { - NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundle(heartbeatNamespaceV2); - ownedServiceUnits.add(fullBundle); - } catch (Exception e) { - log.warn("Failed to get heartbeat namespace V2 bundle.", e); - } - NamespaceName slaMonitorNamespace = NamespaceService .getSLAMonitorNamespace(brokerId, pulsar.getConfiguration()); - try { - NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundle(slaMonitorNamespace); - ownedServiceUnits.add(fullBundle); - } catch (Exception e) { - log.warn("Failed to get SLA Monitor namespace bundle.", e); - } - - return ownedServiceUnits; + return pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(heartbeatNamespace) + .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { + log.warn("Failed to get heartbeat namespace bundle.", e); + return null; + }).thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(heartbeatNamespaceV2)) + .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { + log.warn("Failed to get heartbeat namespace V2 bundle.", e); + return null; + }).thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(slaMonitorNamespace)) + .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { + log.warn("Failed to get SLA Monitor namespace bundle.", e); + return null; + }).thenApply(__ -> ownedServiceUnits); } public enum Role { 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 a15dfce2dcf0a..626a187f46388 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 @@ -801,12 +801,12 @@ public CompletableFuture> getOwnedNameSpac if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); - var statusMap = extensibleLoadManager.getOwnedServiceUnits().stream() - .collect(Collectors.toMap(NamespaceBundle::toString, - bundle -> getNamespaceOwnershipStatus(true, - namespaceIsolationPolicies.getPolicyByNamespace( - bundle.getNamespaceObject())))); - return CompletableFuture.completedFuture(statusMap); + return extensibleLoadManager.getOwnedServiceUnitsAsync() + .thenApply(OwnedServiceUnits -> OwnedServiceUnits.stream() + .collect(Collectors.toMap(NamespaceBundle::toString, + bundle -> getNamespaceOwnershipStatus(true, + namespaceIsolationPolicies.getPolicyByNamespace( + bundle.getNamespaceObject()))))); } Collection> futures = ownershipCache.getOwnedBundlesAsync().values(); @@ -1122,7 +1122,12 @@ public OwnershipCache getOwnershipCache() { public Set getOwnedServiceUnits() { if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); - return extensibleLoadManager.getOwnedServiceUnits(); + try { + return extensibleLoadManager.getOwnedServiceUnitsAsync() + .get(config.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } } return ownershipCache.getOwnedBundles().values().stream().map(OwnedBundle::getNamespaceBundle) .collect(Collectors.toSet()); @@ -1308,42 +1313,40 @@ public CompletableFuture> getOwnedTopicListForNamespaceBundle(Names } public CompletableFuture checkTopicExists(TopicName topic) { - if (topic.isPersistent()) { - if (topic.isPartitioned()) { - return pulsar.getBrokerService() - .fetchPartitionedTopicMetadataAsync(TopicName.get(topic.getPartitionedTopicName())) - .thenCompose(metadata -> { - // Allow creating the non-partitioned persistent topic that name includes `-partition-` - if (metadata.partitions == 0 - || topic.getPartitionIndex() < metadata.partitions) { - return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); - } - return CompletableFuture.completedFuture(false); - }); - } else { - return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); - } + CompletableFuture future; + // If the topic is persistent and the name includes `-partition-`, find the topic from the managed/ledger. + if (topic.isPersistent() && topic.isPartitioned()) { + future = pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); } else { - if (topic.isPartitioned()) { - final TopicName partitionedTopicName = TopicName.get(topic.getPartitionedTopicName()); - return pulsar.getBrokerService() - .fetchPartitionedTopicMetadataAsync(partitionedTopicName) - .thenApply((metadata) -> topic.getPartitionIndex() < metadata.partitions); - } else { - // only checks and don't do any topic creating and loading. - CompletableFuture> topicFuture = - pulsar.getBrokerService().getTopics().get(topic.toString()); - if (topicFuture == null) { - return CompletableFuture.completedFuture(false); - } else { - return topicFuture.thenApply(Optional::isPresent).exceptionally(throwable -> { - LOG.warn("[{}] topicFuture completed with exception when checkTopicExists, {}", - topic, throwable.getMessage()); - return false; - }); - } - } + future = CompletableFuture.completedFuture(false); } + + return future.thenCompose(found -> { + if (found != null && found) { + return CompletableFuture.completedFuture(true); + } + + return pulsar.getBrokerService() + .fetchPartitionedTopicMetadataAsync(TopicName.get(topic.getPartitionedTopicName())) + .thenCompose(metadata -> { + if (metadata.partitions > 0) { + return CompletableFuture.completedFuture(true); + } + + if (topic.isPersistent()) { + return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); + } else { + // The non-partitioned non-persistent topic only exist in the broker topics. + CompletableFuture> nonPersistentTopicFuture = + pulsar.getBrokerService().getTopics().get(topic.toString()); + if (nonPersistentTopicFuture == null) { + return CompletableFuture.completedFuture(false); + } else { + return nonPersistentTopicFuture.thenApply(Optional::isPresent); + } + } + }); + }); } public CompletableFuture> getListOfTopics(NamespaceName namespaceName, Mode mode) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java index d648c261403d4..eb4bcb0a9bf4b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java @@ -26,7 +26,6 @@ import lombok.Data; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.ClassLoaderSwitcher; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.common.nar.NarClassLoader; @@ -44,52 +43,79 @@ class ProtocolHandlerWithClassLoader implements ProtocolHandler { @Override public String protocolName() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.protocolName(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public boolean accept(String protocol) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.accept(protocol); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void initialize(ServiceConfiguration conf) throws Exception { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); handler.initialize(conf); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public String getProtocolDataToAdvertise() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.getProtocolDataToAdvertise(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void start(BrokerService service) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); handler.start(service); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public Map> newChannelInitializers() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.newChannelInitializers(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void close() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); handler.close(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } - try { classLoader.close(); } catch (IOException e) { 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 6ad1697adfc39..c889062088e00 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 @@ -18,10 +18,12 @@ */ package org.apache.pulsar.broker.service; +import static java.util.concurrent.TimeUnit.SECONDS; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursor.IndividualDeletedEntries; @@ -32,6 +34,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics.BacklogQuotaMetrics; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; @@ -41,6 +44,7 @@ @Slf4j public class BacklogQuotaManager { + @Getter private final BacklogQuotaImpl defaultQuota; private final NamespaceResources namespaceResources; @@ -55,10 +59,6 @@ public BacklogQuotaManager(PulsarService pulsar) { this.namespaceResources = pulsar.getPulsarResources().getNamespaceResources(); } - public BacklogQuotaImpl getDefaultQuota() { - return this.defaultQuota; - } - public BacklogQuotaImpl getBacklogQuota(NamespaceName namespace, BacklogQuotaType backlogQuotaType) { try { if (namespaceResources == null) { @@ -86,30 +86,34 @@ public BacklogQuotaImpl getBacklogQuota(NamespaceName namespace, BacklogQuotaTyp public void handleExceededBacklogQuota(PersistentTopic persistentTopic, BacklogQuotaType backlogQuotaType, boolean preciseTimeBasedBacklogQuotaCheck) { BacklogQuota quota = persistentTopic.getBacklogQuota(backlogQuotaType); + BacklogQuotaMetrics topicBacklogQuotaMetrics = + persistentTopic.getPersistentTopicMetrics().getBacklogQuotaMetrics(); log.info("Backlog quota type {} exceeded for topic [{}]. Applying [{}] policy", backlogQuotaType, persistentTopic.getName(), quota.getPolicy()); switch (quota.getPolicy()) { - case consumer_backlog_eviction: - switch (backlogQuotaType) { - case destination_storage: + case consumer_backlog_eviction: + switch (backlogQuotaType) { + case destination_storage: dropBacklogForSizeLimit(persistentTopic, quota); + topicBacklogQuotaMetrics.recordSizeBasedBacklogEviction(); break; - case message_age: + case message_age: dropBacklogForTimeLimit(persistentTopic, quota, preciseTimeBasedBacklogQuotaCheck); + topicBacklogQuotaMetrics.recordTimeBasedBacklogEviction(); break; - default: - break; - } - break; - case producer_exception: - case producer_request_hold: - if (!advanceSlowestSystemCursor(persistentTopic)) { - // The slowest is not a system cursor. Disconnecting producers to put backpressure. - disconnectProducers(persistentTopic); - } - break; - default: - break; + default: + break; + } + break; + case producer_exception: + case producer_request_hold: + if (!advanceSlowestSystemCursor(persistentTopic)) { + // The slowest is not a system cursor. Disconnecting producers to put backpressure. + disconnectProducers(persistentTopic); + } + break; + default: + break; } } @@ -210,7 +214,7 @@ private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuo ); } else { // If disabled precise time based backlog quota check, will try to remove whole ledger from cursor's backlog - Long currentMillis = ((ManagedLedgerImpl) persistentTopic.getManagedLedger()).getClock().millis(); + long currentMillis = ((ManagedLedgerImpl) persistentTopic.getManagedLedger()).getClock().millis(); ManagedLedgerImpl mLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); try { for (; ; ) { @@ -229,7 +233,7 @@ private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuo } // Timestamp only > 0 if ledger has been closed if (ledgerInfo.getTimestamp() > 0 - && currentMillis - ledgerInfo.getTimestamp() > quota.getLimitTime() * 1000) { + && currentMillis - ledgerInfo.getTimestamp() > SECONDS.toMillis(quota.getLimitTime())) { // skip whole ledger for the slowest cursor PositionImpl nextPosition = PositionImpl.get(mLedger.getNextValidLedger(ledgerInfo.getLedgerId()), -1); @@ -263,19 +267,20 @@ private void disconnectProducers(PersistentTopic persistentTopic) { futures.add(producer.disconnect()); }); - FutureUtil.waitForAll(futures).thenRun(() -> { - log.info("All producers on topic [{}] are disconnected", persistentTopic.getName()); - }).exceptionally(exception -> { - log.error("Error in disconnecting producers on topic [{}] [{}]", persistentTopic.getName(), exception); - return null; - + FutureUtil.waitForAll(futures) + .thenRun(() -> + log.info("All producers on topic [{}] are disconnected", persistentTopic.getName())) + .exceptionally(exception -> { + log.error("Error in disconnecting producers on topic [{}] [{}]", persistentTopic.getName(), + exception); + return null; }); } /** * Advances the slowest cursor if that is a system cursor. * - * @param persistentTopic + * @param persistentTopic Persistent topic * @return true if the slowest cursor is a system cursor */ private boolean advanceSlowestSystemCursor(PersistentTopic persistentTopic) { 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 aca9bd1a22806..45f9bdd7b0f7e 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 @@ -20,9 +20,12 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.bookkeeper.mledger.ManagedLedgerConfig.PROPERTY_SOURCE_TOPIC_KEY; import static org.apache.commons.collections4.CollectionUtils.isEmpty; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.pulsar.client.util.RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX; +import static org.apache.pulsar.client.util.RetryMessageUtil.RETRY_GROUP_TOPIC_SUFFIX; import static org.apache.pulsar.common.naming.SystemTopicNames.isTransactionInternalName; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Queues; @@ -36,6 +39,7 @@ import io.netty.channel.socket.SocketChannel; import io.netty.handler.ssl.SslContext; import io.netty.util.concurrent.DefaultThreadFactory; +import io.prometheus.client.Histogram; import java.io.Closeable; import java.io.IOException; import java.lang.reflect.Field; @@ -196,6 +200,12 @@ public class BrokerService implements Closeable { private static final double GRACEFUL_SHUTDOWN_QUIET_PERIOD_RATIO_OF_TOTAL_TIMEOUT = 0.25d; private static final double GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT = 0.5d; + private static final Histogram backlogQuotaCheckDuration = Histogram.build() + .name("pulsar_storage_backlog_quota_check_duration_seconds") + .help("The duration of the backlog quota check process.") + .buckets(5, 10, 30, 60, 300) + .register(); + private final PulsarService pulsar; private final ManagedLedgerFactory managedLedgerFactory; @@ -928,7 +938,7 @@ CompletableFuture shutdownEventLoopGracefully(EventLoopGroup eventLoopGrou long timeout = (long) (GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT * brokerShutdownTimeoutMs); return NettyFutureUtil.toCompletableFutureVoid( eventLoopGroup.shutdownGracefully(quietPeriod, - timeout, TimeUnit.MILLISECONDS)); + timeout, MILLISECONDS)); } private CompletableFuture closeChannel(Channel channel) { @@ -2192,6 +2202,7 @@ public BacklogQuotaManager getBacklogQuotaManager() { } public void monitorBacklogQuota() { + long startTimeMillis = System.currentTimeMillis(); forEachPersistentTopic(topic -> { if (topic.isSizeBacklogExceeded()) { getBacklogQuotaManager().handleExceededBacklogQuota(topic, @@ -2211,6 +2222,9 @@ public void monitorBacklogQuota() { log.error("Error when checkTimeBacklogExceeded({}) in monitorBacklogQuota", topic.getName(), throwable); return null; + }).whenComplete((unused, throwable) -> { + backlogQuotaCheckDuration.observe( + MILLISECONDS.toSeconds(System.currentTimeMillis() - startTimeMillis)); }); } }); @@ -2732,7 +2746,7 @@ private void updateConfigurationAndRegisterListeners() { // add listener to notify broker managedLedgerCacheEvictionTimeThresholdMillis dynamic config registerConfigurationListener( "managedLedgerCacheEvictionTimeThresholdMillis", (cacheEvictionTimeThresholdMills) -> { - managedLedgerFactory.updateCacheEvictionTimeThreshold(TimeUnit.MILLISECONDS + managedLedgerFactory.updateCacheEvictionTimeThreshold(MILLISECONDS .toNanos((long) cacheEvictionTimeThresholdMills)); }); @@ -3190,7 +3204,7 @@ private void createPendingLoadTopic() { pendingTopic.getTopicFuture() .completeExceptionally((e instanceof RuntimeException && e.getCause() != null) ? e.getCause() : e); // schedule to process next pending topic - inactivityMonitor.schedule(this::createPendingLoadTopic, 100, TimeUnit.MILLISECONDS); + inactivityMonitor.schedule(this::createPendingLoadTopic, 100, MILLISECONDS); return null; }); } @@ -3510,6 +3524,10 @@ private CompletableFuture isAllowAutoTopicCreationAsync(final TopicName } public boolean isDefaultTopicTypePartitioned(final TopicName topicName, final Optional policies) { + if (topicName.getPartitionedTopicName().endsWith(DLQ_GROUP_TOPIC_SUFFIX) + || topicName.getPartitionedTopicName().endsWith(RETRY_GROUP_TOPIC_SUFFIX)) { + return false; + } AutoTopicCreationOverride autoTopicCreationOverride = getAutoTopicCreationOverride(topicName, policies); if (autoTopicCreationOverride != null) { return TopicType.PARTITIONED.toString().equals(autoTopicCreationOverride.getTopicType()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java index e06d9b2f81e7c..2e961b793d8e4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java @@ -68,7 +68,7 @@ default void setOriginalSequenceId(long originalSequenceId) { /** * Return the producer name for the original producer. - * + *

* For messages published locally, this will return the same local producer name, though in case of replicated * messages, the original producer name will differ */ @@ -136,7 +136,7 @@ default void setEntryTimestamp(long entryTimestamp) { /** * Tries to add a producer to the topic. Several validations will be performed. * - * @param producer + * @param producer Producer to add * @param producerQueuedFuture * a future that will be triggered if the producer is being queued up prior of getting established * @return the "topic epoch" if there is one or empty @@ -148,7 +148,7 @@ default void setEntryTimestamp(long entryTimestamp) { /** * Wait TransactionBuffer Recovers completely. * Take snapshot after TB Recovers completely. - * @param isTxnEnabled + * @param isTxnEnabled isTxnEnabled * @return a future which has completely if isTxn = false. Or a future return by takeSnapshot. */ CompletableFuture checkIfTransactionBufferRecoverCompletely(boolean isTxnEnabled); @@ -268,6 +268,13 @@ default boolean isDeduplicationEnabled() { BacklogQuota getBacklogQuota(BacklogQuotaType backlogQuotaType); + /** + * Uses the best-effort (not necessarily up-to-date) information available to return the age. + * @return The oldest unacknowledged message age in seconds, or -1 if not available + */ + long getBestEffortOldestUnacknowledgedMessageAgeSeconds(); + + void updateRates(NamespaceStats nsStats, NamespaceBundleStats currentBundleStats, StatsOutputStream topicStatsStream, ClusterReplicationMetrics clusterReplicationMetrics, String namespaceName, boolean hydratePublishers); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index aadb232ede062..23d938d22fd4b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -166,7 +166,7 @@ public CompletableFuture initialize() { .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) .thenCompose(optPolicies -> { final Policies policies; - if (!optPolicies.isPresent()) { + if (optPolicies.isEmpty()) { log.warn("[{}] Policies not present and isEncryptionRequired will be set to false", topic); isEncryptionRequired = false; policies = new Policies(); @@ -1183,4 +1183,9 @@ protected boolean isMigrated() { public boolean isPersistent() { return false; } + + @Override + public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { + return -1; + } } 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 55d6f6d70a1d9..b83a8206b4ab4 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 @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service.persistent; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.pulsar.broker.service.persistent.SubscribeRateLimiter.isSubscribeRateEnabled; import static org.apache.pulsar.common.naming.SystemTopicNames.isEventSystemTopic; @@ -49,10 +50,12 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.function.BiFunction; import java.util.stream.Collectors; import javax.annotation.Nonnull; import lombok.Getter; +import lombok.Value; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; @@ -74,6 +77,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; +import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer.CursorInfo; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -294,6 +298,30 @@ public CloseFutures(CompletableFuture waitDisconnectClients, this.waitDisconnectClients = waitDisconnectClients; this.notWaitDisconnectClients = notWaitDisconnectClients; } + + } + + @Getter + private final PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); + + private volatile TimeBasedBacklogQuotaCheckResult timeBasedBacklogQuotaCheckResult; + private static final AtomicReferenceFieldUpdater + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( + PersistentTopic.class, + TimeBasedBacklogQuotaCheckResult.class, + "timeBasedBacklogQuotaCheckResult"); + @Value + private static class TimeBasedBacklogQuotaCheckResult { + PositionImpl oldestCursorMarkDeletePosition; + String cursorName; + long positionPublishTimestampInMillis; + long dataVersion; + } + + @Value + private static class EstimateTimeBasedBacklogQuotaCheckResult { + boolean truncateBacklogToMatchQuota; + Long estimatedOldestUnacknowledgedMessageTimestamp; } private static class TopicStatsHelper { @@ -509,7 +537,7 @@ public CompletableFuture unloadSubscription(@Nonnull String subName) { if (!lock.writeLock().tryLock()) { return CompletableFuture.failedFuture(new SubscriptionConflictUnloadException(String.format("Conflict" + " topic-close, topic-delete, another-subscribe-unload, cannot unload subscription %s now", - topic, subName))); + subName))); } try { if (isFenced) { @@ -1149,7 +1177,7 @@ private CompletableFuture getNonDurableSubscription(Stri private void resetSubscriptionCursor(Subscription subscription, CompletableFuture subscriptionFuture, long startMessageRollbackDurationSec) { long timestamp = System.currentTimeMillis() - - TimeUnit.SECONDS.toMillis(startMessageRollbackDurationSec); + - SECONDS.toMillis(startMessageRollbackDurationSec); final Subscription finalSubscription = subscription; subscription.resetCursor(timestamp).handle((s, ex) -> { if (ex != null) { @@ -1680,7 +1708,7 @@ CompletableFuture checkReplicationAndRetryOnFailure() { if (!(th.getCause() instanceof TopicFencedException)) { // retriable exception brokerService.executor().schedule(this::checkReplicationAndRetryOnFailure, - POLICY_UPDATE_FAILURE_RETRY_TIME_SECONDS, TimeUnit.SECONDS); + POLICY_UPDATE_FAILURE_RETRY_TIME_SECONDS, SECONDS); } result.completeExceptionally(th); return null; @@ -2453,6 +2481,19 @@ public CompletableFuture asyncGetStats(boolean getPreciseBacklog stats.lastOffloadFailureTimeStamp = ledger.getLastOffloadedFailureTimestamp(); Optional mxBean = getCompactorMXBean(); + stats.backlogQuotaLimitSize = getBacklogQuota(BacklogQuotaType.destination_storage).getLimitSize(); + stats.backlogQuotaLimitTime = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); + + TimeBasedBacklogQuotaCheckResult backlogQuotaCheckResult = timeBasedBacklogQuotaCheckResult; + stats.oldestBacklogMessageAgeSeconds = (backlogQuotaCheckResult == null) + ? (long) -1 + : TimeUnit.MILLISECONDS.toSeconds( + Clock.systemUTC().millis() - backlogQuotaCheckResult.getPositionPublishTimestampInMillis()); + + stats.oldestBacklogMessageSubscriptionName = (backlogQuotaCheckResult == null) + ? null + : backlogQuotaCheckResult.getCursorName(); + stats.compaction.reset(); mxBean.flatMap(bean -> bean.getCompactionRecordForTopic(topic)).map(compactionRecord -> { stats.compaction.lastCompactionRemovedEventCount = compactionRecord.getLastCompactionRemovedEventCount(); @@ -2756,7 +2797,7 @@ public void checkGC() { int maxInactiveDurationInSec = topicPolicies.getInactiveTopicPolicies().get().getMaxInactiveDurationSeconds(); if (isActive(deleteMode)) { lastActive = System.nanoTime(); - } else if (System.nanoTime() - lastActive < TimeUnit.SECONDS.toNanos(maxInactiveDurationInSec)) { + } else if (System.nanoTime() - lastActive < SECONDS.toNanos(maxInactiveDurationInSec)) { // Gc interval did not expire yet return; } else if (shouldTopicBeRetained()) { @@ -3097,36 +3138,128 @@ public boolean isSizeBacklogExceeded() { return (storageSize >= backlogQuotaLimitInBytes); } + @Override + public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { + TimeBasedBacklogQuotaCheckResult result = timeBasedBacklogQuotaCheckResult; + if (result == null) { + return -1; + } else { + return TimeUnit.MILLISECONDS.toSeconds( + Clock.systemUTC().millis() - result.getPositionPublishTimestampInMillis()); + } + } + + private void updateResultIfNewer(TimeBasedBacklogQuotaCheckResult updatedResult) { + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER.updateAndGet(this, + existingResult -> { + if (existingResult == null + || ManagedCursorContainer.DataVersion.compareVersions( + updatedResult.getDataVersion(), existingResult.getDataVersion()) > 0) { + return updatedResult; + } else { + return existingResult; + } + }); + + } + /** * @return determine if backlog quota enforcement needs to be done for topic based on time limit */ public CompletableFuture checkTimeBacklogExceeded() { TopicName topicName = TopicName.get(getName()); int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); + if (log.isDebugEnabled()) { + log.debug("[{}] Time backlog quota = [{}]. Checking if exceeded.", topicName, backlogQuotaLimitInSecond); + } + + // If backlog quota by time is not set + if (backlogQuotaLimitInSecond <= 0) { + return CompletableFuture.completedFuture(false); + } - // If backlog quota by time is not set and we have no durable cursor. - if (backlogQuotaLimitInSecond <= 0 - || ((ManagedCursorContainer) ledger.getCursors()).getSlowestReaderPosition() == null) { + ManagedCursorContainer managedCursorContainer = (ManagedCursorContainer) ledger.getCursors(); + CursorInfo oldestMarkDeleteCursorInfo = managedCursorContainer.getCursorWithOldestPosition(); + + // If we have no durable cursor since `ledger.getCursors()` only managed durable cursors + if (oldestMarkDeleteCursorInfo == null + || oldestMarkDeleteCursorInfo.getPosition() == null) { + if (log.isDebugEnabled()) { + log.debug("[{}] No durable cursor found. Skipping time based backlog quota check." + + " Oldest mark-delete cursor info: {}", topicName, oldestMarkDeleteCursorInfo); + } return CompletableFuture.completedFuture(false); } + PositionImpl oldestMarkDeletePosition = oldestMarkDeleteCursorInfo.getPosition(); + + TimeBasedBacklogQuotaCheckResult lastCheckResult = timeBasedBacklogQuotaCheckResult; + if (lastCheckResult != null + && oldestMarkDeletePosition.compareTo(lastCheckResult.getOldestCursorMarkDeletePosition()) == 0) { + + // Same position, but the cursor causing it has changed? + if (!lastCheckResult.getCursorName().equals(oldestMarkDeleteCursorInfo.getCursor().getName())) { + final TimeBasedBacklogQuotaCheckResult updatedResult = new TimeBasedBacklogQuotaCheckResult( + lastCheckResult.getOldestCursorMarkDeletePosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + lastCheckResult.getPositionPublishTimestampInMillis(), + oldestMarkDeleteCursorInfo.getVersion()); + + updateResultIfNewer(updatedResult); + if (log.isDebugEnabled()) { + log.debug("[{}] Time-based backlog quota check. Updating cached result for position {}, " + + "since cursor causing it has changed from {} to {}", + topicName, + oldestMarkDeletePosition, + lastCheckResult.getCursorName(), + oldestMarkDeleteCursorInfo.getCursor().getName()); + } + } + + long entryTimestamp = lastCheckResult.getPositionPublishTimestampInMillis(); + boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); + if (log.isDebugEnabled()) { + log.debug("[{}] Time based backlog quota check. Using cache result for position {}. " + + "Entry timestamp: {}, expired: {}", + topicName, oldestMarkDeletePosition, entryTimestamp, expired); + } + return CompletableFuture.completedFuture(expired); + } + if (brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()) { CompletableFuture future = new CompletableFuture<>(); // Check if first unconsumed message(first message after mark delete position) // for slowest cursor's has expired. - PositionImpl position = ((ManagedLedgerImpl) ledger).getNextValidPosition(((ManagedCursorContainer) - ledger.getCursors()).getSlowestReaderPosition()); + PositionImpl position = ((ManagedLedgerImpl) ledger).getNextValidPosition(oldestMarkDeletePosition); ((ManagedLedgerImpl) ledger).asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { @Override public void readEntryComplete(Entry entry, Object ctx) { try { long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); + + updateResultIfNewer( + new TimeBasedBacklogQuotaCheckResult( + oldestMarkDeleteCursorInfo.getPosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + entryTimestamp, + oldestMarkDeleteCursorInfo.getVersion())); + boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); - if (expired && log.isDebugEnabled()) { - log.debug("Time based backlog quota exceeded, oldest entry in cursor {}'s backlog" - + "exceeded quota {}", ((ManagedLedgerImpl) ledger).getSlowestConsumer().getName(), - backlogQuotaLimitInSecond); + if (log.isDebugEnabled()) { + log.debug("[{}] Time based backlog quota check. Oldest unacked entry read from BK. " + + "Oldest entry in cursor {}'s backlog: {}. " + + "Oldest mark-delete position: {}. " + + "Quota {}. Last check result position [{}]. " + + "Expired: {}, entryTimestamp: {}", + topicName, + oldestMarkDeleteCursorInfo.getCursor().getName(), + position, + oldestMarkDeletePosition, + backlogQuotaLimitInSecond, + lastCheckResult.getOldestCursorMarkDeletePosition(), + expired, + entryTimestamp); } future.complete(expired); } catch (Exception e) { @@ -3146,9 +3279,19 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { }, null); return future; } else { - PositionImpl slowestPosition = ((ManagedCursorContainer) ledger.getCursors()).getSlowestReaderPosition(); try { - return slowestReaderTimeBasedBacklogQuotaCheck(slowestPosition); + EstimateTimeBasedBacklogQuotaCheckResult checkResult = + estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); + if (checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp() != null) { + updateResultIfNewer( + new TimeBasedBacklogQuotaCheckResult( + oldestMarkDeleteCursorInfo.getPosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp(), + oldestMarkDeleteCursorInfo.getVersion())); + } + + return CompletableFuture.completedFuture(checkResult.isTruncateBacklogToMatchQuota()); } catch (Exception e) { log.error("[{}][{}] Error reading entry for precise time based backlog check", topicName, e); return CompletableFuture.completedFuture(false); @@ -3156,33 +3299,47 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { } } - private CompletableFuture slowestReaderTimeBasedBacklogQuotaCheck(PositionImpl slowestPosition) + private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck( + PositionImpl markDeletePosition) throws ExecutionException, InterruptedException { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); - Long ledgerId = slowestPosition.getLedgerId(); - if (((ManagedLedgerImpl) ledger).getLedgersInfo().lastKey().equals(ledgerId)) { - return CompletableFuture.completedFuture(false); + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) ledger; + + // The ledger timestamp is only known when ledger is closed, hence when the mark-delete + // is at active ledger (open) we can't estimate it. + if (managedLedger.getLedgersInfo().lastKey().equals(markDeletePosition.getLedgerId())) { + return new EstimateTimeBasedBacklogQuotaCheckResult(false, null); } - int result; + org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo - ledgerInfo = ledger.getLedgerInfo(ledgerId).get(); - if (ledgerInfo != null && ledgerInfo.hasTimestamp() && ledgerInfo.getTimestamp() > 0 - && ((ManagedLedgerImpl) ledger).getClock().millis() - ledgerInfo.getTimestamp() - > backlogQuotaLimitInSecond * 1000 && (result = slowestPosition.compareTo( - new PositionImpl(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1))) <= 0) { - if (result < 0) { - if (log.isDebugEnabled()) { - log.debug("Time based backlog quota exceeded, quota {}, age of ledger " - + "slowest cursor currently on {}", backlogQuotaLimitInSecond * 1000, - ((ManagedLedgerImpl) ledger).getClock().millis() - ledgerInfo.getTimestamp()); - } - return CompletableFuture.completedFuture(true); - } else { - return slowestReaderTimeBasedBacklogQuotaCheck( - ((ManagedLedgerImpl) ledger).getNextValidPosition(slowestPosition)); + markDeletePositionLedgerInfo = ledger.getLedgerInfo(markDeletePosition.getLedgerId()).get(); + + org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo positionToCheckLedgerInfo = + markDeletePositionLedgerInfo; + + // if the mark-delete position is the last entry it means all entries for + // that ledger are acknowledged + if (markDeletePosition.getEntryId() == markDeletePositionLedgerInfo.getEntries() - 1) { + PositionImpl positionToCheck = managedLedger.getNextValidPosition(markDeletePosition); + positionToCheckLedgerInfo = ledger.getLedgerInfo(positionToCheck.getLedgerId()).get(); + } + + if (positionToCheckLedgerInfo != null + && positionToCheckLedgerInfo.hasTimestamp() + && positionToCheckLedgerInfo.getTimestamp() > 0) { + long estimateMsgAgeMs = managedLedger.getClock().millis() - positionToCheckLedgerInfo.getTimestamp(); + boolean shouldTruncateBacklog = estimateMsgAgeMs > SECONDS.toMillis(backlogQuotaLimitInSecond); + if (log.isDebugEnabled()) { + log.debug("Time based backlog quota exceeded, quota {}[ms], age of ledger " + + "slowest cursor currently on {}[ms]", backlogQuotaLimitInSecond * 1000, + estimateMsgAgeMs); } + + return new EstimateTimeBasedBacklogQuotaCheckResult( + shouldTruncateBacklog, + positionToCheckLedgerInfo.getTimestamp()); } else { - return CompletableFuture.completedFuture(false); + return new EstimateTimeBasedBacklogQuotaCheckResult(false, null); } } @@ -3573,7 +3730,7 @@ private synchronized void fence() { final int timeout = brokerService.pulsar().getConfiguration().getTopicFencingTimeoutSeconds(); if (timeout > 0) { this.fencedTopicMonitoringTask = brokerService.executor().schedule(this::closeFencedTopicForcefully, - timeout, TimeUnit.SECONDS); + timeout, SECONDS); } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java new file mode 100644 index 0000000000000..f79d053a9790d --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java @@ -0,0 +1,50 @@ +/* + * 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 java.util.concurrent.atomic.LongAdder; +import lombok.Getter; + +@SuppressWarnings("LombokGetterMayBeUsed") +public class PersistentTopicMetrics { + + @Getter + private final BacklogQuotaMetrics backlogQuotaMetrics = new BacklogQuotaMetrics(); + + public static class BacklogQuotaMetrics { + private final LongAdder timeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); + private final LongAdder sizeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); + + public void recordTimeBasedBacklogEviction() { + timeBasedBacklogQuotaExceededEvictionCount.increment(); + } + + public void recordSizeBasedBacklogEviction() { + sizeBasedBacklogQuotaExceededEvictionCount.increment(); + } + + public long getSizeBasedBacklogQuotaExceededEvictionCount() { + return sizeBasedBacklogQuotaExceededEvictionCount.longValue(); + } + + public long getTimeBasedBacklogQuotaExceededEvictionCount() { + return timeBasedBacklogQuotaExceededEvictionCount.longValue(); + } + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java index f93e561542eeb..53418744b5486 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java @@ -197,7 +197,8 @@ protected EntryFilter load(EntryFilterMetaData metadata) + " does not implement entry filter interface"); } EntryFilter pi = (EntryFilter) filter; - return new EntryFilterWithClassLoader(pi, ncl); + // the classloader is shared with the broker, the instance doesn't own it + return new EntryFilterWithClassLoader(pi, ncl, false); } catch (Throwable e) { if (e instanceof IOException) { throw (IOException) e; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java index c5c5721087788..aab46c62acdb4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java @@ -30,15 +30,23 @@ public class EntryFilterWithClassLoader implements EntryFilter { private final EntryFilter entryFilter; private final NarClassLoader classLoader; + private final boolean classLoaderOwned; - public EntryFilterWithClassLoader(EntryFilter entryFilter, NarClassLoader classLoader) { + public EntryFilterWithClassLoader(EntryFilter entryFilter, NarClassLoader classLoader, boolean classLoaderOwned) { this.entryFilter = entryFilter; this.classLoader = classLoader; + this.classLoaderOwned = classLoaderOwned; } @Override public FilterResult filterEntry(Entry entry, FilterContext context) { - return entryFilter.filterEntry(entry, context); + ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); + return entryFilter.filterEntry(entry, context); + } finally { + Thread.currentThread().setContextClassLoader(currentClassLoader); + } } @VisibleForTesting @@ -48,11 +56,20 @@ public EntryFilter getEntryFilter() { @Override public void close() { - entryFilter.close(); + ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); try { - classLoader.close(); - } catch (IOException e) { - log.error("close EntryFilterWithClassLoader failed", e); + Thread.currentThread().setContextClassLoader(classLoader); + entryFilter.close(); + } finally { + Thread.currentThread().setContextClassLoader(currentClassLoader); + } + if (classLoaderOwned) { + log.info("Closing classloader {} for EntryFilter {}", classLoader, entryFilter.getClass().getName()); + try { + classLoader.close(); + } catch (IOException e) { + log.error("close EntryFilterWithClassLoader failed", e); + } } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/AvroSchemaBasedCompatibilityCheck.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/AvroSchemaBasedCompatibilityCheck.java index 1e75834a12988..e5fc7800c5170 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/AvroSchemaBasedCompatibilityCheck.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/AvroSchemaBasedCompatibilityCheck.java @@ -64,8 +64,10 @@ public void checkCompatible(Iterable from, SchemaData to, SchemaComp log.warn("Error during schema parsing: {}", e.getMessage()); throw new IncompatibleSchemaException(e); } catch (SchemaValidationException e) { - log.warn("Error during schema compatibility check: {}", e.getMessage()); - throw new IncompatibleSchemaException(e); + String msg = String.format("Error during schema compatibility check with strategy %s: %s: %s", + strategy, e.getClass().getName(), e.getMessage()); + log.warn(msg); + throw new IncompatibleSchemaException(msg, e); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/ProtobufNativeSchemaCompatibilityCheck.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/ProtobufNativeSchemaCompatibilityCheck.java index 16b3b33ec7894..fc935e80dca36 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/ProtobufNativeSchemaCompatibilityCheck.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/ProtobufNativeSchemaCompatibilityCheck.java @@ -67,7 +67,9 @@ public void checkCompatible(Iterable from, SchemaData to, SchemaComp private void checkRootMessageChange(Descriptor fromDescriptor, Descriptor toDescriptor, SchemaCompatibilityStrategy strategy) throws IncompatibleSchemaException { if (!fromDescriptor.getFullName().equals(toDescriptor.getFullName())) { - throw new IncompatibleSchemaException("Protobuf root message isn't allow change!"); + throw new IncompatibleSchemaException("Protobuf root message change is not allowed under the '" + + strategy + "' strategy. Original message name: '" + fromDescriptor.getFullName() + + "', new message name: '" + toDescriptor.getFullName() + "'."); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java index ae56df248d85d..903f57cb7803a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java @@ -473,7 +473,7 @@ private CompletableFuture checkCompatibilityWithLatest(String schemaId, Sc } return result; } else { - return FutureUtils.exception(new IncompatibleSchemaException("Do not have existing schema.")); + return CompletableFuture.completedFuture(null); } }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/IncompatibleSchemaException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/IncompatibleSchemaException.java index c1a2d9fd703fd..bbe2f4111d759 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/IncompatibleSchemaException.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/IncompatibleSchemaException.java @@ -33,6 +33,10 @@ public IncompatibleSchemaException(String message) { super(message); } + public IncompatibleSchemaException(String message, Throwable e) { + super(message, e); + } + public IncompatibleSchemaException(Throwable e) { super(e); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java index 715231d3c6ee1..037fb29a999e3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java @@ -33,7 +33,10 @@ public class AggregatedBrokerStats { public double storageReadRate; public double storageReadCacheMissesRate; public long msgBacklog; + public long sizeBasedBacklogQuotaExceededEvictionCount; + public long timeBasedBacklogQuotaExceededEvictionCount; + @SuppressWarnings("DuplicatedCode") void updateStats(TopicStats stats) { topicsCount++; subscriptionsCount += stats.subscriptionsCount; @@ -49,8 +52,11 @@ void updateStats(TopicStats stats) { storageReadRate += stats.managedLedgerStats.storageReadRate; storageReadCacheMissesRate += stats.managedLedgerStats.storageReadCacheMissesRate; msgBacklog += stats.msgBacklog; + timeBasedBacklogQuotaExceededEvictionCount += stats.timeBasedBacklogQuotaExceededEvictionCount; + sizeBasedBacklogQuotaExceededEvictionCount += stats.sizeBasedBacklogQuotaExceededEvictionCount; } + @SuppressWarnings("DuplicatedCode") public void reset() { topicsCount = 0; subscriptionsCount = 0; @@ -66,5 +72,7 @@ public void reset() { storageReadRate = 0; storageReadCacheMissesRate = 0; msgBacklog = 0; + sizeBasedBacklogQuotaExceededEvictionCount = 0; + timeBasedBacklogQuotaExceededEvictionCount = 0; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index d0dc4fe2a7e7d..3975cd89cfa6b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -51,6 +51,9 @@ public class AggregatedNamespaceStats { long backlogQuotaLimit; long backlogQuotaLimitTime; + public long sizeBasedBacklogQuotaExceededEvictionCount; + public long timeBasedBacklogQuotaExceededEvictionCount; + public Map replicationStats = new HashMap<>(); public Map subscriptionStats = new HashMap<>(); @@ -68,6 +71,7 @@ public class AggregatedNamespaceStats { Map bucketDelayedIndexStats = new HashMap<>(); + @SuppressWarnings("DuplicatedCode") void updateStats(TopicStats stats) { topicsCount++; @@ -105,6 +109,9 @@ void updateStats(TopicStats stats) { backlogQuotaLimit = Math.max(backlogQuotaLimit, stats.backlogQuotaLimit); backlogQuotaLimitTime = Math.max(backlogQuotaLimitTime, stats.backlogQuotaLimitTime); + sizeBasedBacklogQuotaExceededEvictionCount += stats.sizeBasedBacklogQuotaExceededEvictionCount; + timeBasedBacklogQuotaExceededEvictionCount += stats.timeBasedBacklogQuotaExceededEvictionCount; + managedLedgerStats.storageWriteRate += stats.managedLedgerStats.storageWriteRate; managedLedgerStats.storageReadRate += stats.managedLedgerStats.storageReadRate; managedLedgerStats.storageReadCacheMissesRate += stats.managedLedgerStats.storageReadCacheMissesRate; @@ -172,6 +179,7 @@ void updateStats(TopicStats stats) { compactionLatencyBuckets.addAll(stats.compactionLatencyBuckets); } + @SuppressWarnings("DuplicatedCode") public void reset() { managedLedgerStats.reset(); topicsCount = 0; @@ -201,6 +209,9 @@ public void reset() { replicationStats.clear(); subscriptionStats.clear(); + sizeBasedBacklogQuotaExceededEvictionCount = 0; + timeBasedBacklogQuotaExceededEvictionCount = 0; + compactionRemovedEventCount = 0; compactionSucceedCount = 0; compactionFailedCount = 0; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index 4e72fa0d72b16..3728c3edd1e8b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -32,7 +32,10 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics.BacklogQuotaMetrics; +import org.apache.pulsar.broker.stats.prometheus.metrics.PrometheusLabels; +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentSubscriptionStatsImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentTopicStatsImpl; @@ -159,14 +162,15 @@ private static void aggregateTopicStats(TopicStats stats, SubscriptionStatsImpl subsStats.bucketDelayedIndexStats = subscriptionStats.bucketDelayedIndexStats; } + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private static void getTopicStats(Topic topic, TopicStats stats, boolean includeConsumerMetrics, boolean includeProducerMetrics, boolean getPreciseBacklog, boolean subscriptionBacklogSize, Optional compactorMXBean) { stats.reset(); - if (topic instanceof PersistentTopic) { + if (topic instanceof PersistentTopic persistentTopic) { // Managed Ledger stats - ManagedLedger ml = ((PersistentTopic) topic).getManagedLedger(); + ManagedLedger ml = persistentTopic.getManagedLedger(); ManagedLedgerMBeanImpl mlStats = (ManagedLedgerMBeanImpl) ml.getStats(); stats.managedLedgerStats.storageSize = mlStats.getStoredMessagesSize(); @@ -174,9 +178,10 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include stats.managedLedgerStats.backlogSize = ml.getEstimatedBacklogSize(); stats.managedLedgerStats.offloadedStorageUsed = ml.getOffloadedSize(); stats.backlogQuotaLimit = topic - .getBacklogQuota(BacklogQuota.BacklogQuotaType.destination_storage).getLimitSize(); + .getBacklogQuota(BacklogQuotaType.destination_storage).getLimitSize(); stats.backlogQuotaLimitTime = topic - .getBacklogQuota(BacklogQuota.BacklogQuotaType.message_age).getLimitTime(); + .getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); + stats.backlogAgeSeconds = topic.getBestEffortOldestUnacknowledgedMessageAgeSeconds(); stats.managedLedgerStats.storageWriteLatencyBuckets .addAll(mlStats.getInternalAddEntryLatencyBuckets()); @@ -191,7 +196,17 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include stats.managedLedgerStats.storageWriteRate = mlStats.getAddEntryMessagesRate(); stats.managedLedgerStats.storageReadRate = mlStats.getReadEntriesRate(); stats.managedLedgerStats.storageReadCacheMissesRate = mlStats.getReadEntriesOpsCacheMissesRate(); + + // Topic Stats + PersistentTopicMetrics persistentTopicMetrics = persistentTopic.getPersistentTopicMetrics(); + + BacklogQuotaMetrics backlogQuotaMetrics = persistentTopicMetrics.getBacklogQuotaMetrics(); + stats.sizeBasedBacklogQuotaExceededEvictionCount = + backlogQuotaMetrics.getSizeBasedBacklogQuotaExceededEvictionCount(); + stats.timeBasedBacklogQuotaExceededEvictionCount = + backlogQuotaMetrics.getTimeBasedBacklogQuotaExceededEvictionCount(); } + TopicStatsImpl tStatus = topic.getStats(getPreciseBacklog, subscriptionBacklogSize, false); stats.msgInCounter = tStatus.msgInCounter; stats.bytesInCounter = tStatus.bytesInCounter; @@ -334,6 +349,14 @@ private static void printBrokerStats(PrometheusMetricStreams stream, String clus writeMetric(stream, "pulsar_broker_storage_read_rate", brokerStats.storageReadRate, cluster); writeMetric(stream, "pulsar_broker_storage_read_cache_misses_rate", brokerStats.storageReadCacheMissesRate, cluster); + + writePulsarBacklogQuotaMetricBrokerLevel(stream, + "pulsar_broker_storage_backlog_quota_exceeded_evictions_total", + brokerStats.sizeBasedBacklogQuotaExceededEvictionCount, cluster, BacklogQuotaType.destination_storage); + writePulsarBacklogQuotaMetricBrokerLevel(stream, + "pulsar_broker_storage_backlog_quota_exceeded_evictions_total", + brokerStats.timeBasedBacklogQuotaExceededEvictionCount, cluster, BacklogQuotaType.message_age); + writeMetric(stream, "pulsar_broker_msg_backlog", brokerStats.msgBacklog, cluster); } @@ -372,6 +395,7 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat stats.managedLedgerStats.storageLogicalSize, cluster, namespace); writeMetric(stream, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize, cluster, namespace); + writeMetric(stream, "pulsar_storage_offloaded_size", stats.managedLedgerStats.offloadedStorageUsed, cluster, namespace); @@ -392,6 +416,14 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat }); writePulsarMsgBacklog(stream, stats.msgBacklog, cluster, namespace); + writePulsarBacklogQuotaMetricNamespaceLevel(stream, + "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.sizeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, + BacklogQuotaType.destination_storage); + writePulsarBacklogQuotaMetricNamespaceLevel(stream, + "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.timeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, + BacklogQuotaType.message_age); stats.managedLedgerStats.storageWriteLatencyBuckets.refresh(); long[] latencyBuckets = stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets(); @@ -471,6 +503,25 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat replStats -> replStats.replicationDelayInSeconds, cluster, namespace); } + @SuppressWarnings("SameParameterValue") + private static void writePulsarBacklogQuotaMetricBrokerLevel(PrometheusMetricStreams stream, String metricName, + Number value, String cluster, + BacklogQuotaType backlogQuotaType) { + String quotaTypeLabelValue = PrometheusLabels.backlogQuotaTypeLabel(backlogQuotaType); + stream.writeSample(metricName, value, "cluster", cluster, + "quota_type", quotaTypeLabelValue); + } + + @SuppressWarnings("SameParameterValue") + private static void writePulsarBacklogQuotaMetricNamespaceLevel(PrometheusMetricStreams stream, String metricName, + Number value, String cluster, String namespace, + BacklogQuotaType backlogQuotaType) { + String quotaTypeLabelValue = PrometheusLabels.backlogQuotaTypeLabel(backlogQuotaType); + stream.writeSample(metricName, value, "cluster", cluster, + "namespace", namespace, + "quota_type", quotaTypeLabelValue); + } + private static void writePulsarMsgBacklog(PrometheusMetricStreams stream, Number value, String cluster, String namespace) { stream.writeSample("pulsar_msg_backlog", value, "cluster", cluster, "namespace", namespace, 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 33ef05df9ebc4..27288291d2969 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 @@ -25,6 +25,8 @@ import org.apache.bookkeeper.mledger.util.StatsBuckets; import org.apache.commons.lang3.ArrayUtils; import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.stats.prometheus.metrics.PrometheusLabels; +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; import org.apache.pulsar.compaction.CompactionRecord; import org.apache.pulsar.compaction.CompactorMXBean; @@ -52,6 +54,7 @@ class TopicStats { long backlogQuotaLimit; long backlogQuotaLimitTime; + long backlogAgeSeconds; ManagedLedgerStats managedLedgerStats = new ManagedLedgerStats(); @@ -73,6 +76,11 @@ class TopicStats { Map bucketDelayedIndexStats = new HashMap<>(); + public long sizeBasedBacklogQuotaExceededEvictionCount; + public long timeBasedBacklogQuotaExceededEvictionCount; + + + @SuppressWarnings("DuplicatedCode") public void reset() { subscriptionsCount = 0; producersCount = 0; @@ -111,8 +119,13 @@ public void reset() { compactionLatencyBuckets.reset(); delayedMessageIndexSizeInBytes = 0; bucketDelayedIndexStats.clear(); + + timeBasedBacklogQuotaExceededEvictionCount = 0; + sizeBasedBacklogQuotaExceededEvictionCount = 0; + backlogAgeSeconds = -1; } + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") public static void printTopicStats(PrometheusMetricStreams stream, TopicStats stats, Optional compactorMXBean, String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel) { @@ -165,6 +178,14 @@ public static void printTopicStats(PrometheusMetricStreams stream, TopicStats st cluster, namespace, topic, splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_storage_backlog_quota_limit_time", stats.backlogQuotaLimitTime, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_backlog_age_seconds", stats.backlogAgeSeconds, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeBacklogQuotaMetric(stream, "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.sizeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, topic, + splitTopicAndPartitionIndexLabel, BacklogQuotaType.destination_storage); + writeBacklogQuotaMetric(stream, "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.timeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, topic, + splitTopicAndPartitionIndexLabel, BacklogQuotaType.message_age); writeMetric(stream, "pulsar_delayed_message_index_size_bytes", stats.delayedMessageIndexSizeInBytes, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); @@ -442,6 +463,17 @@ private static void writeMetric(PrometheusMetricStreams stream, String metricNam writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); } + @SuppressWarnings("SameParameterValue") + private static void writeBacklogQuotaMetric(PrometheusMetricStreams stream, String metricName, Number value, + String cluster, String namespace, String topic, + boolean splitTopicAndPartitionIndexLabel, + BacklogQuotaType backlogQuotaType) { + + String quotaTypeLabelValue = PrometheusLabels.backlogQuotaTypeLabel(backlogQuotaType); + writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel, + "quota_type", quotaTypeLabelValue); + } + private static void writeMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, String namespace, String topic, String remoteCluster, boolean splitTopicAndPartitionIndexLabel) { diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ClassLoaderSwitcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java similarity index 54% rename from pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ClassLoaderSwitcher.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java index 55cb9198da2bc..9a2c520731468 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ClassLoaderSwitcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java @@ -16,22 +16,17 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker; +package org.apache.pulsar.broker.stats.prometheus.metrics; -/** - * Help to switch the class loader of current thread to the NarClassLoader, and change it back when it's done. - * With the help of try-with-resources statement, the code would be cleaner than using try finally every time. - */ -public class ClassLoaderSwitcher implements AutoCloseable { - private final ClassLoader prevClassLoader; +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; - public ClassLoaderSwitcher(ClassLoader classLoader) { - prevClassLoader = Thread.currentThread().getContextClassLoader(); - Thread.currentThread().setContextClassLoader(classLoader); - } +public class PrometheusLabels { - @Override - public void close() { - Thread.currentThread().setContextClassLoader(prevClassLoader); + public static String backlogQuotaTypeLabel(BacklogQuotaType backlogQuotaType) { + if (backlogQuotaType == BacklogQuotaType.message_age) { + return "time"; + } else /* destination_storage */ { + return "size"; + } } -} \ No newline at end of file +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java index f67bd6fcfce5b..34d7dbeb8183c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java @@ -467,4 +467,34 @@ public void testCompatibility() throws Exception { assertTrue(e.getMessage().contains("Incompatible schema: exists schema type STRING, new schema type INT8")); } } + + @Test + public void testCompatibilityWithEmpty() throws Exception { + List> checkSchemas = List.of( + Schema.STRING, + Schema.JSON(SchemaDefinition.builder().withPojo(Foo.class).withProperties(PROPS).build()), + Schema.AVRO(SchemaDefinition.builder().withPojo(Foo.class).withProperties(PROPS).build()), + Schema.KeyValue(Schema.STRING, Schema.STRING) + ); + for (Schema schema : checkSchemas) { + SchemaInfo schemaInfo = schema.getSchemaInfo(); + String topicName = schemaCompatibilityNamespace + "/testCompatibilityWithEmpty"; + PostSchemaPayload postSchemaPayload = new PostSchemaPayload(schemaInfo.getType().toString(), + schemaInfo.getSchemaDefinition(), new HashMap<>()); + + // check compatibility with empty schema + IsCompatibilityResponse isCompatibilityResponse = + admin.schemas().testCompatibility(topicName, postSchemaPayload); + assertTrue(isCompatibilityResponse.isCompatibility()); + assertEquals(isCompatibilityResponse.getSchemaCompatibilityStrategy(), SchemaCompatibilityStrategy.FULL.name()); + + // set schema compatibility strategy is FULL_TRANSITIVE to cover checkCompatibilityWithAll + admin.namespaces().setSchemaCompatibilityStrategy(schemaCompatibilityNamespace, SchemaCompatibilityStrategy.FULL_TRANSITIVE); + isCompatibilityResponse = admin.schemas().testCompatibility(topicName, postSchemaPayload); + assertTrue(isCompatibilityResponse.isCompatibility()); + assertEquals(isCompatibilityResponse.getSchemaCompatibilityStrategy(), SchemaCompatibilityStrategy.FULL_TRANSITIVE.name()); + // set back to FULL + admin.namespaces().setSchemaCompatibilityStrategy(schemaCompatibilityNamespace, SchemaCompatibilityStrategy.FULL); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java index 049fd0f5f4400..f6429251690e7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java @@ -38,6 +38,7 @@ import lombok.Cleanup; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.http.HttpStatus; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; @@ -48,12 +49,16 @@ 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.api.TransactionIsolationLevel; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.transaction.TransactionImpl; +import org.apache.pulsar.common.api.proto.MarkerType; +import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicDomain; @@ -894,6 +899,127 @@ public void testGetPositionStatsInPendingAckStatsFroBatch() throws Exception { } + @Test + public void testPeekMessageForSkipTxnMarker() throws Exception { + initTransaction(1); + + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/peek_marker"); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + int n = 10; + for (int i = 0; i < n; i++) { + Transaction txn = pulsarClient.newTransaction().build().get(); + producer.newMessage(txn).value("msg").send(); + txn.commit().get(); + } + + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", n, + false, TransactionIsolationLevel.READ_UNCOMMITTED); + assertEquals(peekMsgs.size(), n); + for (Message peekMsg : peekMsgs) { + assertEquals(new String(peekMsg.getValue()), "msg"); + } + } + + @Test + public void testPeekMessageFoReadCommittedMessages() throws Exception { + initTransaction(1); + + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/peek_txn"); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + int n = 10; + // Alternately sends `n` committed transactional messages and `n` abort transactional messages. + for (int i = 0; i < 2 * n; i++) { + Transaction txn = pulsarClient.newTransaction().build().get(); + if (i % 2 == 0) { + producer.newMessage(txn).value("msg").send(); + txn.commit().get(); + } else { + producer.newMessage(txn).value("msg-aborted").send(); + txn.abort(); + } + } + // Then sends 1 uncommitted transactional messages. + Transaction txn = pulsarClient.newTransaction().build().get(); + producer.newMessage(txn).value("msg-uncommitted").send(); + // Then sends n-1 no transaction messages. + for (int i = 0; i < n - 1; i++) { + producer.newMessage().value("msg-after-uncommitted").send(); + } + + // peek n message, all messages value should be "msg" + { + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", n, + false, TransactionIsolationLevel.READ_COMMITTED); + assertEquals(peekMsgs.size(), n); + for (Message peekMsg : peekMsgs) { + assertEquals(new String(peekMsg.getValue()), "msg"); + } + } + + // peek 3 * n message, and still get n message, all messages value should be "msg" + { + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", 2 * n, + false, TransactionIsolationLevel.READ_COMMITTED); + assertEquals(peekMsgs.size(), n); + for (Message peekMsg : peekMsgs) { + assertEquals(new String(peekMsg.getValue()), "msg"); + } + } + } + + @Test + public void testPeekMessageForShowAllMessages() throws Exception { + initTransaction(1); + + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/peek_all"); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + int n = 10; + // Alternately sends `n` committed transactional messages and `n` abort transactional messages. + for (int i = 0; i < 2 * n; i++) { + Transaction txn = pulsarClient.newTransaction().build().get(); + if (i % 2 == 0) { + producer.newMessage(txn).value("msg").send(); + txn.commit().get(); + } else { + producer.newMessage(txn).value("msg-aborted").send(); + txn.abort(); + } + } + // Then sends `n` uncommitted transactional messages. + Transaction txn = pulsarClient.newTransaction().build().get(); + for (int i = 0; i < n; i++) { + producer.newMessage(txn).value("msg-uncommitted").send(); + } + + // peek 5 * n message, will get 5 * n msg. + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", 5 * n, + true, TransactionIsolationLevel.READ_UNCOMMITTED); + assertEquals(peekMsgs.size(), 5 * n); + + for (int i = 0; i < 4 * n; i++) { + Message peekMsg = peekMsgs.get(i); + MessageImpl peekMsgImpl = (MessageImpl) peekMsg; + MessageMetadata metadata = peekMsgImpl.getMessageBuilder(); + if (metadata.hasMarkerType()) { + assertTrue(metadata.getMarkerType() == MarkerType.TXN_COMMIT_VALUE || + metadata.getMarkerType() == MarkerType.TXN_ABORT_VALUE); + } else { + String value = new String(peekMsg.getValue()); + assertTrue(value.equals("msg") || value.equals("msg-aborted")); + } + } + for (int i = 4 * n; i < peekMsgs.size(); i++) { + Message peekMsg = peekMsgs.get(i); + assertEquals(new String(peekMsg.getValue()), "msg-uncommitted"); + } + } + private static void verifyCoordinatorStats(String state, long sequenceId, long lowWaterMark) { assertEquals(state, "Ready"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java index 5abe8a69ee499..979bf6cd0d5db 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java @@ -65,7 +65,7 @@ public void testLoadBrokerEventListener() throws Exception { BrokerInterceptorWithClassLoader returnedPhWithCL = BrokerInterceptorUtils.load(metadata, ""); BrokerInterceptor returnedPh = returnedPhWithCL.getInterceptor(); - assertSame(mockLoader, returnedPhWithCL.getClassLoader()); + assertSame(mockLoader, returnedPhWithCL.getNarClassLoader()); assertTrue(returnedPh instanceof MockBrokerInterceptor); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java index a2f97e16a76ae..64d4b5ee6cca5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java @@ -135,7 +135,7 @@ public void close() { new BrokerInterceptorWithClassLoader(interceptor, narLoader); ClassLoader curClassLoader = Thread.currentThread().getContextClassLoader(); // test class loader - assertEquals(brokerInterceptorWithClassLoader.getClassLoader(), narLoader); + assertEquals(brokerInterceptorWithClassLoader.getNarClassLoader(), narLoader); // test initialize brokerInterceptorWithClassLoader.initialize(mock(PulsarService.class)); assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); 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 169ff89fe3c0d..d5aaed3436824 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 @@ -1125,13 +1125,15 @@ public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exceptio .getFullBundle(slaMonitorNamespacePulsar2); - Set ownedServiceUnitsByPulsar1 = primaryLoadManager.getOwnedServiceUnits(); + Set ownedServiceUnitsByPulsar1 = primaryLoadManager.getOwnedServiceUnitsAsync() + .get(5, TimeUnit.SECONDS); log.info("Owned service units: {}", ownedServiceUnitsByPulsar1); // heartbeat namespace bundle will own by pulsar1 assertTrue(ownedServiceUnitsByPulsar1.contains(bundle1)); assertTrue(ownedServiceUnitsByPulsar1.contains(bundle2)); assertTrue(ownedServiceUnitsByPulsar1.contains(slaBundle1)); - Set ownedServiceUnitsByPulsar2 = secondaryLoadManager.getOwnedServiceUnits(); + Set ownedServiceUnitsByPulsar2 = secondaryLoadManager.getOwnedServiceUnitsAsync() + .get(5, TimeUnit.SECONDS); log.info("Owned service units: {}", ownedServiceUnitsByPulsar2); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle3)); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle4)); @@ -1167,7 +1169,8 @@ private void assertOwnedServiceUnits( ExtensibleLoadManagerImpl extensibleLoadManager, NamespaceBundle bundle) throws PulsarAdminException { Awaitility.await().untilAsserted(() -> { - Set ownedBundles = extensibleLoadManager.getOwnedServiceUnits(); + Set ownedBundles = extensibleLoadManager.getOwnedServiceUnitsAsync() + .get(5, TimeUnit.SECONDS); assertTrue(ownedBundles.contains(bundle)); }); Map ownedNamespaces = @@ -1180,9 +1183,11 @@ private void assertOwnedServiceUnits( } @Test(timeOut = 30 * 1000) - public void testGetOwnedServiceUnitsWhenLoadManagerNotStart() { + public void testGetOwnedServiceUnitsWhenLoadManagerNotStart() + throws Exception { ExtensibleLoadManagerImpl loadManager = new ExtensibleLoadManagerImpl(); - Set ownedServiceUnits = loadManager.getOwnedServiceUnits(); + Set ownedServiceUnits = loadManager.getOwnedServiceUnitsAsync() + .get(5, TimeUnit.SECONDS); assertNotNull(ownedServiceUnits); assertTrue(ownedServiceUnits.isEmpty()); } @@ -1197,7 +1202,7 @@ public void testTryAcquiringOwnership() NamespaceEphemeralData namespaceEphemeralData = primaryLoadManager.tryAcquiringOwnership(bundle).get(); assertTrue(Set.of(pulsar1.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrl()) .contains(namespaceEphemeralData.getNativeUrl())); - admin.namespaces().deleteNamespace(namespace, true); + admin.namespaces().deleteNamespace(namespace); } @Test(timeOut = 30 * 1000) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index f3fabc910346d..2a8a849ef9c06 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -44,6 +44,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; @@ -72,6 +73,7 @@ import org.apache.pulsar.common.naming.NamespaceBundles; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.ServiceUnitId; +import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.LocalPolicies; @@ -94,6 +96,7 @@ import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Test(groups = "flaky") @@ -800,6 +803,30 @@ public void testModularLoadManagerRemoveBundleAndLoad() throws Exception { assertFalse(getResult.isPresent()); } + @DataProvider(name = "topicDomain") + public Object[] topicDomain() { + return new Object[]{ + TopicDomain.persistent.value(), + TopicDomain.non_persistent.value() + }; + } + + @Test(dataProvider = "topicDomain") + public void testCheckTopicExists(String topicDomain) throws Exception { + String topic = topicDomain + "://prop/ns-abc/" + UUID.randomUUID(); + admin.topics().createNonPartitionedTopic(topic); + Awaitility.await().untilAsserted(() -> { + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(topic)).get()); + }); + + String partitionedTopic = topicDomain + "://prop/ns-abc/" + UUID.randomUUID(); + admin.topics().createPartitionedTopic(partitionedTopic, 5); + Awaitility.await().untilAsserted(() -> { + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic)).get()); + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic + "-partition-2")).get()); + }); + } + /** * 1. Manually trigger "LoadReportUpdaterTask" * 2. Registry another new zk-node-listener "waitForBrokerChangeNotice". 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 3918dcbe86d66..f194dfe1340e9 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 @@ -18,6 +18,13 @@ */ package org.apache.pulsar.broker.service; +import static java.util.Map.entry; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType.destination_storage; +import static org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType.message_age; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.AssertionsForClassTypes.within; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; @@ -33,15 +40,18 @@ import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; 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.broker.stats.prometheus.PrometheusMetricsClient; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metrics; import org.apache.pulsar.client.admin.GetStatsOptions; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -73,6 +83,9 @@ @Test(groups = "broker") public class BacklogQuotaManagerTest { + private static final Logger log = LoggerFactory.getLogger(BacklogQuotaManagerTest.class); + + public static final String CLUSTER_NAME = "usc"; PulsarService pulsar; ServiceConfiguration config; @@ -80,6 +93,7 @@ public class BacklogQuotaManagerTest { PulsarAdmin admin; LocalBookkeeperEnsemble bkEnsemble; + PrometheusMetricsClient prometheusMetricsClient; private static final int TIME_TO_CHECK_BACKLOG_QUOTA = 2; private static final int MAX_ENTRIES_PER_LEDGER = 5; @@ -117,7 +131,7 @@ void setup() throws Exception { config.setMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort()); config.setAdvertisedAddress("localhost"); config.setWebServicePort(Optional.of(0)); - config.setClusterName("usc"); + config.setClusterName(CLUSTER_NAME); config.setBrokerShutdownTimeoutMs(0L); config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config.setBrokerServicePort(Optional.of(0)); @@ -136,6 +150,7 @@ void setup() throws Exception { adminUrl = new URL("http://127.0.0.1" + ":" + pulsar.getListenPortHTTP().get()); admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl.toString()).build(); + prometheusMetricsClient = new PrometheusMetricsClient("127.0.0.1", pulsar.getListenPortHTTP().get()); admin.clusters().createCluster("usc", ClusterData.builder().serviceUrl(adminUrl.toString()).build()); admin.tenants().createTenant("prop", @@ -190,7 +205,7 @@ private void rolloverStats() { } /** - * Readers should not effect backlog quota + * Readers should not affect backlog quota */ @Test public void testBacklogQuotaWithReader() throws Exception { @@ -202,18 +217,18 @@ public void testBacklogQuotaWithReader() throws Exception { .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) .build()); - try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS).build()) { final String topic1 = "persistent://prop/ns-quota/topic1" + UUID.randomUUID(); final int numMsgs = 20; Reader reader = client.newReader().topic(topic1).receiverQueueSize(1).startMessageId(MessageId.latest).create(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { content[0] = (byte) (content[0] + 1); - MessageId msgId = producer.send(content); + producer.send(content); } Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); @@ -259,7 +274,7 @@ public void testBacklogQuotaWithReader() throws Exception { // check reader can still read with out error while (true) { - Message msg = reader.readNext(5, TimeUnit.SECONDS); + Message msg = reader.readNext(5, SECONDS); if (msg == null) { break; } @@ -284,10 +299,11 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) .build()); - try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS).build();) { final String topic1 = "persistent://prop/ns-quota/topic1" + UUID.randomUUID(); final int numMsgs = 20; Reader reader = client.newReader().topic(topic1).receiverQueueSize(1).startMessageId(MessageId.latest).create(); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { @@ -327,10 +343,10 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { // 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 + // check reader can still read without error while (true) { - Message msg = reader.readNext(5, TimeUnit.SECONDS); + Message msg = reader.readNext(5, SECONDS); if (msg == null) { break; } @@ -341,6 +357,282 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { } } + @Test + public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientException, InterruptedException { + config.setPreciseTimeBasedBacklogQuotaCheck(true); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + final int sizeLimitBytes = 15 * 1024 * 1024; + final int timeLimitSeconds = 123; + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitSize(sizeLimitBytes) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + destination_storage); + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitTime(timeLimitSeconds) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + message_age); + + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) + .statsInterval(0, SECONDS).build()) { + final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); + + final String subName1 = "c1"; + final String subName2 = "c2"; + final int numMsgs = 4; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) + .acknowledgmentGroupTime(0, SECONDS) + .subscribe(); + Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2) + .acknowledgmentGroupTime(0, SECONDS) + .subscribe(); + Producer producer = createProducer(client, topic1); + + byte[] content = new byte[1024]; + for (int i = 0; i < numMsgs; i++) { + Thread.sleep(3000); // Guarantees if we use wrong message in age, to show up in failed test + producer.send(content); + } + + String c1MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName1).markDeletePosition; + + // Move subscription 1, one message, such that subscription 2 is the oldest + // S2 S1 + // 0 1 + Message oldestMessage = consumer1.receive(); + consumer1.acknowledge(oldestMessage); + + log.info("Subscription 1 moved 1 message. Now subscription 2 is the oldest. Oldest message:"+ + oldestMessage.getMessageId()); + + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + Metrics metrics = prometheusMetricsClient.getMetrics(); + TopicStats topicStats = getTopicStats(topic1); + + assertThat(topicStats.getBacklogQuotaLimitSize()).isEqualTo(sizeLimitBytes); + assertThat(topicStats.getBacklogQuotaLimitTime()).isEqualTo(timeLimitSeconds); + long expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - oldestMessage.getPublishTime()); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()) + .isCloseTo(expectedMessageAgeSeconds, within(1L)); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.tags).containsExactly( + entry("cluster", CLUSTER_NAME), + entry("namespace", namespace), + entry("topic", topic1)); + assertThat((long) backlogAgeMetric.value).isCloseTo(expectedMessageAgeSeconds, within(2L)); + + // Move subscription 2 away from being the oldest mark delete + // S2/S1 + // 0 1 + Message firstOldestMessage = consumer2.receive(); + consumer2.acknowledge(firstOldestMessage); + // We only read and not ack, since we just need its publish-timestamp for later assert + Message secondOldestMessage = consumer2.receive(); + + // Switch subscription 1 to be where subscription 2 was in terms of oldest mark delete + // S1 S2 + // 0 1 + consumer1.seek(MessageId.earliest); + + log.info("Subscription 1 moved to be the oldest"); + + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + metrics = prometheusMetricsClient.getMetrics(); + long actualAge = (long) metrics.findByNameAndLabels( + "pulsar_storage_backlog_age_seconds", "topic", topic1) + .get(0).value; + + expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - oldestMessage.getPublishTime()); + assertThat(actualAge).isCloseTo(expectedMessageAgeSeconds, within(2L)); + + topicStats = getTopicStats(topic1); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName1); + + long entriesReadBefore = getReadEntries(topic1); + + // Move subscription 1 passed subscription 2 + for (int i = 0; i < 3; i++) { + Message message = consumer1.receive(); + log.info("Subscription 1 about to ack message ID {}", message.getMessageId()); + consumer1.acknowledge(message); + } + + log.info("Subscription 1 moved 3 messages. Now subscription 2 is the oldest"); + waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + // Cache shouldn't be used, since position has changed + long readEntries = getReadEntries(topic1); + assertThat(readEntries).isGreaterThan(entriesReadBefore); + + topicStats = getTopicStats(topic1); + expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - secondOldestMessage.getPublishTime()); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedMessageAgeSeconds, within(2L)); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + + waitForQuotaCheckToRunTwice(); + + // Cache should be used, since position hasn't changed + assertThat(getReadEntries(topic1)).isEqualTo(readEntries); + } + } + + private long getReadEntries(String topic1) { + return ((PersistentTopic) pulsar.getBrokerService().getTopicReference(topic1).get()) + .getManagedLedger().getStats().getEntriesReadTotalCount(); + } + + @Test + public void backlogsStatsNotPrecise() throws PulsarAdminException, PulsarClientException, InterruptedException { + config.setPreciseTimeBasedBacklogQuotaCheck(false); + config.setManagedLedgerMaxEntriesPerLedger(6); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + final int sizeLimitBytes = 15 * 1024 * 1024; + final int timeLimitSeconds = 123; + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitSize(sizeLimitBytes) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + destination_storage); + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitTime(timeLimitSeconds) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + message_age); + + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) + .statsInterval(0, SECONDS).build()) { + final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); + + final String subName1 = "brandNewC1"; + final String subName2 = "brandNewC2"; + final int numMsgs = 5; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) + .acknowledgmentGroupTime(0, SECONDS) + .isAckReceiptEnabled(true) + .subscribe(); + Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2) + .acknowledgmentGroupTime(0, SECONDS) + .isAckReceiptEnabled(true) + .subscribe(); + Producer producer = createProducer(client, topic1); + + byte[] content = new byte[1024]; + for (int i = 0; i < numMsgs; i++) { + Thread.sleep(500); + producer.send(content); + } + + String c1MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName1).markDeletePosition; + + consumer1.acknowledge(consumer1.receive()); + log.info("Moved subscription 1, by 1 message"); + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + TopicStats topicStats = getTopicStats(topic1); + + // We have only one ledger, and it is not closed yet, so we can't tell the age until it is closed + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); + + Metrics metrics = prometheusMetricsClient.getMetrics(); + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.value).isEqualTo(-1); + + unloadAndLoadTopic(topic1, producer); + long unloadTime = System.currentTimeMillis(); + + waitForQuotaCheckToRunTwice(); + + topicStats = getTopicStats(topic1); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + // age is measured against the ledger closing time + long expectedAge = MILLISECONDS.toSeconds(System.currentTimeMillis() - unloadTime); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedAge, within(1L)); + + String c2MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName2).markDeletePosition; + Message message; + for (int i = 0; i < numMsgs-1; i++) { + consumer1.acknowledge(consumer1.receive()); + message = consumer2.receive(); + consumer2.acknowledge(message); + } + // At this point subscription 2 is the oldest + + waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForMarkDeletePositionToChange(topic1, subName2, c2MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + topicStats = getTopicStats(topic1); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + expectedAge = MILLISECONDS.toSeconds(System.currentTimeMillis() - unloadTime); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedAge, within(1L)); + } + config.setManagedLedgerMaxEntriesPerLedger(MAX_ENTRIES_PER_LEDGER); + } + + private void unloadAndLoadTopic(String topic, Producer producer) throws PulsarAdminException, + PulsarClientException { + admin.topics().unload(topic); + // This will load the topic + producer.send("Bla".getBytes()); + Awaitility.await().pollInterval(100, MILLISECONDS).atMost(5, SECONDS) + .until(() -> admin.topics().getInternalStats(topic).numberOfEntries > 0); + } + + private void waitForQuotaCheckToRunTwice() { + final long initialQuotaCheckCount = getQuotaCheckCount(); + Awaitility.await() + .pollInterval(1, SECONDS) + .atMost(TIME_TO_CHECK_BACKLOG_QUOTA*3, SECONDS) + .until(() -> getQuotaCheckCount() > initialQuotaCheckCount + 1); + } + + /** + * @return The new mark delete position + */ + private String waitForMarkDeletePositionToChange(String topic, + String subscriptionName, + String previousMarkDeletePosition) { + return Awaitility.await().pollInterval(1, SECONDS).atMost(5, SECONDS).until( + () -> admin.topics().getInternalStats(topic).cursors.get(subscriptionName).markDeletePosition, + markDeletePosition -> markDeletePosition != null && !markDeletePosition.equals(previousMarkDeletePosition)); + } + + private long getQuotaCheckCount() { + Metrics metrics = prometheusMetricsClient.getMetrics(); + return (long) metrics.findByNameAndLabels( + "pulsar_storage_backlog_quota_check_duration_seconds_count", + "cluster", CLUSTER_NAME) + .get(0).value; + } + /** * Time based backlog quota won't affect reader since broker doesn't keep track of consuming position for reader * and can't do message age check against the quota. @@ -356,7 +648,7 @@ public void testTriggerBacklogTimeQuotaWithReader() throws Exception { .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) .build()); - try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS).build();) { final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); final int numMsgs = 9; Reader reader = client.newReader().topic(topic1).receiverQueueSize(1).startMessageId(MessageId.latest).create(); @@ -402,7 +694,7 @@ public void testTriggerBacklogTimeQuotaWithReader() throws Exception { // check reader can still read without error while (true) { - Message msg = reader.readNext(5, TimeUnit.SECONDS); + Message msg = reader.readNext(5, SECONDS); if (msg == null) { break; } @@ -423,7 +715,7 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) .build()); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); @@ -433,7 +725,7 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -446,6 +738,8 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { TopicStats stats = getTopicStats(topic1); assertTrue(stats.getBacklogSize() < 10 * 1024, "Storage size is [" + stats.getStorageSize() + "]"); + assertThat(evictionCountMetric("prop/ns-quota", topic1, "size")).isEqualTo(1); + assertThat(evictionCountMetric("size")).isEqualTo(1); } @Test @@ -456,10 +750,10 @@ public void testConsumerBacklogEvictionTimeQuotaPrecise() throws Exception { BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic3" + UUID.randomUUID(); @@ -469,7 +763,7 @@ public void testConsumerBacklogEvictionTimeQuotaPrecise() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -488,8 +782,32 @@ public void testConsumerBacklogEvictionTimeQuotaPrecise() throws Exception { // All messages for both subscription should be cleaned up from backlog by backlog monitor task. assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 0); assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 0); + assertThat(evictionCountMetric("prop/ns-quota", topic1, "time")).isEqualTo(1); + assertThat(evictionCountMetric("time")).isEqualTo(1); + } + + @SuppressWarnings("SameParameterValue") + private long evictionCountMetric(String namespace, String topic, String quotaType) { + Metrics metrics = prometheusMetricsClient.getMetrics(); + Metric topicEvictionsTotal = metrics.findSingleMetricByNameAndLabels( + "pulsar_storage_backlog_quota_exceeded_evictions_total", + Pair.of("topic", topic), + Pair.of("quota_type", quotaType), + Pair.of("namespace", namespace), + Pair.of("cluster", CLUSTER_NAME)); + return (long) topicEvictionsTotal.value; + } + + private long evictionCountMetric(String quotaType) { + Metrics metrics = prometheusMetricsClient.getMetrics(); + Metric topicEvictionsTotal = metrics.findSingleMetricByNameAndLabels( + "pulsar_broker_storage_backlog_quota_exceeded_evictions_total", + Pair.of("quota_type", quotaType), + Pair.of("cluster", CLUSTER_NAME)); + return (long) topicEvictionsTotal.value; } + @Test(timeOut = 60000) public void testConsumerBacklogEvictionTimeQuota() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), @@ -498,9 +816,9 @@ public void testConsumerBacklogEvictionTimeQuota() throws Exception { BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic3" + UUID.randomUUID(); @@ -510,7 +828,7 @@ public void testConsumerBacklogEvictionTimeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -548,9 +866,9 @@ public void testConsumerBacklogEvictionTimeQuotaWithPartEviction() throws Except BacklogQuota.builder() .limitTime(5) // set limit time as 5 seconds .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic3" + UUID.randomUUID(); @@ -560,7 +878,7 @@ public void testConsumerBacklogEvictionTimeQuotaWithPartEviction() throws Except Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -602,17 +920,17 @@ public void testConsumerBacklogEvictionTimeQuotaWithEmptyLedger() throws Excepti BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic = "persistent://prop/ns-quota/topic4" + UUID.randomUUID(); final String subName = "c1"; Consumer consumer = client.newConsumer().topic(topic).subscriptionName(subName).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic); + Producer producer = createProducer(client, topic); producer.send(new byte[1024]); consumer.receive(); @@ -660,7 +978,7 @@ public void testConsumerBacklogEvictionWithAckSizeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -684,7 +1002,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuotaPrecise() throws Exceptio BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).build(); @@ -696,7 +1014,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuotaPrecise() throws Exceptio Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { @@ -734,7 +1052,7 @@ private Producer createProducer(PulsarClient client, String topic) throws PulsarClientException { return client.newProducer() .enableBatching(false) - .sendTimeout(2, TimeUnit.SECONDS) + .sendTimeout(2, SECONDS) .topic(topic) .create(); } @@ -753,7 +1071,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; List> messagesToAcknowledge = new ArrayList<>(); @@ -794,7 +1112,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuota() throws Exception { BacklogQuota.builder() .limitTime(2 * TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); Awaitility.await() .pollInterval(Duration.ofSeconds(1)) @@ -828,10 +1146,10 @@ public void testConcurrentAckAndEviction() throws Exception { final CountDownLatch counter = new CountDownLatch(2); final AtomicBoolean gotException = new AtomicBoolean(false); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); @Cleanup - PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); Consumer consumer1 = client2.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client2.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); @@ -871,7 +1189,7 @@ public void testConcurrentAckAndEviction() throws Exception { consumerThread.start(); // test hangs without timeout since there is nothing to consume due to eviction - counter.await(20, TimeUnit.SECONDS); + counter.await(20, SECONDS); assertFalse(gotException.get()); Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); rolloverStats(); @@ -900,13 +1218,13 @@ public void testNoEviction() throws Exception { final AtomicBoolean gotException = new AtomicBoolean(false); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); final Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); @Cleanup final PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); Thread producerThread = new Thread(() -> { try { @@ -964,16 +1282,16 @@ public void testEvictionMulti() throws Exception { final AtomicBoolean gotException = new AtomicBoolean(false); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); final Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); @Cleanup final PulsarClient client3 = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); @Cleanup final PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); Thread producerThread1 = new Thread(() -> { try { @@ -1037,7 +1355,7 @@ public void testEvictionMulti() throws Exception { producerThread2.start(); consumerThread1.start(); consumerThread2.start(); - counter.await(20, TimeUnit.SECONDS); + counter.await(20, SECONDS); assertFalse(gotException.get()); Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); rolloverStats(); @@ -1057,7 +1375,7 @@ public void testAheadProducerOnHold() throws Exception { .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/hold"; final String subName1 = "c1hold"; final int numMsgs = 10; @@ -1099,7 +1417,7 @@ public void testAheadProducerOnHoldTimeout() throws Exception { .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/holdtimeout"; final String subName1 = "c1holdtimeout"; boolean gotException = false; @@ -1137,7 +1455,7 @@ public void testProducerException() throws Exception { .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/except"; final String subName1 = "c1except"; boolean gotException = false; @@ -1182,7 +1500,7 @@ public void testProducerExceptionAndThenUnblockSizeQuota(boolean dedupTestSet) t .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/exceptandunblock"; final String subName1 = "c1except"; boolean gotException = false; @@ -1266,11 +1584,11 @@ public void testProducerExceptionAndThenUnblockTimeQuotaPrecise() throws Excepti BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/exceptandunblock2"; final String subName1 = "c1except"; boolean gotException = false; @@ -1332,10 +1650,10 @@ public void testProducerExceptionAndThenUnblockTimeQuota() throws Exception { BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/exceptandunblock2"; final String subName1 = "c1except"; boolean gotException = false; @@ -1406,7 +1724,7 @@ public void testBacklogQuotaInGB(boolean backlogQuotaSizeGB) throws Exception { admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl.toString()).build(); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); @@ -1416,7 +1734,7 @@ public void testBacklogQuotaInGB(boolean backlogQuotaSizeGB) throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index c6a6b5e4fb8ca..e6e45eebf8d24 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -94,15 +94,6 @@ public void cleanup() throws Exception { super.cleanup(); } - private void waitReplicatorStarted(String topicName) { - Awaitility.await().untilAsserted(() -> { - Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); - assertTrue(topicOptional2.isPresent()); - PersistentTopic persistentTopic2 = (PersistentTopic) topicOptional2.get(); - assertFalse(persistentTopic2.getProducers().isEmpty()); - }); - } - private void waitReplicatorStopped(String topicName) { Awaitility.await().untilAsserted(() -> { Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); @@ -475,8 +466,17 @@ public void testPartitionedTopicLevelReplicationRemoteTopicExist() throws Except admin2.topics().createPartitionedTopic(topicName, 2); admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); // Check the partitioned topic has been created at the remote cluster. - PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); - assertEquals(topicMetadata2.partitions, 2); + Awaitility.await().untilAsserted(() -> { + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 2); + }); + + // Expand partitions + admin2.topics().updatePartitionedTopic(topicName, 3); + Awaitility.await().untilAsserted(() -> { + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 3); + }); // cleanup. admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); waitReplicatorStopped(partition0); @@ -667,4 +667,115 @@ public void testUnFenceTopicToReuse() throws Exception { admin2.topics().delete(topicName); }); } + + @Test + public void testDeleteNonPartitionedTopic() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createNonPartitionedTopic(topicName); + + // Verify replicator works. + verifyReplicationWorks(topicName); + + // Disable replication. + setTopicLevelClusters(topicName, Arrays.asList(cluster1), admin1, pulsar1); + setTopicLevelClusters(topicName, Arrays.asList(cluster2), admin2, pulsar2); + + // Delete topic. + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + + // Verify the topic was deleted. + assertFalse(pulsar1.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName)).join()); + assertFalse(pulsar2.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName)).join()); + } + + @Test + public void testDeletePartitionedTopic() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createPartitionedTopic(topicName, 2); + + // Verify replicator works. + verifyReplicationWorks(topicName); + + // Disable replication. + setTopicLevelClusters(topicName, Arrays.asList(cluster1), admin1, pulsar1); + setTopicLevelClusters(topicName, Arrays.asList(cluster2), admin2, pulsar2); + + // Delete topic. + admin1.topics().deletePartitionedTopic(topicName); + if (!usingGlobalZK) { + admin2.topics().deletePartitionedTopic(topicName); + } + + // Verify the topic was deleted. + assertFalse(pulsar1.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(TopicName.get(topicName))); + assertFalse(pulsar2.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(TopicName.get(topicName))); + if (!usingGlobalZK) { + // So far, the topic partitions on the remote cluster are needed to delete manually when using global ZK. + assertFalse(pulsar1.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName).getPartition(0)).join()); + assertFalse(pulsar2.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName).getPartition(0)).join()); + assertFalse(pulsar1.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName).getPartition(1)).join()); + assertFalse(pulsar2.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName).getPartition(1)).join()); + } + } + + @Test + public void testNoExpandTopicPartitionsWhenDisableTopicLevelReplication() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createPartitionedTopic(topicName, 2); + + // Verify replicator works. + verifyReplicationWorks(topicName); + + // Disable topic level replication. + setTopicLevelClusters(topicName, Arrays.asList(cluster1), admin1, pulsar1); + setTopicLevelClusters(topicName, Arrays.asList(cluster2), admin2, pulsar2); + + // Expand topic. + admin1.topics().updatePartitionedTopic(topicName, 3); + assertEquals(admin1.topics().getPartitionedTopicMetadata(topicName).partitions, 3); + + // Wait for async tasks that were triggered by expanding topic partitions. + Thread.sleep(3 * 1000); + + + // Verify: the topics on the remote cluster did not been expanded. + assertEquals(admin2.topics().getPartitionedTopicMetadata(topicName).partitions, 2); + + cleanupTopics(() -> { + admin1.topics().deletePartitionedTopic(topicName, false); + admin2.topics().deletePartitionedTopic(topicName, false); + }); + } + + @Test + public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createPartitionedTopic(topicName, 2); + + // Verify replicator works. + verifyReplicationWorks(topicName); + + // Expand topic. + admin1.topics().updatePartitionedTopic(topicName, 3); + assertEquals(admin1.topics().getPartitionedTopicMetadata(topicName).partitions, 3); + + // Verify: the topics on the remote cluster will be expanded. + Awaitility.await().untilAsserted(() -> { + assertEquals(admin2.topics().getPartitionedTopicMetadata(topicName).partitions, 3); + }); + + cleanupTopics(() -> { + admin1.topics().deletePartitionedTopic(topicName, false); + admin2.topics().deletePartitionedTopic(topicName, false); + }); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index b4eed00c4470f..7372b2e478475 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -19,20 +19,35 @@ package org.apache.pulsar.broker.service; import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.assertEquals; import com.google.common.collect.Sets; import java.net.URL; import java.time.Duration; +import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; +import java.util.List; import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.Consumer; +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.common.naming.SystemTopicNames; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.tests.TestRetrySupport; @@ -49,13 +64,16 @@ public abstract class OneWayReplicatorTestBase extends TestRetrySupport { protected final String nonReplicatedNamespace = defaultTenant + "/ns1"; protected final String cluster1 = "r1"; + + protected boolean usingGlobalZK = false; + protected URL url1; protected URL urlTls1; protected ServiceConfiguration config1 = new ServiceConfiguration(); protected ZookeeperServerTest brokerConfigZk1; protected LocalBookkeeperEnsemble bkEnsemble1; protected PulsarService pulsar1; - protected BrokerService ns1; + protected BrokerService broker1; protected PulsarAdmin admin1; protected PulsarClient client1; @@ -66,7 +84,7 @@ public abstract class OneWayReplicatorTestBase extends TestRetrySupport { protected ZookeeperServerTest brokerConfigZk2; protected LocalBookkeeperEnsemble bkEnsemble2; protected PulsarService pulsar2; - protected BrokerService ns2; + protected BrokerService broker2; protected PulsarAdmin admin2; protected PulsarClient client2; @@ -74,8 +92,12 @@ protected void startZKAndBK() throws Exception { // Start ZK. brokerConfigZk1 = new ZookeeperServerTest(0); brokerConfigZk1.start(); - brokerConfigZk2 = new ZookeeperServerTest(0); - brokerConfigZk2.start(); + if (usingGlobalZK) { + brokerConfigZk2 = brokerConfigZk1; + } else { + brokerConfigZk2 = new ZookeeperServerTest(0); + brokerConfigZk2.start(); + } // Start BK. bkEnsemble1 = new LocalBookkeeperEnsemble(3, 0, () -> 0); @@ -89,23 +111,29 @@ protected void startBrokers() throws Exception { setConfigDefaults(config1, cluster1, bkEnsemble1, brokerConfigZk1); pulsar1 = new PulsarService(config1); pulsar1.start(); - ns1 = pulsar1.getBrokerService(); - + broker1 = pulsar1.getBrokerService(); url1 = new URL(pulsar1.getWebServiceAddress()); urlTls1 = new URL(pulsar1.getWebServiceAddressTls()); - admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); - client1 = PulsarClient.builder().serviceUrl(url1.toString()).build(); // Start region 2 setConfigDefaults(config2, cluster2, bkEnsemble2, brokerConfigZk2); pulsar2 = new PulsarService(config2); pulsar2.start(); - ns2 = pulsar2.getBrokerService(); - + broker2 = pulsar2.getBrokerService(); url2 = new URL(pulsar2.getWebServiceAddress()); urlTls2 = new URL(pulsar2.getWebServiceAddressTls()); + } + + protected void startAdminClient() throws Exception { + admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); admin2 = PulsarAdmin.builder().serviceHttpUrl(url2.toString()).build(); - client2 = PulsarClient.builder().serviceUrl(url2.toString()).build(); + } + + protected void startPulsarClient() throws Exception{ + ClientBuilder clientBuilder1 = PulsarClient.builder().serviceUrl(url1.toString()); + client1 = initClient(clientBuilder1); + ClientBuilder clientBuilder2 = PulsarClient.builder().serviceUrl(url2.toString()); + client2 = initClient(clientBuilder2); } protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { @@ -123,30 +151,32 @@ protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { .brokerServiceUrlTls(pulsar2.getBrokerServiceUrlTls()) .brokerClientTlsEnabled(false) .build()); - admin2.clusters().createCluster(cluster1, ClusterData.builder() - .serviceUrl(url1.toString()) - .serviceUrlTls(urlTls1.toString()) - .brokerServiceUrl(pulsar1.getBrokerServiceUrl()) - .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) - .brokerClientTlsEnabled(false) - .build()); - admin2.clusters().createCluster(cluster2, ClusterData.builder() - .serviceUrl(url2.toString()) - .serviceUrlTls(urlTls2.toString()) - .brokerServiceUrl(pulsar2.getBrokerServiceUrl()) - .brokerServiceUrlTls(pulsar2.getBrokerServiceUrlTls()) - .brokerClientTlsEnabled(false) - .build()); - admin1.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), Sets.newHashSet(cluster1, cluster2))); - admin2.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), - Sets.newHashSet(cluster1, cluster2))); - admin1.namespaces().createNamespace(replicatedNamespace, Sets.newHashSet(cluster1, cluster2)); - admin2.namespaces().createNamespace(replicatedNamespace); admin1.namespaces().createNamespace(nonReplicatedNamespace); - admin2.namespaces().createNamespace(nonReplicatedNamespace); + + if (!usingGlobalZK) { + admin2.clusters().createCluster(cluster1, ClusterData.builder() + .serviceUrl(url1.toString()) + .serviceUrlTls(urlTls1.toString()) + .brokerServiceUrl(pulsar1.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(false) + .build()); + admin2.clusters().createCluster(cluster2, ClusterData.builder() + .serviceUrl(url2.toString()) + .serviceUrlTls(urlTls2.toString()) + .brokerServiceUrl(pulsar2.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar2.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(false) + .build()); + admin2.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1, cluster2))); + admin2.namespaces().createNamespace(replicatedNamespace); + admin2.namespaces().createNamespace(nonReplicatedNamespace); + } + } protected void cleanupTopics(CleanupTopicAction cleanupTopicAction) throws Exception { @@ -154,6 +184,9 @@ protected void cleanupTopics(CleanupTopicAction cleanupTopicAction) throws Excep } protected void cleanupTopics(String namespace, CleanupTopicAction cleanupTopicAction) throws Exception { + if (usingGlobalZK) { + throw new IllegalArgumentException("The method cleanupTopics does not support for global ZK"); + } waitChangeEventsInit(namespace); admin1.namespaces().setNamespaceReplicationClusters(namespace, Collections.singleton(cluster1)); admin1.namespaces().unload(namespace); @@ -163,9 +196,16 @@ protected void cleanupTopics(String namespace, CleanupTopicAction cleanupTopicAc } protected void waitChangeEventsInit(String namespace) { - PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService() - .getTopic(namespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME, false) - .join().get(); + CompletableFuture> future = pulsar1.getBrokerService() + .getTopic(namespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME, false); + if (future == null) { + return; + } + Optional optional = future.join(); + if (!optional.isPresent()) { + return; + } + PersistentTopic topic = (PersistentTopic) optional.get(); Awaitility.await().atMost(Duration.ofSeconds(180)).untilAsserted(() -> { TopicStatsImpl topicStats = topic.getStats(true, false, false); topicStats.getSubscriptions().entrySet().forEach(entry -> { @@ -196,8 +236,12 @@ protected void setup() throws Exception { startBrokers(); + startAdminClient(); + createDefaultTenantsAndClustersAndNamespace(); + startPulsarClient(); + Thread.sleep(100); log.info("--- OneWayReplicatorTestBase::setup completed ---"); } @@ -229,11 +273,15 @@ protected void cleanup() throws Exception { // delete namespaces. waitChangeEventsInit(replicatedNamespace); admin1.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster1)); + if (!usingGlobalZK) { + admin2.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster2)); + } admin1.namespaces().deleteNamespace(replicatedNamespace); - admin2.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster2)); - admin2.namespaces().deleteNamespace(replicatedNamespace); admin1.namespaces().deleteNamespace(nonReplicatedNamespace); - admin2.namespaces().deleteNamespace(nonReplicatedNamespace); + if (!usingGlobalZK) { + admin2.namespaces().deleteNamespace(replicatedNamespace); + admin2.namespaces().deleteNamespace(nonReplicatedNamespace); + } // shutdown. markCurrentSetupNumberCleaned(); @@ -278,7 +326,7 @@ protected void cleanup() throws Exception { brokerConfigZk1.stop(); brokerConfigZk1 = null; } - if (brokerConfigZk2 != null) { + if (!usingGlobalZK && brokerConfigZk2 != null) { brokerConfigZk2.stop(); brokerConfigZk2 = null; } @@ -287,4 +335,109 @@ protected void cleanup() throws Exception { config1 = new ServiceConfiguration(); config2 = new ServiceConfiguration(); } + + protected void waitReplicatorStarted(String topicName) { + Awaitility.await().untilAsserted(() -> { + Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); + assertTrue(topicOptional2.isPresent()); + PersistentTopic persistentTopic2 = (PersistentTopic) topicOptional2.get(); + assertFalse(persistentTopic2.getProducers().isEmpty()); + }); + } + + protected PulsarClient initClient(ClientBuilder clientBuilder) throws Exception { + return clientBuilder.build(); + } + + protected void verifyReplicationWorks(String topic) throws Exception { + final String subscription = "__subscribe_1"; + final String msgValue = "__msg1"; + Producer producer1 = client1.newProducer(Schema.STRING).topic(topic).create(); + Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(topic).isAckReceiptEnabled(true) + .subscriptionName(subscription).subscribe(); + producer1.newMessage().value(msgValue).send(); + pulsar1.getBrokerService().checkReplicationPolicies(); + assertEquals(consumer2.receive(10, TimeUnit.SECONDS).getValue(), msgValue); + consumer2.unsubscribe(); + producer1.close(); + } + + protected void setTopicLevelClusters(String topic, List clusters, PulsarAdmin admin, + PulsarService pulsar) throws Exception { + Set expected = new HashSet<>(clusters); + TopicName topicName = TopicName.get(TopicName.get(topic).getPartitionedTopicName()); + int partitions = ensurePartitionsAreSame(topic); + admin.topics().setReplicationClusters(topic, clusters); + Awaitility.await().untilAsserted(() -> { + TopicPolicies policies = pulsar.getTopicPoliciesService().getTopicPolicies(topicName); + assertEquals(new HashSet<>(policies.getReplicationClusters()), expected); + if (partitions == 0) { + checkNonPartitionedTopicLevelClusters(topicName.toString(), clusters, admin, pulsar.getBrokerService()); + } else { + for (int i = 0; i < partitions; i++) { + checkNonPartitionedTopicLevelClusters(topicName.getPartition(i).toString(), clusters, admin, + pulsar.getBrokerService()); + } + } + }); + } + + protected void checkNonPartitionedTopicLevelClusters(String topic, List clusters, PulsarAdmin admin, + BrokerService broker) throws Exception { + CompletableFuture> future = broker.getTopic(topic, false); + if (future == null) { + return; + } + Optional optional = future.join(); + if (optional == null || !optional.isPresent()) { + return; + } + PersistentTopic persistentTopic = (PersistentTopic) optional.get(); + Set expected = new HashSet<>(clusters); + Set act = new HashSet<>(persistentTopic.getTopicPolicies().get().getReplicationClusters()); + assertEquals(act, expected); + } + + protected int ensurePartitionsAreSame(String topic) throws Exception { + TopicName topicName = TopicName.get(TopicName.get(topic).getPartitionedTopicName()); + boolean isPartitionedTopic1 = pulsar1.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources().partitionedTopicExists(topicName); + boolean isPartitionedTopic2 = pulsar2.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources().partitionedTopicExists(topicName); + if (isPartitionedTopic1 != isPartitionedTopic2) { + throw new IllegalArgumentException(String.format("Can not delete topic." + + " isPartitionedTopic1: %s, isPartitionedTopic2: %s", + isPartitionedTopic1, isPartitionedTopic2)); + } + if (!isPartitionedTopic1) { + return 0; + } + int partitions1 = pulsar1.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources().getPartitionedTopicMetadataAsync(topicName).join().get().partitions; + int partitions2 = pulsar2.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources().getPartitionedTopicMetadataAsync(topicName).join().get().partitions; + if (partitions1 != partitions2) { + throw new IllegalArgumentException(String.format("Can not delete topic." + + " partitions1: %s, partitions2: %s", + partitions1, partitions2)); + } + return partitions1; + } + + protected void deleteTopicAfterDisableTopicLevelReplication(String topic) throws Exception { + setTopicLevelClusters(topic, Arrays.asList(cluster1), admin1, pulsar1); + setTopicLevelClusters(topic, Arrays.asList(cluster1), admin2, pulsar2); + admin2.topics().setReplicationClusters(topic, Arrays.asList(cluster2)); + + int partitions = ensurePartitionsAreSame(topic); + + TopicName topicName = TopicName.get(TopicName.get(topic).getPartitionedTopicName()); + if (partitions != 0) { + admin1.topics().deletePartitionedTopic(topicName.toString()); + admin2.topics().deletePartitionedTopic(topicName.toString()); + } else { + admin1.topics().delete(topicName.toString()); + admin2.topics().delete(topicName.toString()); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java new file mode 100644 index 0000000000000..b4747a8bd0e47 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java @@ -0,0 +1,107 @@ +/* + * 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 lombok.extern.slf4j.Slf4j; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class OneWayReplicatorUsingGlobalZKTest extends OneWayReplicatorTest { + + @Override + @BeforeClass(alwaysRun = true, timeOut = 300000) + public void setup() throws Exception { + super.usingGlobalZK = true; + super.setup(); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @Test(enabled = false) + public void testReplicatorProducerStatInTopic() throws Exception { + super.testReplicatorProducerStatInTopic(); + } + + @Test(enabled = false) + public void testCreateRemoteConsumerFirst() throws Exception { + super.testReplicatorProducerStatInTopic(); + } + + @Test(enabled = false) + public void testTopicCloseWhenInternalProducerCloseErrorOnce() throws Exception { + super.testReplicatorProducerStatInTopic(); + } + + @Test(enabled = false) + public void testConcurrencyOfUnloadBundleAndRecreateProducer() throws Exception { + super.testConcurrencyOfUnloadBundleAndRecreateProducer(); + } + + @Test(enabled = false) + public void testPartitionedTopicLevelReplication() throws Exception { + super.testPartitionedTopicLevelReplication(); + } + + @Test(enabled = false) + public void testPartitionedTopicLevelReplicationRemoteTopicExist() throws Exception { + super.testPartitionedTopicLevelReplicationRemoteTopicExist(); + } + + @Test(enabled = false) + public void testPartitionedTopicLevelReplicationRemoteConflictTopicExist() throws Exception { + super.testPartitionedTopicLevelReplicationRemoteConflictTopicExist(); + } + + @Test(enabled = false) + public void testConcurrencyOfUnloadBundleAndRecreateProducer2() throws Exception { + super.testConcurrencyOfUnloadBundleAndRecreateProducer2(); + } + + @Test(enabled = false) + public void testUnFenceTopicToReuse() throws Exception { + super.testUnFenceTopicToReuse(); + } + + @Test + public void testDeleteNonPartitionedTopic() throws Exception { + super.testDeleteNonPartitionedTopic(); + } + + @Test + public void testDeletePartitionedTopic() throws Exception { + super.testDeletePartitionedTopic(); + } + + @Test(enabled = false) + public void testNoExpandTopicPartitionsWhenDisableTopicLevelReplication() throws Exception { + super.testNoExpandTopicPartitionsWhenDisableTopicLevelReplication(); + } + + @Test(enabled = false) + public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Exception { + super.testExpandTopicPartitionsOnNamespaceLevelReplication(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java new file mode 100644 index 0000000000000..3caf4a1f2398c --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java @@ -0,0 +1,262 @@ +/* + * 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.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; +import static org.apache.pulsar.transaction.coordinator.impl.MLTransactionLogImpl.TRANSACTION_LOG_PREFIX; +import static org.testng.Assert.assertEquals; +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 com.google.common.collect.Sets; +import java.util.Collections; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; +import org.apache.pulsar.client.api.ClientBuilder; +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.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.apache.pulsar.zookeeper.ZookeeperServerTest; +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 ReplicationTxnTest extends OneWayReplicatorTestBase { + + private boolean transactionBufferSegmentedSnapshotEnabled = false; + private int txnLogPartitions = 4; + + @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(); + } + + @Override + protected PulsarClient initClient(ClientBuilder clientBuilder) throws Exception { + return clientBuilder.enableTransaction(true).build(); + } + + @Override + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, + LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { + super.setConfigDefaults(config, clusterName, bookkeeperEnsemble, brokerConfigZk); + config.setSystemTopicEnabled(true); + config.setTopicLevelPoliciesEnabled(true); + config.setTransactionCoordinatorEnabled(true); + config.setTransactionLogBatchedWriteEnabled(true); + config.setTransactionPendingAckBatchedWriteEnabled(true); + config.setTransactionBufferSegmentedSnapshotEnabled(transactionBufferSegmentedSnapshotEnabled); + } + + @Override + protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { + super.createDefaultTenantsAndClustersAndNamespace(); + + // Create resource that transaction function relies on. + admin1.tenants().createTenant(SYSTEM_NAMESPACE.getTenant(), new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1, cluster2))); + admin1.namespaces().createNamespace(SYSTEM_NAMESPACE.toString(), 4); + pulsar1.getPulsarResources().getNamespaceResources().getPartitionedTopicResources().createPartitionedTopic( + SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, new PartitionedTopicMetadata(txnLogPartitions)); + //admin1.topics().createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN.toString(), 4); + + admin2.tenants().createTenant(SYSTEM_NAMESPACE.getTenant(), new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1, cluster2))); + admin2.namespaces().createNamespace(SYSTEM_NAMESPACE.toString(), 4); + pulsar2.getPulsarResources().getNamespaceResources().getPartitionedTopicResources().createPartitionedTopic( + SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, new PartitionedTopicMetadata(txnLogPartitions)); + } + + private void pubAndSubOneMsg(String topic, String subscription) throws Exception { + Consumer consumer1 = client1.newConsumer(Schema.STRING).topic(topic).subscriptionName(subscription) + .isAckReceiptEnabled(true).subscribe(); + Producer producer1 = client1.newProducer(Schema.STRING).topic(topic).create(); + producer1.newMessage().value("msg1").send(); + // start txn. + Transaction txn = client1.newTransaction().withTransactionTimeout(1, TimeUnit.MINUTES).build().get(); + // consume. + Message c1Msg1 = consumer1.receive(5, TimeUnit.SECONDS); + assertNotNull(c1Msg1); + assertEquals(c1Msg1.getValue(), "msg1"); + consumer1.acknowledgeAsync(c1Msg1.getMessageId(), txn).join(); + // send. + producer1.newMessage(txn).value("msg2").send(); + // commit. + txn.commit().get(); + + // Consume the msg with TXN. + Message c1Msg2 = consumer1.receive(5, TimeUnit.SECONDS); + assertNotNull(c1Msg2); + assertEquals(c1Msg2.getValue(), "msg2"); + consumer1.acknowledgeAsync(c1Msg2.getMessageId()).join(); + + // Consume messages on the remote cluster. + Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(topic).subscriptionName(subscription).subscribe(); + Message c2Msg1 = consumer2.receive(15, TimeUnit.SECONDS); + assertNotNull(c2Msg1); + MessageMetadata msgMetadata1 = WhiteboxImpl.getInternalState(c2Msg1, "msgMetadata"); + // Verify: the messages replicated has no TXN id. + assertFalse(msgMetadata1.hasTxnidMostBits()); + assertFalse(msgMetadata1.hasTxnidLeastBits()); + consumer2.acknowledge(c2Msg1); + Message c2Msg2 = consumer2.receive(15, TimeUnit.SECONDS); + assertNotNull(c2Msg2); + MessageMetadata msgMetadata2 = WhiteboxImpl.getInternalState(c2Msg2, "msgMetadata"); + // Verify: the messages replicated has no TXN id. + assertFalse(msgMetadata2.hasTxnidMostBits()); + assertFalse(msgMetadata2.hasTxnidLeastBits()); + consumer2.acknowledge(c2Msg2); + + // cleanup. + producer1.close(); + consumer1.close(); + consumer2.close(); + } + + private void verifyNoReplicator(BrokerService broker, TopicName topicName) throws Exception { + String tpStr = topicName.toString(); + CompletableFuture> future = broker.getTopic(tpStr, true); + if (future == null) { + return; + } + PersistentTopic persistentTopic = (PersistentTopic) future.join().get(); + assertTrue(persistentTopic.getReplicators().isEmpty()); + } + + @Test + public void testTxnLogNotBeReplicated() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp"); + final String subscription = "s1"; + admin1.topics().createNonPartitionedTopic(topic); + waitReplicatorStarted(topic); + admin1.topics().createSubscription(topic, subscription, MessageId.earliest); + admin2.topics().createSubscription(topic, subscription, MessageId.earliest); + // Pub & Sub. + pubAndSubOneMsg(topic, subscription); + // To cover more cases, sleep 3s. + Thread.sleep(3000); + + // Verify: messages on the TXN system topic did not been replicated. + // __transaction_log_: it only uses ML, will not create topic. + for (int i = 0; i < txnLogPartitions; i++) { + TopicName txnLog = TopicName.get(TopicDomain.persistent.value(), + NamespaceName.SYSTEM_NAMESPACE, TRANSACTION_LOG_PREFIX + i); + assertNotNull(pulsar1.getManagedLedgerFactory() + .getManagedLedgerInfo(txnLog.getPersistenceNamingEncoding())); + assertFalse(broker1.getTopics().containsKey(txnLog.toString())); + } + // __transaction_pending_ack: it only uses ML, will not create topic. + TopicName pendingAck = TopicName.get( + MLPendingAckStore.getTransactionPendingAckStoreSuffix(topic, subscription)); + assertNotNull(pulsar1.getManagedLedgerFactory() + .getManagedLedgerInfo(pendingAck.getPersistenceNamingEncoding())); + assertFalse(broker1.getTopics().containsKey(pendingAck.toString())); + // __transaction_buffer_snapshot. + verifyNoReplicator(broker1, TopicName.get(TopicDomain.persistent.value(), + TopicName.get(topic).getNamespaceObject(), + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT)); + verifyNoReplicator(broker1, TopicName.get(TopicDomain.persistent.value(), + TopicName.get(topic).getNamespaceObject(), + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS)); + verifyNoReplicator(broker1, TopicName.get(TopicDomain.persistent.value(), + TopicName.get(topic).getNamespaceObject(), + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES)); + + // cleanup. + cleanupTopics(() -> { + admin1.topics().delete(topic); + admin2.topics().delete(topic); + try { + admin1.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + try { + admin2.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + }); + } + + @Test + public void testOngoingMessagesWillNotBeReplicated() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp"); + final String subscription = "s1"; + admin1.topics().createNonPartitionedTopic(topic); + waitReplicatorStarted(topic); + admin1.topics().createSubscription(topic, subscription, MessageId.earliest); + admin2.topics().createSubscription(topic, subscription, MessageId.earliest); + // Pub without commit. + Producer producer1 = client1.newProducer(Schema.STRING).topic(topic).create(); + Transaction txn = client1.newTransaction().withTransactionTimeout(1, TimeUnit.HOURS).build().get(); + producer1.newMessage(txn).value("msg1").send(); + // Verify: receive nothing on the remote cluster. + Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(topic).subscriptionName(subscription).subscribe(); + Message msg = consumer2.receive(15, TimeUnit.SECONDS); + assertNull(msg); + // Verify: the repl cursor is not end of the topic. + PersistentTopic persistentTopic = (PersistentTopic) broker1.getTopic(topic, false).join().get(); + GeoPersistentReplicator replicator = + (GeoPersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + assertTrue(replicator.getCursor().hasMoreEntries()); + + // cleanup. + producer1.close(); + consumer2.close(); + cleanupTopics(() -> { + admin1.topics().delete(topic); + admin2.topics().delete(topic); + TopicName pendingAck = TopicName.get( + MLPendingAckStore.getTransactionPendingAckStoreSuffix(topic, subscription)); + try { + admin1.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + try { + admin2.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + }); + } +} 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 16bce498e913f..ad7728319c9a7 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 @@ -21,6 +21,7 @@ import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs; import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgsRecordingInvocations; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyInt; @@ -1847,14 +1848,14 @@ public void testBrokerClosedProducerClientRecreatesProducerThenSendCommand() thr ByteBuf clientCommand1 = Commands.newProducer(successTopicName, 1 /* producer id */, 1 /* request id */, producerName, Collections.emptyMap(), false); channel.writeInbound(clientCommand1); - assertTrue(getResponse() instanceof CommandProducerSuccess); + assertThat(getResponse()).isInstanceOf(CommandProducerSuccess.class); // Call disconnect method on producer to trigger activity similar to unloading Producer producer = serverCnx.getProducers().get(1).get(); assertNotNull(producer); producer.disconnect(); channel.runPendingTasks(); - assertTrue(getResponse() instanceof CommandCloseProducer); + assertThat(getResponse()).isInstanceOf(CommandCloseProducer.class); // Send message and expect no response sendMessage(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java index b11946069c9dd..fd08f284bbf99 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -128,6 +129,29 @@ public void testSeek() throws Exception { assertEquals(sub.getNumberOfEntriesInBacklog(false), 0); } + @Test + public void testSeekIsByReceive() throws PulsarClientException { + final String topicName = "persistent://prop/use/ns-abc/testSeek"; + + Producer producer = pulsarClient.newProducer().topic(topicName).create(); + + String subscriptionName = "my-subscription"; + org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) + .subscriptionName(subscriptionName) + .subscribe(); + + List messageIds = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + String message = "my-message-" + i; + MessageId msgId = producer.send(message.getBytes()); + messageIds.add(msgId); + } + + consumer.seek(messageIds.get(5)); + Message message = consumer.receive(); + assertThat(message.getMessageId()).isEqualTo(messageIds.get(6)); + } + @Test public void testSeekForBatch() throws Exception { final String topicName = "persistent://prop/use/ns-abcd/testSeekForBatch"; 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 981feb002871e..ff8e418c024a0 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 @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.service.persistent; import static org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.CURSOR_INTERNAL_PROPERTY_PREFIX; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @@ -42,7 +44,6 @@ 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.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -219,9 +220,9 @@ public void testBucketDelayedIndexMetrics() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, true, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - List bucketsMetrics = + List bucketsMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_total").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt bucketsSum = new MutableInt(); @@ -230,12 +231,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { bucketsSum.add(metric.value); }); assertEquals(6, bucketsSum.intValue()); - Optional bucketsTopicMetric = + Optional bucketsTopicMetric = bucketsMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(bucketsTopicMetric.isPresent()); assertEquals(bucketsSum.intValue(), bucketsTopicMetric.get().value); - List loadedIndexMetrics = + List loadedIndexMetrics = metricsMap.get("pulsar_delayed_message_index_loaded").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt loadedIndexSum = new MutableInt(); @@ -244,12 +245,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { loadedIndexSum.add(metric.value); }).count(); assertEquals(2, count); - Optional loadedIndexTopicMetrics = + Optional loadedIndexTopicMetrics = bucketsMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(loadedIndexTopicMetrics.isPresent()); assertEquals(loadedIndexSum.intValue(), loadedIndexTopicMetrics.get().value); - List snapshotSizeBytesMetrics = + List snapshotSizeBytesMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_snapshot_size_bytes").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt snapshotSizeBytesSum = new MutableInt(); @@ -259,12 +260,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { snapshotSizeBytesSum.add(metric.value); }).count(); assertEquals(2, count); - Optional snapshotSizeBytesTopicMetrics = + Optional snapshotSizeBytesTopicMetrics = snapshotSizeBytesMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(snapshotSizeBytesTopicMetrics.isPresent()); assertEquals(snapshotSizeBytesSum.intValue(), snapshotSizeBytesTopicMetrics.get().value); - List opCountMetrics = + List opCountMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_op_count").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt opCountMetricsSum = new MutableInt(); @@ -276,14 +277,14 @@ public void testBucketDelayedIndexMetrics() throws Exception { opCountMetricsSum.add(metric.value); }).count(); assertEquals(2, count); - Optional opCountTopicMetrics = + Optional opCountTopicMetrics = opCountMetrics.stream() .filter(metric -> metric.tags.get("state").equals("succeed") && metric.tags.get("type") .equals("create") && !metric.tags.containsKey("subscription")).findFirst(); assertTrue(opCountTopicMetrics.isPresent()); assertEquals(opCountMetricsSum.intValue(), opCountTopicMetrics.get().value); - List opLatencyMetrics = + List opLatencyMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_op_latency_ms").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt opLatencyMetricsSum = new MutableInt(); @@ -295,7 +296,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { opLatencyMetricsSum.add(metric.value); }).count(); assertTrue(count >= 2); - Optional opLatencyTopicMetrics = + Optional opLatencyTopicMetrics = opCountMetrics.stream() .filter(metric -> metric.tags.get("type").equals("create") && !metric.tags.containsKey("subscription")).findFirst(); @@ -304,9 +305,9 @@ public void testBucketDelayedIndexMetrics() throws Exception { ByteArrayOutputStream namespaceOutput = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, false, true, true, namespaceOutput); - Multimap namespaceMetricsMap = PrometheusMetricsTest.parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); + Multimap namespaceMetricsMap = parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); - Optional namespaceMetric = + Optional namespaceMetric = namespaceMetricsMap.get("pulsar_delayed_message_index_bucket_total").stream().findFirst(); assertTrue(namespaceMetric.isPresent()); assertEquals(6, namespaceMetric.get().value); 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 8130c818e3a8a..d523586c2e2d3 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 @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.doAnswer; @@ -71,7 +73,6 @@ 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.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -448,14 +449,14 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); - Collection metrics = metricsMap.get("pulsar_delayed_message_index_size_bytes"); + Multimap metricsMap = parseMetrics(metricsStr); + Collection metrics = metricsMap.get("pulsar_delayed_message_index_size_bytes"); Assert.assertTrue(metrics.size() > 0); int topicLevelNum = 0; int namespaceLevelNum = 0; int subscriptionLevelNum = 0; - for (PrometheusMetricsTest.Metric metric : metrics) { + for (Metric metric : metrics) { if (exposeTopicLevelMetrics && metric.tags.get("topic").equals(topic)) { Assert.assertTrue(metric.value > 0); topicLevelNum++; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java index 1c4f88bc0273c..6c847db05a0e3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java @@ -239,9 +239,9 @@ public void testFilter() throws Exception { hasFilterField.setAccessible(true); NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); - EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter1, narClassLoader); + EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); EntryFilter filter2 = new EntryFilter2Test(); - EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter2, narClassLoader); + EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter2, narClassLoader, false); field.set(dispatcher, List.of(loader1, loader2)); hasFilterField.set(dispatcher, true); @@ -371,9 +371,9 @@ public void testFilteredMsgCount(String topic) throws Throwable { hasFilterField.setAccessible(true); NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); - EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader); + EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); EntryFilter filter2 = new EntryFilter2Test(); - EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader); + EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader, false); field.set(dispatcher, List.of(loader1, loader2)); hasFilterField.set(dispatcher, true); @@ -463,10 +463,10 @@ public void testEntryFilterRescheduleMessageDependingOnConsumerSharedSubscriptio NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); EntryFilterWithClassLoader loader1 = - spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader); + spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); EntryFilter filter2 = new EntryFilterTest(); EntryFilterWithClassLoader loader2 = - spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader); + spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader, false); field.set(dispatcher, List.of(loader1, loader2)); hasFilterField.set(dispatcher, true); 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 3623a47d9ef67..9e455f60b6e41 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 @@ -18,7 +18,8 @@ */ package org.apache.pulsar.broker.service.schema; -import static org.testng.Assert.assertThrows; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertFalse; @@ -46,8 +47,6 @@ 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.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; @@ -123,29 +122,29 @@ public void testSchemaRegistryMetrics() throws Exception { PrometheusMetricsTestUtil.generate(pulsar, false, false, false, output); output.flush(); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection delMetrics = metrics.get("pulsar_schema_del_ops_failed_total"); + Collection delMetrics = metrics.get("pulsar_schema_del_ops_failed_total"); Assert.assertEquals(delMetrics.size(), 0); - Collection getMetrics = metrics.get("pulsar_schema_get_ops_failed_total"); + Collection getMetrics = metrics.get("pulsar_schema_get_ops_failed_total"); Assert.assertEquals(getMetrics.size(), 0); - Collection putMetrics = metrics.get("pulsar_schema_put_ops_failed_total"); + Collection putMetrics = metrics.get("pulsar_schema_put_ops_failed_total"); Assert.assertEquals(putMetrics.size(), 0); - Collection deleteLatency = metrics.get("pulsar_schema_del_ops_latency_count"); - for (PrometheusMetricsTest.Metric metric : deleteLatency) { + Collection deleteLatency = metrics.get("pulsar_schema_del_ops_latency_count"); + for (Metric metric : deleteLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } - Collection getLatency = metrics.get("pulsar_schema_get_ops_latency_count"); - for (PrometheusMetricsTest.Metric metric : getLatency) { + Collection getLatency = metrics.get("pulsar_schema_get_ops_latency_count"); + for (Metric metric : getLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } - Collection putLatency = metrics.get("pulsar_schema_put_ops_latency_count"); - for (PrometheusMetricsTest.Metric metric : putLatency) { + Collection putLatency = metrics.get("pulsar_schema_put_ops_latency_count"); + for (Metric metric : putLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } @@ -406,7 +405,7 @@ public void testKeyValueSchema() throws Exception { .build(), SchemaInfo.builder().type(SchemaType.BOOLEAN).schema(new byte[0]) .build(), KeyValueEncodingType.SEPARATED); - assertThrows(PulsarAdminException.ServerSideErrorException.class, () -> admin.schemas().testCompatibility(topicName, schemaInfo)); + Assert.assertTrue(admin.schemas().testCompatibility(topicName, schemaInfo).isCompatibility()); admin.schemas().createSchema(topicName, schemaInfo); final IsCompatibilityResponse isCompatibilityResponse = admin.schemas().testCompatibility(topicName, schemaInfo); 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 de65d5db56475..5aeed40107d5d 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 @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.stats; import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.Mockito.mock; import static org.testng.Assert.assertNotEquals; import static org.testng.AssertJUnit.assertEquals; @@ -336,11 +338,11 @@ private void testMessageAckRateMetric(String topicName, boolean exposeTopicLevel PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricStr); - Collection ackRateMetric = metricsMap.get("pulsar_consumer_msg_ack_rate"); + Multimap metricsMap = parseMetrics(metricStr); + Collection ackRateMetric = metricsMap.get("pulsar_consumer_msg_ack_rate"); String rateOutMetricName = exposeTopicLevelMetrics ? "pulsar_consumer_msg_rate_out" : "pulsar_rate_out"; - Collection rateOutMetric = metricsMap.get(rateOutMetricName); + Collection rateOutMetric = metricsMap.get(rateOutMetricName); Assert.assertTrue(ackRateMetric.size() > 0); Assert.assertTrue(rateOutMetric.size() > 0); @@ -407,7 +409,7 @@ public void testAvgMessagesPerEntry() throws Exception { EntryFilter filter = new EntryFilterProducerTest(); EntryFilterWithClassLoader loader = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter, - narClassLoader); + narClassLoader, false); Pair> entryFilters = Pair.of("filter", List.of(loader)); PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService() 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 7368d42355c1b..726bde3f3d0a9 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 @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.stats; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; import java.util.Collection; @@ -101,12 +103,12 @@ public void testMetadataStoreStats() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); String metricsDebugMessage = "Assertion failed with metrics:\n" + metricsStr + "\n"; - Collection opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum"); - Collection putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total"); + Collection opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum"); + Collection putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total"); Assert.assertTrue(opsLatency.size() > 1, metricsDebugMessage); Assert.assertTrue(putBytes.size() > 1, metricsDebugMessage); @@ -116,7 +118,7 @@ public void testMetadataStoreStats() throws Exception { expectedMetadataStoreName.add(MetadataStoreConfig.CONFIGURATION_METADATA_STORE); AtomicInteger matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : opsLatency) { + for (Metric m : opsLatency) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (!isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -150,7 +152,7 @@ public void testMetadataStoreStats() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size() * 6); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : putBytes) { + for (Metric m : putBytes) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (!isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -191,12 +193,12 @@ public void testBatchMetadataStoreMetrics() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - Collection executorQueueSize = metricsMap.get("pulsar_batch_metadata_store_executor_queue_size"); - Collection opsWaiting = metricsMap.get("pulsar_batch_metadata_store_queue_wait_time_ms" + "_sum"); - Collection batchExecuteTime = metricsMap.get("pulsar_batch_metadata_store_batch_execute_time_ms" + "_sum"); - Collection opsPerBatch = metricsMap.get("pulsar_batch_metadata_store_batch_size" + "_sum"); + Collection executorQueueSize = metricsMap.get("pulsar_batch_metadata_store_executor_queue_size"); + Collection opsWaiting = metricsMap.get("pulsar_batch_metadata_store_queue_wait_time_ms" + "_sum"); + Collection batchExecuteTime = metricsMap.get("pulsar_batch_metadata_store_batch_execute_time_ms" + "_sum"); + Collection opsPerBatch = metricsMap.get("pulsar_batch_metadata_store_batch_size" + "_sum"); String metricsDebugMessage = "Assertion failed with metrics:\n" + metricsStr + "\n"; @@ -210,7 +212,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { expectedMetadataStoreName.add(MetadataStoreConfig.CONFIGURATION_METADATA_STORE); AtomicInteger matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : executorQueueSize) { + for (Metric m : executorQueueSize) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -221,7 +223,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : opsWaiting) { + for (Metric m : opsWaiting) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -232,7 +234,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : batchExecuteTime) { + for (Metric m : batchExecuteTime) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -243,7 +245,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : opsPerBatch) { + for (Metric m : opsPerBatch) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { 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 244d9eaa81274..971ea0d435f24 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 @@ -21,14 +21,14 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; 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; -import com.google.common.base.MoreObjects; import com.google.common.base.Splitter; -import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; import io.jsonwebtoken.SignatureAlgorithm; import io.prometheus.client.Collector; @@ -53,7 +53,6 @@ import java.util.Properties; import java.util.Random; import java.util.Set; -import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -1920,62 +1919,6 @@ public void testMetricsGroupedByTypeDefinitions() throws Exception { p2.close(); } - /** - * Hacky parsing of Prometheus text format. Should be good enough for unit tests - */ - public static Multimap parseMetrics(String metrics) { - Multimap parsed = ArrayListMultimap.create(); - - // Example of lines are - // jvm_threads_current{cluster="standalone",} 203.0 - // or - // pulsar_subscriptions_count{cluster="standalone", namespace="public/default", - // topic="persistent://public/default/test-2"} 0.0 - Pattern pattern = Pattern.compile("^(\\w+)\\{([^\\}]+)\\}\\s([+-]?[\\d\\w\\.-]+)$"); - Pattern tagsPattern = Pattern.compile("(\\w+)=\"([^\"]+)\"(,\\s?)?"); - - Splitter.on("\n").split(metrics).forEach(line -> { - if (line.isEmpty() || line.startsWith("#")) { - return; - } - - Matcher matcher = pattern.matcher(line); - assertTrue(matcher.matches(), "line " + line + " does not match pattern " + pattern); - String name = matcher.group(1); - - Metric m = new Metric(); - String numericValue = matcher.group(3); - if (numericValue.equalsIgnoreCase("-Inf")) { - m.value = Double.NEGATIVE_INFINITY; - } else if (numericValue.equalsIgnoreCase("+Inf")) { - m.value = Double.POSITIVE_INFINITY; - } else { - m.value = Double.parseDouble(numericValue); - } - String tags = matcher.group(2); - Matcher tagsMatcher = tagsPattern.matcher(tags); - while (tagsMatcher.find()) { - String tag = tagsMatcher.group(1); - String value = tagsMatcher.group(2); - m.tags.put(tag, value); - } - - parsed.put(name, m); - }); - - return parsed; - } - - public static class Metric { - public Map tags = new TreeMap<>(); - public double value; - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("tags", tags).add("value", value).toString(); - } - } - /** * Test both subscription and topic name with special characters. * @throws Exception 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 83e6f43cbafcc..bc4cb73e5b6fe 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 @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.stats; import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.Mockito.mock; import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; @@ -84,7 +86,7 @@ protected void cleanup() throws Exception { @Test public void testConsumersAfterMarkDelete() throws PulsarClientException, PulsarAdminException { final String topicName = "persistent://my-property/my-ns/testConsumersAfterMarkDelete-" - + UUID.randomUUID().toString(); + + UUID.randomUUID(); final String subName = "my-sub"; Consumer consumer1 = pulsarClient.newConsumer() @@ -206,7 +208,7 @@ public void testSubscriptionStats(final String topic, final String subName, bool NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); EntryFilterWithClassLoader loader1 = - spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader); + spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); field.set(dispatcher, List.of(loader1)); hasFilterField.set(dispatcher, true); } @@ -233,15 +235,15 @@ public void testSubscriptionStats(final String topic, final String subName, bool ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, enableTopicStats, false, false, output); String metricsStr = output.toString(); - Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection throughFilterMetrics = + Collection throughFilterMetrics = metrics.get("pulsar_subscription_filter_processed_msg_count"); - Collection acceptedMetrics = + Collection acceptedMetrics = metrics.get("pulsar_subscription_filter_accepted_msg_count"); - Collection rejectedMetrics = + Collection rejectedMetrics = metrics.get("pulsar_subscription_filter_rejected_msg_count"); - Collection rescheduledMetrics = + Collection rescheduledMetrics = metrics.get("pulsar_subscription_filter_rescheduled_msg_count"); if (enableTopicStats) { 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 85c6dd795d759..8d5cb9dc39148 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 @@ -19,7 +19,8 @@ package org.apache.pulsar.broker.stats; import static com.google.common.base.Preconditions.checkArgument; -import static org.apache.pulsar.broker.stats.PrometheusMetricsTest.parseMetrics; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -119,8 +120,8 @@ public void testTransactionCoordinatorMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_txn_active_count"); + Multimap metrics = parseMetrics(metricsStr); + Collection metric = metrics.get("pulsar_txn_active_count"); assertEquals(metric.size(), 2); metric.forEach(item -> { if ("0".equals(item.tags.get("coordinator_id"))) { @@ -187,9 +188,9 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_txn_created_total"); + Collection metric = metrics.get("pulsar_txn_created_total"); assertEquals(metric.size(), 1); metric.forEach(item -> assertEquals(item.value, txnCount)); @@ -274,9 +275,9 @@ public void testManagedLedgerMetrics() throws Exception { PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_storage_size"); + Collection metric = metrics.get("pulsar_storage_size"); checkManagedLedgerMetrics(subName, 32, metric); checkManagedLedgerMetrics(MLTransactionLogImpl.TRANSACTION_SUBSCRIPTION_NAME, 252, metric); @@ -336,12 +337,12 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_storage_size"); + Collection metric = metrics.get("pulsar_storage_size"); checkManagedLedgerMetrics(subName, 32, metric); //No statistics of the pendingAck are generated when the pendingAck is not initialized. - for (PrometheusMetricsTest.Metric metric1 : metric) { + for (Metric metric1 : metric) { if (metric1.tags.containsValue(subName2)) { Assert.fail(); } @@ -431,9 +432,9 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { } - private void checkManagedLedgerMetrics(String tag, double value, Collection metrics) { + private void checkManagedLedgerMetrics(String tag, double value, Collection metrics) { boolean exist = false; - for (PrometheusMetricsTest.Metric metric1 : metrics) { + for (Metric metric1 : metrics) { if (metric1.tags.containsValue(tag)) { assertEquals(metric1.value, value); exist = true; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java index e63f644f3d0e9..cf923df0411dd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java @@ -32,6 +32,7 @@ import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl; @@ -102,6 +103,8 @@ public void testGenerateSubscriptionsStats() { when(topic.getReplicators()).thenReturn(ConcurrentOpenHashMap.newBuilder().build()); when(topic.getManagedLedger()).thenReturn(ml); when(topic.getBacklogQuota(Mockito.any())).thenReturn(Mockito.mock(BacklogQuota.class)); + PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); + when(topic.getPersistentTopicMetrics()).thenReturn(persistentTopicMetrics); topicsMap.put("my-topic", topic); PrometheusMetricStreams metricStreams = Mockito.spy(new PrometheusMetricStreams()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java new file mode 100644 index 0000000000000..6fd509690278d --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java @@ -0,0 +1,154 @@ +/* + * 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.assertj.core.api.Fail.fail; +import static org.testng.Assert.assertTrue; +import com.google.common.base.MoreObjects; +import com.google.common.base.Splitter; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; +import io.restassured.RestAssured; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.commons.lang3.tuple.Pair; + +public class PrometheusMetricsClient { + private final String host; + private final int port; + + public PrometheusMetricsClient(String host, int port) { + this.host = host; + this.port = port; + } + + @SuppressWarnings("HttpUrlsUsage") + public Metrics getMetrics() { + String metrics = RestAssured.given().baseUri("http://" + host).port(port).get("/metrics").asString(); + return new Metrics(parseMetrics(metrics)); + } + + /** + * Hacky parsing of Prometheus text format. Should be good enough for unit tests + */ + public static Multimap parseMetrics(String metrics) { + Multimap parsed = ArrayListMultimap.create(); + + // Example of lines are + // jvm_threads_current{cluster="standalone",} 203.0 + // or + // pulsar_subscriptions_count{cluster="standalone", namespace="public/default", + // topic="persistent://public/default/test-2"} 0.0 + Pattern pattern = Pattern.compile("^(\\w+)\\{([^}]+)}\\s([+-]?[\\d\\w.-]+)$"); + Pattern tagsPattern = Pattern.compile("(\\w+)=\"([^\"]+)\"(,\\s?)?"); + + Splitter.on("\n").split(metrics).forEach(line -> { + if (line.isEmpty() || line.startsWith("#")) { + return; + } + + Matcher matcher = pattern.matcher(line); + assertTrue(matcher.matches(), "line " + line + " does not match pattern " + pattern); + String name = matcher.group(1); + + Metric m = new Metric(); + String numericValue = matcher.group(3); + if (numericValue.equalsIgnoreCase("-Inf")) { + m.value = Double.NEGATIVE_INFINITY; + } else if (numericValue.equalsIgnoreCase("+Inf")) { + m.value = Double.POSITIVE_INFINITY; + } else { + m.value = Double.parseDouble(numericValue); + } + String tags = matcher.group(2); + Matcher tagsMatcher = tagsPattern.matcher(tags); + while (tagsMatcher.find()) { + String tag = tagsMatcher.group(1); + String value = tagsMatcher.group(2); + m.tags.put(tag, value); + } + + parsed.put(name, m); + }); + + return parsed; + } + + public static class Metric { + public Map tags = new TreeMap<>(); + public double value; + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("tags", tags).add("value", value).toString(); + } + + public boolean contains(String labelName, String labelValue) { + String value = tags.get(labelName); + return value != null && value.equals(labelValue); + } + } + + public static class Metrics { + final Multimap nameToDataPoints; + + public Metrics(Multimap nameToDataPoints) { + this.nameToDataPoints = nameToDataPoints; + } + + public List findByNameAndLabels(String metricName, String labelName, String labelValue) { + return nameToDataPoints.get(metricName) + .stream() + .filter(metric -> metric.contains(labelName, labelValue)) + .toList(); + } + + @SafeVarargs + public final List findByNameAndLabels(String metricName, Pair... nameValuePairs) { + return nameToDataPoints.get(metricName) + .stream() + .filter(metric -> { + for (Pair nameValuePair : nameValuePairs) { + String labelName = nameValuePair.getLeft(); + String labelValue = nameValuePair.getRight(); + if (!metric.contains(labelName, labelValue)) { + return false; + } + } + return true; + }) + .toList(); + } + + @SafeVarargs + public final Metric findSingleMetricByNameAndLabels(String metricName, Pair... nameValuePairs) { + List metricByNameAndLabels = findByNameAndLabels(metricName, nameValuePairs); + if (metricByNameAndLabels.size() != 1) { + fail("Expected to find 1 metric, but found the following: "+metricByNameAndLabels + + ". Metrics are = "+nameToDataPoints.get(metricName)+". Labels requested = "+ Arrays.toString( + nameValuePairs)); + } + return metricByNameAndLabels.get(0); + } + } +} 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 0211b860cc4fe..1bc298ac601c8 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 @@ -18,28 +18,36 @@ */ package org.apache.pulsar.broker.transaction.buffer; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; import io.netty.util.HashedWheelTimer; import io.netty.util.concurrent.DefaultThreadFactory; -import lombok.Cleanup; import java.io.ByteArrayOutputStream; +import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentSkipListMap; 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 java.util.concurrent.TimeUnit; +import lombok.Cleanup; 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; @@ -70,14 +78,6 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.lang.reflect.Field; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.TimeUnit; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; @Test(groups = "broker") public class TransactionBufferClientTest extends TransactionTestBase { @@ -228,28 +228,28 @@ public void testTransactionBufferMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - Collection abortFailed = metricsMap.get("pulsar_txn_tb_client_abort_failed_total"); - Collection commitFailed = metricsMap.get("pulsar_txn_tb_client_commit_failed_total"); - Collection abortLatencyCount = + Collection abortFailed = metricsMap.get("pulsar_txn_tb_client_abort_failed_total"); + Collection commitFailed = metricsMap.get("pulsar_txn_tb_client_commit_failed_total"); + Collection abortLatencyCount = metricsMap.get("pulsar_txn_tb_client_abort_latency_count"); - Collection commitLatencyCount = + Collection commitLatencyCount = metricsMap.get("pulsar_txn_tb_client_commit_latency_count"); - Collection pending = metricsMap.get("pulsar_txn_tb_client_pending_requests"); + Collection pending = metricsMap.get("pulsar_txn_tb_client_pending_requests"); assertEquals(abortFailed.stream().mapToDouble(metric -> metric.value).sum(), 0); assertEquals(commitFailed.stream().mapToDouble(metric -> metric.value).sum(), 0); for (int i = 0; i < partitions; i++) { String topic = partitionedTopicName.getPartition(i).toString(); - Optional optional = abortLatencyCount.stream() + Optional optional = abortLatencyCount.stream() .filter(metric -> metric.tags.get("topic").equals(topic)).findFirst(); assertTrue(optional.isPresent()); assertEquals(optional.get().value, 1D); - Optional optional1 = commitLatencyCount.stream() + Optional optional1 = commitLatencyCount.stream() .filter(metric -> metric.tags.get("topic").equals(topic)).findFirst(); assertTrue(optional1.isPresent()); assertEquals(optional1.get().value, 1D); 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 f5ac68950b49d..93a2f274517d5 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 @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.transaction.pendingack; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -28,6 +30,7 @@ import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertTrue; import static org.testng.AssertJUnit.fail; +import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -40,7 +43,6 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import com.google.common.collect.Multimap; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -54,7 +56,6 @@ 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.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; @@ -67,9 +68,9 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.SystemTopicNames; @@ -338,28 +339,28 @@ public void testPendingAckMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - Collection abortedCount = metricsMap.get("pulsar_txn_tp_aborted_count_total"); - Collection committedCount = metricsMap.get("pulsar_txn_tp_committed_count_total"); - Collection commitLatency = metricsMap.get("pulsar_txn_tp_commit_latency"); + Collection abortedCount = metricsMap.get("pulsar_txn_tp_aborted_count_total"); + Collection committedCount = metricsMap.get("pulsar_txn_tp_committed_count_total"); + Collection commitLatency = metricsMap.get("pulsar_txn_tp_commit_latency"); Assert.assertTrue(commitLatency.size() > 0); int count = 0; - for (PrometheusMetricsTest.Metric metric : commitLatency) { + for (Metric metric : commitLatency) { if (metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC) && metric.value > 0) { count++; } } Assert.assertTrue(count > 0); - for (PrometheusMetricsTest.Metric metric : abortedCount) { + for (Metric metric : abortedCount) { if (metric.tags.get("subscription").equals(subName) && metric.tags.get("status").equals("succeed")) { assertTrue(metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC)); assertTrue(metric.value > 0); } } - for (PrometheusMetricsTest.Metric metric : committedCount) { + for (Metric metric : committedCount) { if (metric.tags.get("subscription").equals(subName) && metric.tags.get("status").equals("succeed")) { assertTrue(metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC)); assertTrue(metric.value > 0); 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 999f22dc7f92b..9b0f8d798145a 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 @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.web; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -56,7 +58,6 @@ 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.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; @@ -108,31 +109,31 @@ public void testWebExecutorMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection maxThreads = metrics.get("pulsar_web_executor_max_threads"); - Collection minThreads = metrics.get("pulsar_web_executor_min_threads"); - Collection activeThreads = metrics.get("pulsar_web_executor_active_threads"); - Collection idleThreads = metrics.get("pulsar_web_executor_idle_threads"); - Collection currentThreads = metrics.get("pulsar_web_executor_current_threads"); + Collection maxThreads = metrics.get("pulsar_web_executor_max_threads"); + Collection minThreads = metrics.get("pulsar_web_executor_min_threads"); + Collection activeThreads = metrics.get("pulsar_web_executor_active_threads"); + Collection idleThreads = metrics.get("pulsar_web_executor_idle_threads"); + Collection currentThreads = metrics.get("pulsar_web_executor_current_threads"); - for (PrometheusMetricsTest.Metric metric : maxThreads) { + for (Metric metric : maxThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } - for (PrometheusMetricsTest.Metric metric : minThreads) { + for (Metric metric : minThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } - for (PrometheusMetricsTest.Metric metric : activeThreads) { + for (Metric metric : activeThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value >= 0); } - for (PrometheusMetricsTest.Metric metric : idleThreads) { + for (Metric metric : idleThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value >= 0); } - for (PrometheusMetricsTest.Metric metric : currentThreads) { + for (Metric metric : currentThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicDefaultMultiPartitionsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicDefaultMultiPartitionsTest.java new file mode 100644 index 0000000000000..b8bccb793724b --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicDefaultMultiPartitionsTest.java @@ -0,0 +1,251 @@ +/* + * 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 static org.apache.pulsar.client.util.RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TopicType; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-impl") +public class DeadLetterTopicDefaultMultiPartitionsTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + this.conf.setMaxMessageSize(5 * 1024); + this.conf.setAllowAutoTopicCreation(true); + this.conf.setDefaultNumPartitions(2); + this.conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + private void triggerDLQGenerate(String topic, String subscription) throws Exception { + String DLQ = getDLQName(topic, subscription); + String p0OfDLQ = TopicName.get(DLQ).getPartition(0).toString(); + Consumer consumer = pulsarClient.newConsumer().topic(topic).subscriptionName(subscription) + .ackTimeout(1000, TimeUnit.MILLISECONDS) + .subscriptionType(SubscriptionType.Shared) + .receiverQueueSize(10) + .negativeAckRedeliveryDelay(100, TimeUnit.MILLISECONDS) + .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(1).build()) + .subscribe(); + Producer producer = pulsarClient.newProducer().topic(topic).create(); + producer.newMessage().value(new byte[]{1}).send(); + + Message message1 = consumer.receive(); + consumer.negativeAcknowledge(message1); + Message message2 = consumer.receive(); + consumer.negativeAcknowledge(message2); + + Awaitility.await().atMost(Duration.ofSeconds(1500)).until(() -> { + Message message3 = consumer.receive(2, TimeUnit.SECONDS); + if (message3 != null) { + log.info("===> {}", message3.getRedeliveryCount()); + consumer.negativeAcknowledge(message3); + } + List topicList = pulsar.getPulsarResources().getTopicResources() + .listPersistentTopicsAsync(TopicName.get(topic).getNamespaceObject()).join(); + if (topicList.contains(DLQ) || topicList.contains(p0OfDLQ)) { + return true; + } + int partitions = admin.topics().getPartitionedTopicMetadata(topic).partitions; + for (int i = 0; i < partitions; i++) { + for (int j = -1; j < pulsar.getConfig().getDefaultNumPartitions(); j++) { + String p0OfDLQ2; + if (j == -1) { + p0OfDLQ2 = TopicName + .get(getDLQName(TopicName.get(topic).getPartition(i).toString(), subscription)) + .toString(); + } else { + p0OfDLQ2 = TopicName + .get(getDLQName(TopicName.get(topic).getPartition(i).toString(), subscription)) + .getPartition(j).toString(); + } + if (topicList.contains(p0OfDLQ2)) { + return true; + } + } + } + return false; + }); + producer.close(); + consumer.close(); + admin.topics().unload(topic); + } + + private static String getDLQName(String primaryTopic, String subscription) { + String domain = TopicName.get(primaryTopic).getDomain().toString(); + return domain + "://" + TopicName.get(primaryTopic) + .toString().substring(( domain + "://").length()) + + "-" + subscription + DLQ_GROUP_TOPIC_SUFFIX; + } + + @DataProvider(name = "topicCreationTypes") + public Object[][] topicCreationTypes() { + return new Object[][]{ + //{TopicType.NON_PARTITIONED}, + {TopicType.PARTITIONED} + }; + } + + @Test(dataProvider = "topicCreationTypes") + public void testGenerateNonPartitionedDLQ(TopicType topicType) throws Exception { + final String topic = BrokerTestUtil.newUniqueName( "persistent://public/default/tp"); + final String subscription = "s1"; + switch (topicType) { + case PARTITIONED: { + admin.topics().createPartitionedTopic(topic, 2); + break; + } + case NON_PARTITIONED: { + admin.topics().createNonPartitionedTopic(topic); + } + } + + triggerDLQGenerate(topic, subscription); + + // Verify: no partitioned DLQ. + List partitionedTopics = pulsar.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources() + .listPartitionedTopicsAsync(TopicName.get(topic).getNamespaceObject(), TopicDomain.persistent).join(); + for (String tp : partitionedTopics) { + assertFalse(tp.endsWith("-DLQ")); + } + // Verify: non-partitioned DLQ exists. + List partitions = pulsar.getPulsarResources().getTopicResources() + .listPersistentTopicsAsync(TopicName.get(topic).getNamespaceObject()).join(); + List DLQCreated = new ArrayList<>(); + for (String tp : partitions) { + if (tp.endsWith("-DLQ")) { + DLQCreated.add(tp); + } + assertFalse(tp.endsWith("-partition-0-DLQ")); + } + assertTrue(!DLQCreated.isEmpty()); + + // cleanup. + switch (topicType) { + case PARTITIONED: { + admin.topics().deletePartitionedTopic(topic); + break; + } + case NON_PARTITIONED: { + admin.topics().delete(topic, false); + } + } + for (String t : DLQCreated) { + try { + admin.topics().delete(TopicName.get(t).getPartitionedTopicName(), false); + } catch (Exception ex) {} + try { + admin.topics().deletePartitionedTopic(TopicName.get(t).getPartitionedTopicName(), false); + } catch (Exception ex) {} + } + } + + @Test + public void testManuallyCreatePartitionedDLQ() throws Exception { + final String topic = BrokerTestUtil.newUniqueName( "persistent://public/default/tp"); + final String subscription = "s1"; + String DLQ = getDLQName(topic, subscription); + String p0OfDLQ = TopicName.get(DLQ).getPartition(0).toString(); + String p1OfDLQ = TopicName.get(DLQ).getPartition(1).toString(); + admin.topics().createNonPartitionedTopic(topic); + admin.topics().createPartitionedTopic(DLQ, 2); + + Awaitility.await().untilAsserted(() -> { + // Verify: partitioned DLQ exists. + List partitionedTopics = pulsar.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources() + .listPartitionedTopicsAsync(TopicName.get(topic).getNamespaceObject(), TopicDomain.persistent).join(); + assertTrue(partitionedTopics.contains(DLQ)); + assertFalse(partitionedTopics.contains(p0OfDLQ)); + // Verify: DLQ partitions exists. + List partitions = pulsar.getPulsarResources().getTopicResources() + .listPersistentTopicsAsync(TopicName.get(topic).getNamespaceObject()).join(); + assertFalse(partitions.contains(DLQ)); + assertTrue(partitions.contains(p0OfDLQ)); + assertTrue(partitions.contains(p1OfDLQ)); + }); + + // cleanup. + admin.topics().delete(topic, false); + admin.topics().deletePartitionedTopic(DLQ, false); + } + + @Test + public void testManuallyCreatePartitionedDLQ2() throws Exception { + final String topic = BrokerTestUtil.newUniqueName( "persistent://public/default/tp"); + final String subscription = "s1"; + final String p0OfTopic = TopicName.get(topic).getPartition(0).toString(); + String DLQ = getDLQName(p0OfTopic, subscription); + String p0OfDLQ = TopicName.get(DLQ).getPartition(0).toString(); + admin.topics().createPartitionedTopic(topic, 10); + try { + admin.topics().createPartitionedTopic(DLQ, 2); + } catch (Exception ex) { + // Keep multiple versions compatible. + if (ex.getMessage().contains("Partitioned Topic Name should not contain '-partition-'")){ + return; + } else { + fail("Failed to create partitioned DLQ"); + } + } + + Awaitility.await().untilAsserted(() -> { + // Verify: partitioned DLQ exists. + List partitionedTopics = pulsar.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources() + .listPartitionedTopicsAsync(TopicName.get(topic).getNamespaceObject(), TopicDomain.persistent).join(); + assertTrue(partitionedTopics.contains(DLQ)); + assertFalse(partitionedTopics.contains(p0OfDLQ)); + // Verify: DLQ partitions exists. + List partitions = pulsar.getPulsarResources().getTopicResources() + .listPersistentTopicsAsync(TopicName.get(topic).getNamespaceObject()).join(); + assertFalse(partitions.contains(DLQ)); + }); + + // cleanup. + admin.topics().deletePartitionedTopic(topic, false); + admin.topics().deletePartitionedTopic(DLQ, false); + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java index 156d67e4e58b3..ab7b4ce7111e5 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; @@ -1642,7 +1643,53 @@ void expireMessagesForAllSubscriptions(String topic, long expireTimeInSeconds) * @throws PulsarAdminException * Unexpected error */ - List> peekMessages(String topic, String subName, int numMessages) throws PulsarAdminException; + default List> peekMessages(String topic, String subName, int numMessages) + throws PulsarAdminException { + return peekMessages(topic, subName, numMessages, false, TransactionIsolationLevel.READ_COMMITTED); + } + + /** + * Peek messages from a topic subscription. + * + * @param topic + * topic name + * @param subName + * Subscription name + * @param numMessages + * Number of messages + * @param showServerMarker + * Enables the display of internal server write markers + * @param transactionIsolationLevel + * Sets the isolation level for peeking messages within transactions. + * - 'READ_COMMITTED' allows peeking only committed transactional messages. + * - 'READ_UNCOMMITTED' allows peeking all messages, + * even transactional messages which have been aborted. + * @return + * @throws NotAuthorizedException + * Don't have admin permission + * @throws NotFoundException + * Topic or subscription does not exist + * @throws PulsarAdminException + * Unexpected error + */ + List> peekMessages(String topic, String subName, int numMessages, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel) + throws PulsarAdminException; + + /** + * Peek messages from a topic subscription asynchronously. + * + * @param topic + * topic name + * @param subName + * Subscription name + * @param numMessages + * Number of messages + * @return a future that can be used to track when the messages are returned + */ + default CompletableFuture>> peekMessagesAsync(String topic, String subName, int numMessages) { + return peekMessagesAsync(topic, subName, numMessages, false, TransactionIsolationLevel.READ_COMMITTED); + } /** * Peek messages from a topic subscription asynchronously. @@ -1653,9 +1700,18 @@ void expireMessagesForAllSubscriptions(String topic, long expireTimeInSeconds) * Subscription name * @param numMessages * Number of messages + * @param showServerMarker + * Enables the display of internal server write markers + @param transactionIsolationLevel + * Sets the isolation level for peeking messages within transactions. + * - 'READ_COMMITTED' allows peeking only committed transactional messages. + * - 'READ_UNCOMMITTED' allows peeking all messages, + * even transactional messages which have been aborted. * @return a future that can be used to track when the messages are returned */ - CompletableFuture>> peekMessagesAsync(String topic, String subName, int numMessages); + CompletableFuture>> peekMessagesAsync( + String topic, String subName, int numMessages, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel); /** * Get a message by its messageId via a topic subscription. diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java index 985e42b280eb9..ac50763b7e097 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java @@ -64,6 +64,31 @@ public interface TopicStats { /** Get the publish time of the earliest message over all the backlogs. */ long getEarliestMsgPublishTimeInBacklogs(); + /** the size in bytes of the topic backlog quota. */ + long getBacklogQuotaLimitSize(); + + /** the topic backlog age quota, in seconds. */ + long getBacklogQuotaLimitTime(); + + /** + * Age of oldest unacknowledged message, as recorded in last backlog quota check interval. + *

+ * The age of the oldest unacknowledged (i.e. backlog) message, measured by the time elapsed from its published + * time, in seconds. This value is recorded every backlog quota check interval, hence it represents the value + * seen in the last check. + *

+ */ + long getOldestBacklogMessageAgeSeconds(); + + /** + * The subscription name containing oldest unacknowledged message as recorded in last backlog quota check. + *

+ * The name of the subscription containing the oldest unacknowledged message. This value is recorded every backlog + * quota check interval, hence it represents the value seen in the last check. + *

+ */ + String getOldestBacklogMessageSubscriptionName(); + /** Space used to store the offloaded messages for the topic/. */ long getOffloadedStorageSize(); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index a89da628372bd..42907ca72651f 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -56,6 +56,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; @@ -130,6 +131,8 @@ public class TopicsImpl extends BaseResource implements Topics { private static final String SCHEMA_VERSION = "X-Pulsar-Base64-schema-version-b64encoded"; private static final String ENCRYPTION_PARAM = "X-Pulsar-Base64-encryption-param"; private static final String ENCRYPTION_KEYS = "X-Pulsar-Base64-encryption-keys"; + public static final String TXN_ABORTED = "X-Pulsar-txn-aborted"; + public static final String TXN_UNCOMMITTED = "X-Pulsar-txn-uncommitted"; // CHECKSTYLE.ON: MemberName public static final String PROPERTY_SHADOW_SOURCE_KEY = "PULSAR.SHADOW_SOURCE"; @@ -845,7 +848,9 @@ public CompletableFuture expireMessagesForAllSubscriptionsAsync(String top return asyncPostRequest(path, Entity.entity("", MediaType.APPLICATION_JSON)); } - private CompletableFuture>> peekNthMessage(String topic, String subName, int messagePosition) { + private CompletableFuture>> peekNthMessage( + String topic, String subName, int messagePosition, boolean showServerMarker, + TransactionIsolationLevel transactionIsolationLevel) { TopicName tn = validateTopic(topic); String encodedSubName = Codec.encode(subName); WebTarget path = topicPath(tn, "subscription", encodedSubName, @@ -857,7 +862,8 @@ private CompletableFuture>> peekNthMessage(String topic, St @Override public void completed(Response response) { try { - future.complete(getMessagesFromHttpResponse(tn.toString(), response)); + future.complete(getMessagesFromHttpResponse(tn.toString(), response, + showServerMarker, transactionIsolationLevel)); } catch (Exception e) { future.completeExceptionally(getApiException(e)); } @@ -872,28 +878,35 @@ public void failed(Throwable throwable) { } @Override - public List> peekMessages(String topic, String subName, int numMessages) + public List> peekMessages(String topic, String subName, int numMessages, + boolean showServerMarker, + TransactionIsolationLevel transactionIsolationLevel) throws PulsarAdminException { - return sync(() -> peekMessagesAsync(topic, subName, numMessages)); + return sync(() -> peekMessagesAsync(topic, subName, numMessages, showServerMarker, transactionIsolationLevel)); } @Override - public CompletableFuture>> peekMessagesAsync(String topic, String subName, int numMessages) { + public CompletableFuture>> peekMessagesAsync( + String topic, String subName, int numMessages, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel) { checkArgument(numMessages > 0); CompletableFuture>> future = new CompletableFuture>>(); - peekMessagesAsync(topic, subName, numMessages, new ArrayList<>(), future, 1); + peekMessagesAsync(topic, subName, numMessages, new ArrayList<>(), + future, 1, showServerMarker, transactionIsolationLevel); return future; } private void peekMessagesAsync(String topic, String subName, int numMessages, - List> messages, CompletableFuture>> future, int nthMessage) { + List> messages, CompletableFuture>> future, int nthMessage, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel) { if (numMessages <= 0) { future.complete(messages); return; } // if peeking first message succeeds, we know that the topic and subscription exists - peekNthMessage(topic, subName, nthMessage).handle((r, ex) -> { + peekNthMessage(topic, subName, nthMessage, showServerMarker, transactionIsolationLevel) + .handle((r, ex) -> { if (ex != null) { // if we get a not found exception, it means that the position for the message we are trying to get // does not exist. At this point, we can return the already found messages. @@ -908,7 +921,8 @@ private void peekMessagesAsync(String topic, String subName, int numMessages, for (int i = 0; i < Math.min(r.size(), numMessages); i++) { messages.add(r.get(i)); } - peekMessagesAsync(topic, subName, numMessages - r.size(), messages, future, nthMessage + 1); + peekMessagesAsync(topic, subName, numMessages - r.size(), messages, future, + nthMessage + 1, showServerMarker, transactionIsolationLevel); return null; }); } @@ -1222,6 +1236,13 @@ private TopicName validateTopic(String topic) { } private List> getMessagesFromHttpResponse(String topic, Response response) throws Exception { + return getMessagesFromHttpResponse(topic, response, true, + TransactionIsolationLevel.READ_UNCOMMITTED); + } + + private List> getMessagesFromHttpResponse( + String topic, Response response, boolean showServerMarker, + TransactionIsolationLevel transactionIsolationLevel) throws Exception { if (response.getStatus() != Status.OK.getStatusCode()) { throw getApiException(response); @@ -1253,7 +1274,32 @@ private List> getMessagesFromHttpResponse(String topic, Response Map properties = new TreeMap<>(); MultivaluedMap headers = response.getHeaders(); - Object tmp = headers.getFirst(PUBLISH_TIME); + Object tmp = headers.getFirst(MARKER_TYPE); + if (tmp != null) { + if (!showServerMarker) { + return new ArrayList<>(); + } else { + messageMetadata.setMarkerType(Integer.parseInt(tmp.toString())); + } + } + + tmp = headers.getFirst(TXN_ABORTED); + if (tmp != null && Boolean.parseBoolean(tmp.toString())) { + properties.put(TXN_ABORTED, tmp.toString()); + if (transactionIsolationLevel == TransactionIsolationLevel.READ_COMMITTED) { + return new ArrayList<>(); + } + } + + tmp = headers.getFirst(TXN_UNCOMMITTED); + if (tmp != null && Boolean.parseBoolean(tmp.toString())) { + properties.put(TXN_UNCOMMITTED, tmp.toString()); + if (transactionIsolationLevel == TransactionIsolationLevel.READ_COMMITTED) { + return new ArrayList<>(); + } + } + + tmp = headers.getFirst(PUBLISH_TIME); if (tmp != null) { messageMetadata.setPublishTime(DateFormatter.parse(tmp.toString())); } @@ -1305,10 +1351,6 @@ private List> getMessagesFromHttpResponse(String topic, Response if (tmp != null) { messageMetadata.setPartitionKeyB64Encoded(Boolean.parseBoolean(tmp.toString())); } - tmp = headers.getFirst(MARKER_TYPE); - if (tmp != null) { - messageMetadata.setMarkerType(Integer.parseInt(tmp.toString())); - } tmp = headers.getFirst(TXNID_LEAST_BITS); if (tmp != null) { messageMetadata.setTxnidLeastBits(Long.parseLong(tmp.toString())); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java index c67ad08c83631..f437b28fe1f8c 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java @@ -474,6 +474,9 @@ CompletableFuture reconsumeLaterCumulativeAsync(Message message, *
  • MessageId.earliest : Reset the subscription on the earliest message available in the topic *
  • MessageId.latest : Reset the subscription on the latest message in the topic * + *

    + * This effectively resets the acknowledgement state of the subscription: all messages up to and + * including messageId will be marked as acknowledged and the rest unacknowledged. * *

    Note: For multi-topics consumer, if `messageId` is a {@link TopicMessageId}, the seek operation will happen * on the owner topic of the message, which is returned by {@link TopicMessageId#getOwnerTopic()}. Otherwise, you diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TransactionIsolationLevel.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TransactionIsolationLevel.java new file mode 100644 index 0000000000000..ae385b20232c7 --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TransactionIsolationLevel.java @@ -0,0 +1,31 @@ +/* + * 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 org.apache.pulsar.common.classification.InterfaceAudience; +import org.apache.pulsar.common.classification.InterfaceStability; + +@InterfaceAudience.Public +@InterfaceStability.Stable +public enum TransactionIsolationLevel { + // Consumer can only consume all transactional messages which have been committed. + READ_COMMITTED, + // Consumer can consume all messages, even transactional messages which have been aborted. + READ_UNCOMMITTED; +} diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java index a722abe19df81..8dc6f752c09ad 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java @@ -79,6 +79,7 @@ import org.apache.pulsar.client.admin.internal.OffloadProcessStatusImpl; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.api.transaction.TxnID; @@ -1717,7 +1718,8 @@ public void topics() throws Exception { verify(mockTopics).deletePartitionedTopic("persistent://myprop/clust/ns1/ds1", true); cmdTopics.run(split("peek-messages persistent://myprop/clust/ns1/ds1 -s sub1 -n 3")); - verify(mockTopics).peekMessages("persistent://myprop/clust/ns1/ds1", "sub1", 3); + verify(mockTopics).peekMessages("persistent://myprop/clust/ns1/ds1", "sub1", 3, + false, TransactionIsolationLevel.READ_COMMITTED); MessageImpl message = mock(MessageImpl.class); when(message.getData()).thenReturn(new byte[]{}); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 47d5326b002cf..d0b04198e13fc 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -64,10 +64,13 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.cli.NoSplitter; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; +import org.apache.pulsar.common.api.proto.MarkerType; +import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BacklogQuota; @@ -1226,10 +1229,23 @@ private class PeekMessages extends CliCommand { @Parameter(names = { "-n", "--count" }, description = "Number of messages (default 1)", required = false) private int numMessages = 1; + @Parameter(names = { "-ssm", "--show-server-marker" }, + description = "Enables the display of internal server write markers.", required = false) + private boolean showServerMarker = false; + + @Parameter(names = { "-til", "--transaction-isolation-level" }, + description = "Sets the isolation level for peeking messages within transactions. " + + "'READ_COMMITTED' allows peeking only committed transactional messages. " + + "'READ_UNCOMMITTED' allows peeking all messages, " + + "even transactional messages which have been aborted.", + required = false) + private TransactionIsolationLevel transactionIsolationLevel = TransactionIsolationLevel.READ_COMMITTED; + @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); - List> messages = getTopics().peekMessages(persistentTopic, subName, numMessages); + List> messages = getTopics().peekMessages(persistentTopic, subName, numMessages, + showServerMarker, transactionIsolationLevel); int position = 0; for (Message msg : messages) { MessageImpl message = (MessageImpl) msg; @@ -1251,6 +1267,10 @@ void run() throws PulsarAdminException { if (message.getDeliverAtTime() != 0) { System.out.println("Deliver at time: " + message.getDeliverAtTime()); } + MessageMetadata msgMetaData = message.getMessageBuilder(); + if (showServerMarker && msgMetaData.hasMarkerType()) { + System.out.println("Marker Type: " + MarkerType.valueOf(msgMetaData.getMarkerType())); + } if (message.getBrokerEntryMetadata() != null) { if (message.getBrokerEntryMetadata().hasBrokerTimestamp()) { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java index 9736d8b47ef71..44cfc2872ef6b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java @@ -40,6 +40,7 @@ import java.util.Comparator; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -135,6 +136,7 @@ public class NarClassLoader extends URLClassLoader { * The NAR for which this ClassLoader is responsible. */ private final File narWorkingDirectory; + private final AtomicBoolean closed = new AtomicBoolean(); private static final String TMP_DIR_PREFIX = "pulsar-nar"; @@ -292,4 +294,18 @@ protected String findLibrary(final String libname) { public String toString() { return NarClassLoader.class.getName() + "[" + narWorkingDirectory.getPath() + "]"; } + + @Override + protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + if (closed.get()) { + log.warn("Loading class {} from a closed classloader ({})", name, this); + } + return super.loadClass(name, resolve); + } + + @Override + public void close() throws IOException { + closed.set(true); + super.close(); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java index e50620fb22398..c68bfdbd5d3be 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java @@ -84,6 +84,31 @@ public class TopicStatsImpl implements TopicStats { /** Get estimated total unconsumed or backlog size in bytes. */ public long backlogSize; + /** the size in bytes of the topic backlog quota. */ + public long backlogQuotaLimitSize; + + /** the topic backlog age quota, in seconds. */ + public long backlogQuotaLimitTime; + + /** + * Age of oldest unacknowledged message, as recorded in last backlog quota check interval. + *

    + * The age of the oldest unacknowledged (i.e. backlog) message, measured by the time elapsed from its published + * time, in seconds. This value is recorded every backlog quota check interval, hence it represents the value + * seen in the last check. + *

    + */ + public long oldestBacklogMessageAgeSeconds; + + /** + * The subscription name containing oldest unacknowledged message as recorded in last backlog quota check. + *

    + * The name of the subscription containing the oldest unacknowledged message. This value is recorded every backlog + * quota check interval, hence it represents the value seen in the last check. + *

    + */ + public String oldestBacklogMessageSubscriptionName; + /** The number of times the publishing rate limit was triggered. */ public long publishRateLimitedTimes; @@ -221,6 +246,10 @@ public void reset() { this.compaction.reset(); this.ownerBroker = null; this.bucketDelayedIndexStats.clear(); + this.backlogQuotaLimitSize = 0; + this.backlogQuotaLimitTime = 0; + this.oldestBacklogMessageAgeSeconds = -1; + this.oldestBacklogMessageSubscriptionName = null; } // if the stats are added for the 1st time, we will need to make a copy of these stats and add it to the current @@ -250,6 +279,12 @@ public TopicStatsImpl add(TopicStats ts) { this.ongoingTxnCount = stats.ongoingTxnCount; this.abortedTxnCount = stats.abortedTxnCount; this.committedTxnCount = stats.committedTxnCount; + this.backlogQuotaLimitTime = stats.backlogQuotaLimitTime; + this.backlogQuotaLimitSize = stats.backlogQuotaLimitSize; + if (stats.oldestBacklogMessageAgeSeconds > this.oldestBacklogMessageAgeSeconds) { + this.oldestBacklogMessageAgeSeconds = stats.oldestBacklogMessageAgeSeconds; + this.oldestBacklogMessageSubscriptionName = stats.oldestBacklogMessageSubscriptionName; + } stats.bucketDelayedIndexStats.forEach((k, v) -> { TopicMetricBean topicMetricBean = diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index 3982900041813..faa5fbcd30130 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -1675,6 +1675,7 @@ public static ByteBuf addBrokerEntryMetadata(ByteBuf headerAndPayload, // | 2 bytes | 4 bytes | BROKER_ENTRY_METADATA_SIZE bytes | BrokerEntryMetadata brokerEntryMetadata = BROKER_ENTRY_METADATA.get(); + brokerEntryMetadata.clear(); for (BrokerEntryMetadataInterceptor interceptor : brokerInterceptors) { interceptor.intercept(brokerEntryMetadata); if (numberOfMessages >= 0) { diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java index f0d279ef25050..4516cfea01f05 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.client; +import static com.google.common.base.Preconditions.checkArgument; import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.Arrays; @@ -89,6 +90,7 @@ public static Collection getMockEnsemble() { } final Queue addEntryDelaysMillis = new ConcurrentLinkedQueue<>(); + final Queue addEntryResponseDelaysMillis = new ConcurrentLinkedQueue<>(); final List> failures = new ArrayList<>(); final List> addEntryFailures = new ArrayList<>(); @@ -367,6 +369,11 @@ public synchronized void addEntryDelay(long delay, TimeUnit unit) { addEntryDelaysMillis.add(unit.toMillis(delay)); } + public synchronized void addEntryResponseDelay(long delay, TimeUnit unit) { + checkArgument(delay >= 0, "The delay time must not be negative."); + addEntryResponseDelaysMillis.add(unit.toMillis(delay)); + } + static int getExceptionCode(Throwable t) { if (t instanceof BKException) { return ((BKException) t).getCode(); diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java index dea33a0e67662..aa61e541d0d6b 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java @@ -197,6 +197,13 @@ public void asyncAddEntry(final ByteBuf data, final AddCallback cb, final Object cb.addComplete(PulsarMockBookKeeper.getExceptionCode(exception), PulsarMockLedgerHandle.this, LedgerHandle.INVALID_ENTRY_ID, ctx); } else { + Long responseDelayMillis = bk.addEntryResponseDelaysMillis.poll(); + if (responseDelayMillis != null) { + try { + Thread.sleep(responseDelayMillis); + } catch (InterruptedException e) { + } + } cb.addComplete(BKException.Code.OK, PulsarMockLedgerHandle.this, entryId, ctx); } }, bk.executor); diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java index 6cb60e14984f9..6ebbe5bce582a 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java @@ -107,9 +107,7 @@ private boolean refillBufferIfNeeded() throws IOException { bufferOffsetEnd = endRange; long bytesRead = endRange - startRange + 1; int bytesToCopy = (int) bytesRead; - while (bytesToCopy > 0) { - bytesToCopy -= buffer.writeBytes(stream, bytesToCopy); - } + fillBuffer(stream, bytesToCopy); cursor += buffer.readableBytes(); } @@ -135,6 +133,20 @@ private boolean refillBufferIfNeeded() throws IOException { return true; } + void fillBuffer(InputStream is, int bytesToCopy) throws IOException { + while (bytesToCopy > 0) { + int writeBytes = buffer.writeBytes(is, bytesToCopy); + if (writeBytes < 0) { + break; + } + bytesToCopy -= writeBytes; + } + } + + ByteBuf getBuffer() { + return buffer; + } + @Override public int read() throws IOException { if (refillBufferIfNeeded()) { diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamTest.java new file mode 100644 index 0000000000000..951180e4e18c8 --- /dev/null +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamTest.java @@ -0,0 +1,51 @@ +/* + * 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.bookkeeper.mledger.offload.jcloud.impl; + +import static org.testng.Assert.assertEquals; + +import java.io.IOException; +import java.io.InputStream; +import org.apache.bookkeeper.mledger.offload.jcloud.BlobStoreTestBase; +import org.testng.annotations.Test; + +public class BlobStoreBackedInputStreamTest extends BlobStoreTestBase { + + @Test + public void testFillBuffer() throws Exception { + BlobStoreBackedInputStreamImpl bis = new BlobStoreBackedInputStreamImpl( + blobStore, BUCKET, "testFillBuffer", (k, md) -> { + }, 2048, 512); + + InputStream is = new InputStream() { + int count = 10; + + @Override + public int read() throws IOException { + if (count-- > 0) { + return 1; + } else { + return -1; + } + } + }; + bis.fillBuffer(is, 20); + assertEquals(bis.getBuffer().readableBytes(), 10); + } +}