[
https://issues.apache.org/jira/browse/BOOKKEEPER-516?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13533017#comment-13533017
]
Ivan Kelly commented on BOOKKEEPER-516:
---------------------------------------
Looking at the logs, this jumped out.
{code}
2012-12-14 19:56:28,521 - INFO -
[pool-216-thread-1:PerChannelBookieClient@118] - Connecting to bookie:
/192.87.106.251:15068
3,0001,3,0001,3,0001,2012-12-14 19:56:28,527 - ERROR -
[pool-216-thread-1:PendingReadOp$LedgerEntryRequest@174] - Interrupted reading
entry L3-E1
java.lang.InterruptedException
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1279)
at java.util.concurrent.Semaphore.acquire(Semaphore.java:286)
at
org.apache.bookkeeper.client.PendingReadOp.sendReadTo(PendingReadOp.java:301)
at
org.apache.bookkeeper.client.PendingReadOp$LedgerEntryRequest.sendNextRead(PendingReadOp.java:170)
at
org.apache.bookkeeper.client.PendingReadOp$LedgerEntryRequest.logErrorAndReattemptRead(PendingReadOp.java:206)
at
org.apache.bookkeeper.client.PendingReadOp.readEntryComplete(PendingReadOp.java:315)
at
org.apache.bookkeeper.proto.BookieClient$4.operationComplete(BookieClient.java:149)
at
org.apache.bookkeeper.proto.BookieClient$4.operationComplete(BookieClient.java:144)
at
org.apache.bookkeeper.proto.PerChannelBookieClient$1.operationComplete(PerChannelBookieClient.java:163)
at
org.jboss.netty.channel.DefaultChannelFuture.notifyListener(DefaultChannelFuture.java:381)
at
org.jboss.netty.channel.DefaultChannelFuture.addListener(DefaultChannelFuture.java:148)
at
org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:129)
at
org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:199)
at
org.apache.bookkeeper.proto.BookieClient.readEntry(BookieClient.java:144)
at
org.apache.bookkeeper.client.PendingReadOp.sendReadTo(PendingReadOp.java:303)
at
org.apache.bookkeeper.client.PendingReadOp$LedgerEntryRequest.sendNextRead(PendingReadOp.java:170)
at
org.apache.bookkeeper.client.PendingReadOp$LedgerEntryRequest.maybeSendSpeculativeRead(PendingReadOp.java:142)
at
org.apache.bookkeeper.client.PendingReadOp$1.run(PendingReadOp.java:268)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
at
java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:317)
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:150)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$101(ScheduledThreadPoolExecutor.java:98)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.runPeriodic(ScheduledThreadPoolExecutor.java:180)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:204)
at
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
{code}
Basically, the speculative read is allowing reentrance to occur and we're
getting hung up on the opCounterSemaphore. I've attached a patch that should
fix this. I'll try to think of a way to test it.
> TestSpeculativeRead failed in Jenkins
> -------------------------------------
>
> Key: BOOKKEEPER-516
> URL: https://issues.apache.org/jira/browse/BOOKKEEPER-516
> Project: Bookkeeper
> Issue Type: Bug
> Components: bookkeeper-client
> Reporter: Flavio Junqueira
> Priority: Blocker
> Fix For: 4.2.0
>
> Attachments:
> 0001-BOOKKEEPER-516-TestSpeculativeRead-failed-in-Jenkins.patch
>
>
> Here is the jenkins link:
> https://builds.apache.org/job/bookkeeper-trunk/org.apache.bookkeeper$bookkeeper-server/864/testReport/
> and concretely the relevant part of the stack traces:
> {noformat}
> junit.framework.AssertionFailedError: null
> at junit.framework.Assert.fail(Assert.java:47)
> at junit.framework.Assert.assertTrue(Assert.java:20)
> at junit.framework.Assert.assertTrue(Assert.java:27)
> at
> org.apache.bookkeeper.client.TestSpeculativeRead$LatchCallback.expectSuccess(TestSpeculativeRead.java:100)
> at
> org.apache.bookkeeper.client.TestSpeculativeRead.testSpeculativeReadFirstReadCompleteIsOk(TestSpeculativeRead.java:268)
> {noformat}
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira