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 dead lock issue in ReadOnlyLedgerHandle#handleBookieFailure #4467

Open
wants to merge 1 commit into
base: master
Choose a base branch
from

Conversation

TakaHiR07
Copy link
Contributor

@TakaHiR07 TakaHiR07 commented Jul 22, 2024

Fix #4457, apache/pulsar#22986

Motivation

There is a dead lock issue in pulsar-3.0, which is easily to reproduce when some bookies become unavailable. Then it would cause broker unavailable and topic unavailable

The issue occurred when openLedger and do recoverAdd in ReadOnlyLedgerHandle.

  • this issue is not exist in branch-4.14, but exist after branch-4.16
  • this issue only exist in ReadOnlyLedgerHandle, but not exist in LedgerHandle. Because LedgerHandle#handleBookieFailure's implementation is different from ReadOnlyLedgerHandle. It would not lock metadataLock and then lock other pendingAddOp of the same ledger.

image

We have analysed the stack in issue. The deadlock process can be simplify as:

  1. In thread pulsar-io-2, request2 lock pendingAddOp2 in writeComplete(), because of channel disconnect in ChannelReadyForAddEntryCallback#operationComplete
  2. In thread pulsar-io-1, request1 lock pendingAddOp1 in writeComplete(), because of channel disconnect in ChannelReadyForAddEntryCallback#operationComplete. And then lock metadataLock in ReadOnlyLedgerHandle#handleBookieFailure.
  3. In thread pulsar-io-2, request2 try to apply metadataLock
  4. In thread pulsar-io-1, request1 try to apply pendingAddOp2 because of unsetSuccessAndSendWriteRequest

Therefore, the deadlock occur. And I guess it occur after this pr #3784, because the pr add synchronized in pendingAddOp.

As our known for bookkeeper, handleBookieFailure and the other operations of entry should always run in specific thread for ledgerId. However, we saw that the two pendingAddOps of the same ledger is run in different "pulsar-io" thread. This is not reasonable.

After diving into code, we can find that "pulsar-io" threadPool is used for PerChannelBookieClient connect, and "BookKeeperClientWorker" threadPool is used for handling write or read entry. But if connect fail and throw exception, bookieClient.completeAdd still use "pulsar-io" to execute. That is the root reason of the dead lock issue.
image
image

If we can make sure all the pendingAddOps of the same ledger run in the specific executor, the dead lock will not occur.

Changes

If channel is not ready for addEntry, execute completeAddOperation in specific orderedExecutor of ledger, instead of in eventLoopGroup.

@TakaHiR07
Copy link
Contributor Author

since this is a fatal issue. Could you take a look of this analyse and fix? @shoothzj @dlg99 @merlimat @hangc0276

Copy link
Contributor

@dlg99 dlg99 left a comment

Choose a reason for hiding this comment

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

First of all, this is a great catch and definitely looks like a regression.

There are couple of things that require additional attention:

First, the fix seems to be on the the right track but I think it should be done in BookieClientImpl completeAdd(..) .

Second, the fix reverts part of the change #3784 (and follow up fix #3806 )
I'd love to have @merlimat look at this issue - is there a better fix or should we simply revert these perf improvements (that don't have any perf numbers shared on what they improve) for the sake of correctness?

Third, it would be nice to have a test reproing the problem to prevent future regression.

@dlg99 dlg99 requested a review from merlimat July 22, 2024 16:30
@horizonzy
Copy link
Member

From #4457 stack info.

"pulsar-io-32-13":
        at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:181)
        - waiting to lock <0x00000403f63e7520> (a org.apache.bookkeeper.client.PendingAddOp)
        at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
        at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
        - locked <0x0000044fc7914058> (a java.lang.Object)
        at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
        - locked <0x00000401526088f8> (a org.apache.bookkeeper.client.PendingAddOp)
        at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
        at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2581)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2486)
        at io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:583)
        at io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:559)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:492)
        at io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:636)
        at io.netty.util.concurrent.DefaultPromise.setFailure0(DefaultPromise.java:629)
        at io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:118)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:675)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:694)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
        at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:499)
        at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
        at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run([email protected]/Thread.java:833)


