-
Notifications
You must be signed in to change notification settings - Fork 962
Fix Memory Leak In Netty Recycler of Bookie Client #4609
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
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,130 @@ | ||
| package org.apache.bookkeeper.proto; | ||
|
|
||
| import io.netty.util.Recycler; | ||
| import org.apache.bookkeeper.client.BKException; | ||
| import org.apache.bookkeeper.net.BookieId; | ||
| import org.slf4j.MDC; | ||
|
|
||
| class AddCompletion extends CompletionValue implements BookkeeperInternalCallbacks.WriteCallback { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Refactoring by pulling the classes out masks the actual fix here. Can you, for this fix, leave the refactoring aside?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Actually, after reviewing a bit, it kind of makes sense to split the classes in this PR, since it forces all classes to be static. |
||
|
|
||
| static AddCompletion acquireAddCompletion(final CompletionKey key, | ||
| final BookkeeperInternalCallbacks.WriteCallback originalCallback, | ||
| final Object originalCtx, | ||
| final long ledgerId, final long entryId, | ||
| PerChannelBookieClient perChannelBookieClient) { | ||
| AddCompletion completion = addCompletionRecycler.get(); | ||
| completion.reset(key, originalCallback, originalCtx, ledgerId, entryId, perChannelBookieClient); | ||
| return completion; | ||
| } | ||
|
|
||
| final Recycler.Handle<AddCompletion> handle; | ||
|
|
||
| CompletionKey key = null; | ||
| BookkeeperInternalCallbacks.WriteCallback originalCallback = null; | ||
|
|
||
| AddCompletion(Recycler.Handle<AddCompletion> handle) { | ||
| super("Add", null, -1, -1, null); | ||
| this.handle = handle; | ||
| } | ||
|
|
||
| void reset(final CompletionKey key, | ||
| final BookkeeperInternalCallbacks.WriteCallback originalCallback, | ||
| final Object originalCtx, | ||
| final long ledgerId, final long entryId, | ||
| PerChannelBookieClient perChannelBookieClient) { | ||
| this.key = key; | ||
| this.originalCallback = originalCallback; | ||
| this.ctx = originalCtx; | ||
| this.ledgerId = ledgerId; | ||
| this.entryId = entryId; | ||
| this.startTime = org.apache.bookkeeper.common.util.MathUtils.nowInNano(); | ||
|
|
||
| this.opLogger = perChannelBookieClient.addEntryOpLogger; | ||
| this.timeoutOpLogger = perChannelBookieClient.addTimeoutOpLogger; | ||
| this.perChannelBookieClient = perChannelBookieClient; | ||
| this.mdcContextMap = perChannelBookieClient.preserveMdcForTaskExecution ? MDC.getCopyOfContextMap() : null; | ||
| } | ||
|
|
||
| @Override | ||
| public void release() { | ||
| this.ctx = null; | ||
| this.opLogger = null; | ||
| this.timeoutOpLogger = null; | ||
| this.perChannelBookieClient = null; | ||
| this.mdcContextMap = null; | ||
| handle.recycle(this); | ||
| } | ||
|
|
||
| @Override | ||
| public void writeComplete(int rc, long ledgerId, long entryId, | ||
| BookieId addr, | ||
| Object ctx) { | ||
| logOpResult(rc); | ||
| originalCallback.writeComplete(rc, ledgerId, entryId, addr, ctx); | ||
| key.release(); | ||
| this.release(); | ||
| } | ||
|
|
||
| @Override | ||
| boolean maybeTimeout() { | ||
| if (org.apache.bookkeeper.common.util.MathUtils.elapsedNanos(startTime) >= | ||
| perChannelBookieClient.addEntryTimeoutNanos) { | ||
| timeout(); | ||
| return true; | ||
| } else { | ||
| return false; | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void errorOut() { | ||
| errorOut(BKException.Code.BookieHandleNotAvailableException); | ||
| } | ||
|
|
||
| @Override | ||
| public void errorOut(final int rc) { | ||
| errorOutAndRunCallback( | ||
| () -> writeComplete(rc, ledgerId, entryId, perChannelBookieClient.bookieId, ctx)); | ||
| } | ||
|
|
||
| @Override | ||
| public void setOutstanding() { | ||
| perChannelBookieClient.addEntryOutstanding.inc(); | ||
| } | ||
|
|
||
| @Override | ||
| public void handleV2Response( | ||
| long ledgerId, long entryId, BookkeeperProtocol.StatusCode status, | ||
| BookieProtocol.Response response) { | ||
| perChannelBookieClient.addEntryOutstanding.dec(); | ||
| handleResponse(ledgerId, entryId, status); | ||
| } | ||
|
|
||
| @Override | ||
| public void handleV3Response( | ||
| BookkeeperProtocol.Response response) { | ||
| perChannelBookieClient.addEntryOutstanding.dec(); | ||
| BookkeeperProtocol.AddResponse addResponse = response.getAddResponse(); | ||
| BookkeeperProtocol.StatusCode status = response.getStatus() == BookkeeperProtocol.StatusCode.EOK | ||
| ? addResponse.getStatus() : response.getStatus(); | ||
| handleResponse(addResponse.getLedgerId(), addResponse.getEntryId(), | ||
| status); | ||
| } | ||
|
|
||
| private void handleResponse(long ledgerId, long entryId, | ||
| BookkeeperProtocol.StatusCode status) { | ||
| if (LOG.isDebugEnabled()) { | ||
| logResponse(status, "ledger", ledgerId, "entry", entryId); | ||
| } | ||
|
|
||
| int rc = convertStatus(status, BKException.Code.WriteException); | ||
| writeComplete(rc, ledgerId, entryId, perChannelBookieClient.bookieId, ctx); | ||
| } | ||
|
|
||
| private static final Recycler<AddCompletion> addCompletionRecycler = new Recycler<AddCompletion>() { | ||
merlimat marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| @Override | ||
| protected AddCompletion newObject(Recycler.Handle<AddCompletion> handle) { | ||
| return new AddCompletion(handle); | ||
| } | ||
| }; | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,85 @@ | ||
| package org.apache.bookkeeper.proto; | ||
|
|
||
| import org.apache.bookkeeper.client.BKException; | ||
| import org.apache.bookkeeper.util.ByteBufList; | ||
|
|
||
| import static org.apache.bookkeeper.client.LedgerHandle.INVALID_ENTRY_ID; | ||
|
|
||
| class BatchedReadCompletion extends CompletionValue { | ||
|
|
||
| final BookkeeperInternalCallbacks.BatchedReadEntryCallback cb; | ||
|
|
||
| public BatchedReadCompletion(final CompletionKey key, | ||
| final BookkeeperInternalCallbacks.BatchedReadEntryCallback originalCallback, | ||
| final Object originalCtx, | ||
| long ledgerId, final long entryId, | ||
| PerChannelBookieClient perChannelBookieClient) { | ||
| super("BatchedRead", originalCtx, ledgerId, entryId, perChannelBookieClient); | ||
| this.opLogger = perChannelBookieClient.readEntryOpLogger; | ||
| this.timeoutOpLogger = perChannelBookieClient.readTimeoutOpLogger; | ||
| this.cb = new BookkeeperInternalCallbacks.BatchedReadEntryCallback() { | ||
|
|
||
| @Override | ||
| public void readEntriesComplete(int rc, | ||
| long ledgerId, | ||
| long startEntryId, | ||
| ByteBufList bufList, | ||
| Object ctx) { | ||
| logOpResult(rc); | ||
| originalCallback.readEntriesComplete(rc, | ||
| ledgerId, entryId, | ||
| bufList, originalCtx); | ||
| key.release(); | ||
| } | ||
| }; | ||
| } | ||
|
|
||
| @Override | ||
| public void errorOut() { | ||
| errorOut(BKException.Code.BookieHandleNotAvailableException); | ||
| } | ||
|
|
||
| @Override | ||
| public void errorOut(final int rc) { | ||
| errorOutAndRunCallback( | ||
| () -> cb.readEntriesComplete(rc, ledgerId, | ||
| entryId, null, ctx)); | ||
| } | ||
|
|
||
| @Override | ||
| public void handleV2Response(long ledgerId, | ||
| long entryId, | ||
| BookkeeperProtocol.StatusCode status, | ||
| BookieProtocol.Response response) { | ||
|
|
||
| perChannelBookieClient.readEntryOutstanding.dec(); | ||
| if (!(response instanceof BookieProtocol.BatchedReadResponse)) { | ||
| return; | ||
| } | ||
| BookieProtocol.BatchedReadResponse readResponse = (BookieProtocol.BatchedReadResponse) response; | ||
| handleBatchedReadResponse(ledgerId, entryId, status, readResponse.getData(), | ||
| INVALID_ENTRY_ID, -1L); | ||
| } | ||
|
|
||
| @Override | ||
| public void handleV3Response(BookkeeperProtocol.Response response) { | ||
| // V3 protocol haven't supported batched read yet. | ||
| } | ||
|
|
||
| private void handleBatchedReadResponse(long ledgerId, | ||
| long entryId, | ||
| BookkeeperProtocol.StatusCode status, | ||
| ByteBufList buffers, | ||
| long maxLAC, // max known lac piggy-back from bookies | ||
| long lacUpdateTimestamp) { // the timestamp when the lac is updated. | ||
| int rc = convertStatus(status, BKException.Code.ReadException); | ||
|
|
||
| if (maxLAC > INVALID_ENTRY_ID && (ctx instanceof BookkeeperInternalCallbacks.ReadEntryCallbackCtx)) { | ||
| ((BookkeeperInternalCallbacks.ReadEntryCallbackCtx) ctx).setLastAddConfirmed(maxLAC); | ||
| } | ||
| if (lacUpdateTimestamp > -1L && (ctx instanceof ReadLastConfirmedAndEntryContext)) { | ||
| ((ReadLastConfirmedAndEntryContext) ctx).setLacUpdateTimestamp(lacUpdateTimestamp); | ||
| } | ||
| cb.readEntriesComplete(rc, ledgerId, entryId, buffers, ctx); | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,13 @@ | ||
| package org.apache.bookkeeper.proto; | ||
|
|
||
| import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; | ||
|
|
||
| abstract class CompletionKey { | ||
| OperationType operationType; | ||
|
|
||
| CompletionKey(OperationType operationType) { | ||
| this.operationType = operationType; | ||
| } | ||
|
|
||
| public void release() {} | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,159 @@ | ||
| package org.apache.bookkeeper.proto; | ||
|
|
||
| import com.google.common.base.Joiner; | ||
| import io.netty.channel.Channel; | ||
| import org.apache.bookkeeper.client.BKException; | ||
| import org.apache.bookkeeper.common.util.MathUtils; | ||
| import org.apache.bookkeeper.common.util.MdcUtils; | ||
| import org.apache.bookkeeper.stats.OpStatsLogger; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
| import org.slf4j.MDC; | ||
|
|
||
| import java.util.Map; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| abstract class CompletionValue { | ||
| private final String operationName; | ||
| protected Object ctx; | ||
| protected long ledgerId; | ||
| protected long entryId; | ||
| protected long startTime; | ||
| protected OpStatsLogger opLogger; | ||
| protected OpStatsLogger timeoutOpLogger; | ||
| protected Map<String, String> mdcContextMap; | ||
| protected PerChannelBookieClient perChannelBookieClient; | ||
|
|
||
| static final Logger LOG = LoggerFactory.getLogger(CompletionValue.class); | ||
|
|
||
| public CompletionValue(String operationName, | ||
| Object ctx, | ||
| long ledgerId, long entryId, PerChannelBookieClient perChannelBookieClient) { | ||
| this.operationName = operationName; | ||
| this.ctx = ctx; | ||
| this.ledgerId = ledgerId; | ||
| this.entryId = entryId; | ||
| this.startTime = MathUtils.nowInNano(); | ||
| this.perChannelBookieClient = perChannelBookieClient; | ||
| if (perChannelBookieClient != null) { | ||
| this.mdcContextMap = perChannelBookieClient.preserveMdcForTaskExecution ? MDC.getCopyOfContextMap() : null; | ||
| } | ||
| } | ||
|
|
||
| private long latency() { | ||
| return MathUtils.elapsedNanos(startTime); | ||
| } | ||
|
|
||
| void logOpResult(int rc) { | ||
| if (rc != BKException.Code.OK) { | ||
| opLogger.registerFailedEvent(latency(), TimeUnit.NANOSECONDS); | ||
| } else { | ||
| opLogger.registerSuccessfulEvent(latency(), TimeUnit.NANOSECONDS); | ||
| } | ||
|
|
||
| if (rc != BKException.Code.OK | ||
| && !PerChannelBookieClient.expectedBkOperationErrors.contains(rc)) { | ||
| perChannelBookieClient.recordError(); | ||
| } | ||
| } | ||
|
|
||
| boolean maybeTimeout() { | ||
| if (MathUtils.elapsedNanos(startTime) >= perChannelBookieClient.readEntryTimeoutNanos) { | ||
| timeout(); | ||
| return true; | ||
| } else { | ||
| return false; | ||
| } | ||
| } | ||
|
|
||
| void timeout() { | ||
| errorOut(BKException.Code.TimeoutException); | ||
| timeoutOpLogger.registerSuccessfulEvent(latency(), | ||
| TimeUnit.NANOSECONDS); | ||
| } | ||
|
|
||
| protected void logResponse(BookkeeperProtocol.StatusCode status, Object... extraInfo) { | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("Got {} response from bookie:{} rc:{}, {}", operationName, | ||
| perChannelBookieClient.bookieId, status, Joiner.on(":").join(extraInfo)); | ||
| } | ||
| } | ||
|
|
||
| protected int convertStatus(BookkeeperProtocol.StatusCode status, int defaultStatus) { | ||
| // convert to BKException code | ||
| int rcToRet = statusCodeToExceptionCode(status); | ||
| if (rcToRet == BKException.Code.UNINITIALIZED) { | ||
| LOG.error("{} for failed on bookie {} code {}", | ||
| operationName, perChannelBookieClient.bookieId, status); | ||
| return defaultStatus; | ||
| } else { | ||
| return rcToRet; | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * @param status | ||
| * @return {@link BKException.Code.UNINITIALIZED} if the statuscode is unknown. | ||
| */ | ||
| private int statusCodeToExceptionCode(BookkeeperProtocol.StatusCode status) { | ||
| switch (status) { | ||
| case EOK: | ||
| return BKException.Code.OK; | ||
| case ENOENTRY: | ||
| return BKException.Code.NoSuchEntryException; | ||
| case ENOLEDGER: | ||
| return BKException.Code.NoSuchLedgerExistsException; | ||
| case EBADVERSION: | ||
| return BKException.Code.ProtocolVersionException; | ||
| case EUA: | ||
| return BKException.Code.UnauthorizedAccessException; | ||
| case EFENCED: | ||
| return BKException.Code.LedgerFencedException; | ||
| case EREADONLY: | ||
| return BKException.Code.WriteOnReadOnlyBookieException; | ||
| case ETOOMANYREQUESTS: | ||
| return BKException.Code.TooManyRequestsException; | ||
| case EUNKNOWNLEDGERSTATE: | ||
| return BKException.Code.DataUnknownException; | ||
| default: | ||
| return BKException.Code.UNINITIALIZED; | ||
| } | ||
| } | ||
|
|
||
| public void restoreMdcContext() { | ||
| MdcUtils.restoreContext(mdcContextMap); | ||
| } | ||
|
|
||
| public abstract void errorOut(); | ||
| public abstract void errorOut(int rc); | ||
| public void setOutstanding() { | ||
| // no-op | ||
| } | ||
|
|
||
| protected void errorOutAndRunCallback(final Runnable callback) { | ||
| perChannelBookieClient.executor.executeOrdered(ledgerId, () -> { | ||
| String bAddress = "null"; | ||
| Channel c = perChannelBookieClient.channel; | ||
| if (c != null && c.remoteAddress() != null) { | ||
| bAddress = c.remoteAddress().toString(); | ||
| } | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("Could not write {} request to bookie {} for ledger {}, entry {}", | ||
| operationName, bAddress, | ||
| ledgerId, entryId); | ||
| } | ||
| callback.run(); | ||
| }); | ||
| } | ||
|
|
||
| public void handleV2Response( | ||
| long ledgerId, long entryId, BookkeeperProtocol.StatusCode status, | ||
| BookieProtocol.Response response) { | ||
| LOG.warn("Unhandled V2 response {}", response); | ||
| } | ||
|
|
||
| public abstract void handleV3Response( | ||
| BookkeeperProtocol.Response response); | ||
|
|
||
| public void release() {} | ||
| } |
Uh oh!
There was an error while loading. Please reload this page.