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 extends Position> positions)
- throws InterruptedException, ManagedLedgerException {
+ public List replayEntries(Set extends Position> positions) {
return null;
}
@@ -300,8 +297,7 @@ public Set extends Position> asyncReplayEntries(Set extends Position> 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 extends Subscription> 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