"pulsar-io-32-14":
        at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:216)
        - waiting to lock <0x0000044fc7914058> (a java.lang.Object)
        at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
        - locked <0x00000403f63e7520> (a org.apache.bookkeeper.client.PendingAddOp)
        at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
        at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
        at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2581)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2486)
        at io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:590)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:583)
        at io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:559)
        at io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:492)
        at io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:636)
        at io.netty.util.concurrent.DefaultPromise.setFailure0(DefaultPromise.java:629)
        at io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:118)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:675)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:694)
        at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
        at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:499)
        at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
        at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run([email protected]/Thread.java:833)

thread pulsar-io-32-13:
Timepoint 1:PendingAddOp(1) writeComplete method lock, object level. (0x00000401526088f8)
Timepoint 3: PendingAddOp(1) handleBookieFailure object(ReadOblyLedgerHandle#metadata) lock, object level. (0x0000044fc7914058)
Timepoint 4: PendingAddOp(2) unsetSuccessAndSendWriteRequest method lock, object level (0x00000403f63e7520)

thread pulsar-io-32-14:
Timepoint 2: PendingAddOp(2) writeComplete method lock, object level. (0x00000403f63e7520).
Timepoint 5: PendingAddOp(2) handleBookieFailure object(ReadOblyLedgerHandle#metadata) lock, object level. (0x0000044fc7914058)

I think we should fix the Timepoint4 behavior, it will handle all the pendindAddOps, and try to send the data to new replaced bookie. I think this behavior shouldn't use the same thread as before.

@TakaHiR07 TakaHiR07 force-pushed the fix_dead_lock_in_handleBookieFailure branch from 3639c50 to bbccd3f Compare July 23, 2024 04:02
@TakaHiR07
Copy link
Contributor Author

First of all, this is a great catch and definitely looks like a regression.

There are couple of things that require additional attention:

First, the fix seems to be on the the right track but I think it should be done in BookieClientImpl completeAdd(..) .

Second, the fix reverts part of the change #3784 (and follow up fix #3806 ) I'd love to have @merlimat look at this issue - is there a better fix or should we simply revert these perf improvements (that don't have any perf numbers shared on what they improve) for the sake of correctness?

Third, it would be nice to have a test reproing the problem to prevent future regression.

  1. That's right. we should do in BookieClientImpl completeAdd(..), otherwise it would cause serious issue. I have updated this pr.
  2. I also think about whether we should revert the whole change of Made PendingAddOp thread safe #3784. There is a concern of not revert. If it exist other place not obey the specific thread rule, maybe have other dead lock issue. If we revert the whole change, maybe affect some other pr's change after branch-4.16.0
  3. Now I prefer to test this pr in our test cluster. If more views agree with this pr's modification, I would try to add unittest to repoving the problem.

@TakaHiR07
Copy link
Contributor Author

I think we should fix the Timepoint4 behavior, it will handle all the pendindAddOps, and try to send the data to new replaced bookie. I think this behavior shouldn't use the same thread as before.

@horizonzy I think your solution is try to fix this dead lock issue itself, but I am still concerned that whether #3784 would bring out other dead lock risk, because it add so many synchronized in PendingAddOp.

Actually, I hold 3 kinds of solution idea.

  1. break the current dead lock, which is similar to your solution.
  2. revert part of Made PendingAddOp thread safe #3784, as this pr modify.
  3. revert whole of Made PendingAddOp thread safe #3784.

Besides, I think your solution is able to fix current dead lock, but maybe result in other problem. As I observe in both early version bookkeeper and pulsar. In the same ledger, PendingAddOp#sendWriteRequest is always in same thread. If we use different "pulsar-io" thread to send data of the same ledger, It seems break the bookkeeper's design.

https://github.com/apache/pulsar/blob/fca9c5c392cb72fa15f8d9211e39e1c55afd3281/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L778-L811

https://github.com/apache/pulsar/blob/fca9c5c392cb72fa15f8d9211e39e1c55afd3281/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L357

@chenhongSZ
Copy link
Contributor

chenhongSZ commented Sep 7, 2024

We also faced this bug (version:4.16.5), where two BookKeeperClientWorker-OrderedExecutor threads are deadlocked. I guess Moving the unsetSuccessAndSendWriteRequestoutside out of the metadata lock scope is better. because the metadataLock is only used to protect the newEnsemblesFromRecovery map. (I've fixed it in our production env, it's working well)

image

just like org.apache.bookkeeper.client.LedgerHandle#ensembleChangeLoop

image

the stack snippet is below:

	Found one Java-level deadlock:
=============================
"BookKeeperClientWorker-OrderedExecutor-0-0":
  waiting to lock monitor 0x00007f77ac006160 (object 0x00000004080b9600, a org.apache.bookkeeper.client.PendingAddOp),
  which is held by "BookKeeperClientWorker-OrderedExecutor-8-0"

"BookKeeperClientWorker-OrderedExecutor-8-0":
  waiting to lock monitor 0x00007f778c001230 (object 0x00000004080bf600, a java.lang.Object),
  which is held by "BookKeeperClientWorker-OrderedExecutor-0-0"

Java stack information for the threads listed above:
===================================================
"BookKeeperClientWorker-OrderedExecutor-0-0":
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:264)
	- waiting to lock <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9858> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf600> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b99d0> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.initiate(PendingAddOp.java:258)
	- locked <0x00000004080b99d0> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.doAsyncAddEntry(LedgerHandle.java:1358)
	at org.apache.bookkeeper.client.LedgerHandle.asyncRecoveryAddEntry(LedgerHandle.java:1212)
	at org.apache.bookkeeper.client.LedgerRecoveryOp.onEntryComplete(LedgerRecoveryOp.java:204)
	at org.apache.bookkeeper.client.ListenerBasedPendingReadOp.submitCallback(ListenerBasedPendingReadOp.java:67)
	at org.apache.bookkeeper.client.PendingReadOp.readEntryComplete(PendingReadOp.java:608)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion$1.readEntryComplete(PerChannelBookieClient.java:1885)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleReadResponse(PerChannelBookieClient.java:1967)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleV2Response(PerChannelBookieClient.java:1919)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadV2ResponseCallback.run(PerChannelBookieClient.java:1397)
	at org.apache.bookkeeper.common.util.SingleThreadExecutor.safeRunTask(SingleThreadExecutor.java:137)
	at org.apache.bookkeeper.common.util.SingleThreadExecutor.run(SingleThreadExecutor.java:113)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run([email protected]/Thread.java:840)
"BookKeeperClientWorker-OrderedExecutor-8-0":
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:215)
	- waiting to lock <0x00000004080bf600> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080bf708> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:396)
	at org.apache.bookkeeper.proto.BookieClientImpl$ChannelReadyForAddEntryCallback.operationComplete(BookieClientImpl.java:356)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2548)
	at org.apache.bookkeeper.proto.PerChannelBookieClient$ConnectionFutureListener.operationComplete(PerChannelBookieClient.java:2453)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.processBookieNotResolvedError(PerChannelBookieClient.java:334)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:543)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:677)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:120)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.obtain(DefaultPerChannelBookieClientPool.java:115)
	at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:309)
	at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150)
	at org.apache.bookkeeper.client.PendingAddOp.unsetSuccessAndSendWriteRequest(PendingAddOp.java:225)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.client.LedgerHandle.unsetSuccessAndSendWriteRequest(LedgerHandle.java:2007)
	at org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:227)
	- locked <0x00000004080bf9b8> (a java.lang.Object)
	at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:353)
	- locked <0x00000004080b9600> (a org.apache.bookkeeper.client.PendingAddOp)
	at org.apache.bookkeeper.proto.BookieClientImpl.completeAdd(BookieClientImpl.java:284)
	at org.apache.bookkeeper.proto.BookieClientImpl.access$000(BookieClientImpl.java:78)

Found 1 deadlock.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Dead lock error.
4 participants