Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix ledgerHandle pendingAddEntries stats #4104

Open
wants to merge 3 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,8 @@ public interface BookKeeperClientStats {
String READ_REQUESTS_REORDERED = "READ_REQUESTS_REORDERED";
String GET_LIST_OF_ENTRIES_OF_LEDGER_OP = "GET_LIST_OF_ENTRIES_OF_LEDGER";

String NUMBER_OPEN_LEDGER_HANDLE = "NUM_OPEN_LEDGER_HANDLE";

// per channel stats
String CHANNEL_SCOPE = "per_channel_bookie_client";

Expand Down Expand Up @@ -130,6 +132,7 @@ public interface BookKeeperClientStats {
Counter getWriteDelayedDueToNotEnoughFaultDomains();
Counter getWriteTimedOutDueToNotEnoughFaultDomains();
void registerPendingAddsGauge(Gauge<Integer> gauge);
void registerOpenLedgerHandleGauge(Gauge<Integer> gauge);

static BookKeeperClientStats newInstance(StatsLogger stats) {
return new BookKeeperClientStatsImpl(stats);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import org.apache.bookkeeper.meta.LedgerIdGenerator;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
import org.apache.bookkeeper.stats.Gauge;
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.util.MathUtils;
import org.apache.bookkeeper.versioning.Versioned;
Expand Down Expand Up @@ -259,6 +260,35 @@ private void metadataCallback(Versioned<LedgerMetadata> writtenMetadata,
clientStats.getEnsembleBookieDistributionCounter(bsa.toString()).inc();
}

LedgerHandle.INSTANCES.add(lh);

clientStats.registerPendingAddsGauge(new Gauge<Integer>() {
@Override
public Integer getDefaultValue() {
return 0;
}

@Override
public Integer getSample() {
return LedgerHandle.INSTANCES.
stream().
mapToInt(ledgerHandle -> ledgerHandle.pendingAddOps.size()).
sum();
}
});

clientStats.registerOpenLedgerHandleGauge(new Gauge<Integer>() {
@Override
public Integer getDefaultValue() {
return 0;
}

@Override
public Integer getSample() {
return LedgerHandle.INSTANCES.size();
}
});

// return the ledger handle back
createComplete(BKException.Code.OK, lh);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.RejectedExecutionException;
Expand Down Expand Up @@ -80,7 +81,6 @@
import org.apache.bookkeeper.proto.BookieProtocol;
import org.apache.bookkeeper.proto.checksum.DigestManager;
import org.apache.bookkeeper.stats.Counter;
import org.apache.bookkeeper.stats.Gauge;
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.versioning.Versioned;
import org.apache.commons.collections4.IteratorUtils;
Expand All @@ -92,6 +92,8 @@
* write operations to a ledger.
*/
public class LedgerHandle implements WriteHandle {
public static Set<LedgerHandle> INSTANCES = ConcurrentHashMap.newKeySet();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Store the LedgerHandle object will bring extra memory overload for the client.


static final Logger LOG = LoggerFactory.getLogger(LedgerHandle.class);

private static final int STICKY_READ_BOOKIE_INDEX_UNSET = -1;
Expand Down Expand Up @@ -247,17 +249,6 @@ public long getBookiePendingRequests(BookieId bookieSocketAddress) {
lacUpdateMissesCounter = clientCtx.getClientStats().getLacUpdateMissesCounter();
clientChannelWriteWaitStats = clientCtx.getClientStats().getClientChannelWriteWaitLogger();

clientCtx.getClientStats().registerPendingAddsGauge(new Gauge<Integer>() {
@Override
public Integer getDefaultValue() {
return 0;
}
@Override
public Integer getSample() {
return pendingAddOps.size();
}
});

initializeWriteHandleState();
}

Expand Down Expand Up @@ -521,6 +512,8 @@ void asyncCloseInternal(final CloseCallback cb, final Object ctx, final int rc)
errorOutPendingAdds(BookKeeper.getReturnRc(clientCtx.getBookieClient(), rc));
cb.closeComplete(BookKeeper.getReturnRc(clientCtx.getBookieClient(), BKException.Code.InterruptedException),
this, ctx);
} finally {
LedgerHandle.INSTANCES.remove(this);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -292,4 +292,9 @@ public Counter getWriteTimedOutDueToNotEnoughFaultDomains() {
public void registerPendingAddsGauge(Gauge<Integer> gauge) {
stats.registerGauge(PENDING_ADDS, gauge);
}

@Override
public void registerOpenLedgerHandleGauge(Gauge<Integer> gauge) {
stats.registerGauge(NUMBER_OPEN_LEDGER_HANDLE, gauge);
}
}
Loading