Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Critical Critical
    • Resolution: Fixed
    • Affects Version/s: 4.1.0
    • Fix Version/s: 4.1.0
    • Component/s: hedwig-server
    • Labels:
      None

      Description

      LedgerHandle uses a Semaphore(opCounterSem) with a default value of 5000 permits to implement throttling for outstanding requests. This is causing a deadlock under high load. What I've observed is the following - There are a fixed number of threads created by OrderedSafeExecutor(mainWorkerPool in BookKeeper) and this is used to execute operations by PerChannelBookieClient. Under high load, the bookies are not able to satisfy requests at the rate at which they are being generated. This exhausts all permits in the Semaphore and any further operations block on lh.opCounterSem.acquire(). In this scenario, if the connection to the bookies is shut down, channelDisconnected in PerChannelBookieClient tries to error out all outstanding entries. The errorOutReadKey and errorOutAddKey functions enqueue these operations in the same mainWorkerPool, all threads in which are blocked on acquire. So, handleBookieFailure is never executed and the server stops responding.

      Blocking operations in a fixed size thread pool doesn't sound quite right. Temporarily, I fixed this by having another ExecutorService for every PerChannelBookieClient and queuing the operations from the errorOut* functions in it, but this is just a quick fix. I feel that the server shouldn't rely on LedgerHandle to throttle connections, but do this itself. Any other ideas on how to fix this? I'd be happy to contribute a patch.

      1. BK-215.patch
        21 kB
        Sijie Guo
      2. BK-215.patch_v2
        22 kB
        Sijie Guo
      3. BK-215.patch_v3
        29 kB
        Sijie Guo
      4. BK-215.patch_v4
        21 kB
        Sijie Guo
      5. BK-215-check-deadlock.patch
        52 kB
        Sijie Guo
      6. DeadlockCheckOrderedSafeExecutor.java
        3 kB
        Sijie Guo
      7. hedwig_ts.log
        52 kB
        Aniruddha

        Issue Links

          Activity

          Hide
          Flavio Junqueira added a comment -

          Hi Aniruddha, The thread blocking due to the exhaustion of permits is the application thread, not a thread from the bk client pool. Consequently, I don't see how the deadlock situation you describe can happen. Is there anything I'm missing?

          Show
          Flavio Junqueira added a comment - Hi Aniruddha, The thread blocking due to the exhaustion of permits is the application thread, not a thread from the bk client pool. Consequently, I don't see how the deadlock situation you describe can happen. Is there anything I'm missing?
          Hide
          Ivan Kelly added a comment -

          opCounterSem is only acquired in PendingReadOp#initiate and LedgerHandle#doAsyncAddEntry, which are only run in the context of the application thread, so this deadlock cannot happen as described.

          @Aniruddha, a dump of all thread stacks would be very helpful in diagnosing the problem here.

          Show
          Ivan Kelly added a comment - opCounterSem is only acquired in PendingReadOp#initiate and LedgerHandle#doAsyncAddEntry, which are only run in the context of the application thread, so this deadlock cannot happen as described. @Aniruddha, a dump of all thread stacks would be very helpful in diagnosing the problem here.
          Hide
          Aniruddha added a comment - - edited

          Hi Ivan and Flavio,

          I've attached the thread dump of the hedwig-server when it is deadlocked. As you can see, both pool-1-thread-1 and pool-52-thread-1 are blocked on the semaphore. pool-52-thread-1 is one of the threads in the mainWorkerPool. It seems that PendingReadOp is executed in pool-1-thread-1 always, except for just before I get the error, when it's executed in pool-52-thread-1. Now, when the channel is disconnected, the errorOut* functions enqueue all their SafeRunnables in this same thread (using OrderedSafeExecutor.submitOrdered with ledgerId as the key) - pool-52-thread-1. Even when PerChannelBookieClient closes the channel after errorOutOutstandingEntries, the callbacks that would release the semaphore are never called because all of them are being queued in the same thread. So, now you have a PendingReadOp waiting to acquire a lock at the front of the queue while operations that will release the lock are queued behind it.

          In short, there is a PendingReadOp being queued in pool-52-thread-1 which shouldn't be there. The only operations in that thread should be PendingAddOps.

          edit : From the logfile it seems that PendingReadOps are being queued in pool-52-thread-1 quite a few times. In all previous instances, it could acquire the Semaphore.

          Show
          Aniruddha added a comment - - edited Hi Ivan and Flavio, I've attached the thread dump of the hedwig-server when it is deadlocked. As you can see, both pool-1-thread-1 and pool-52-thread-1 are blocked on the semaphore. pool-52-thread-1 is one of the threads in the mainWorkerPool. It seems that PendingReadOp is executed in pool-1-thread-1 always, except for just before I get the error, when it's executed in pool-52-thread-1. Now, when the channel is disconnected, the errorOut* functions enqueue all their SafeRunnables in this same thread (using OrderedSafeExecutor.submitOrdered with ledgerId as the key) - pool-52-thread-1. Even when PerChannelBookieClient closes the channel after errorOutOutstandingEntries, the callbacks that would release the semaphore are never called because all of them are being queued in the same thread. So, now you have a PendingReadOp waiting to acquire a lock at the front of the queue while operations that will release the lock are queued behind it. In short, there is a PendingReadOp being queued in pool-52-thread-1 which shouldn't be there. The only operations in that thread should be PendingAddOps. edit : From the logfile it seems that PendingReadOps are being queued in pool-52-thread-1 quite a few times. In all previous instances, it could acquire the Semaphore.
          Hide
          Sijie Guo added a comment -

          thanks Aniruddha providing thread stacks.

          yes, pool-52-thread-1 introduced the deadlock. in Hedwig bookkeeper persistence manager, it call next asyncReadEntries in the callback of previous asyncReadEntries.

          One possible fix is in hedwig part, we need to submit a Runnable to execute asyncReadEntries in other thread, not in the bookkeeper threads pool to avoid such deadlock. This solution could fix the hedwig issue.

          beside that, we can't assume that other users would not do such action to call readEntries in a callback. so a fix in bookkeeper itself may be needed.

          Show
          Sijie Guo added a comment - thanks Aniruddha providing thread stacks. yes, pool-52-thread-1 introduced the deadlock. in Hedwig bookkeeper persistence manager, it call next asyncReadEntries in the callback of previous asyncReadEntries. One possible fix is in hedwig part, we need to submit a Runnable to execute asyncReadEntries in other thread, not in the bookkeeper threads pool to avoid such deadlock. This solution could fix the hedwig issue. beside that, we can't assume that other users would not do such action to call readEntries in a callback. so a fix in bookkeeper itself may be needed.
          Hide
          Flavio Junqueira added a comment -

          I don't think it is good practice to have calls to bookkeeper itself from a callback. The callback will be executed by a bookkeeper thread, so you really want to give control back to the application shortly.

          My preference is to leave the bookkeeper client as is.

          Show
          Flavio Junqueira added a comment - I don't think it is good practice to have calls to bookkeeper itself from a callback. The callback will be executed by a bookkeeper thread, so you really want to give control back to the application shortly. My preference is to leave the bookkeeper client as is.
          Hide
          Sijie Guo added a comment -

          adding a patch, 1) includes a test case which reproduce the hedwig issue as thread stacks provided in this jira, 2) includes a test case to test ledger recovery with limited permits, which also introduce same deadlock issue as hedwig, 3) add a possible fix to avoid call bookkeeper operation in bookkeeper callback.

          this patch is based on BOOKKEEPER-173 's patch, because I need to limit the number of worker threads in the test case to 1, so I could let the ledger operations are executed in same thread to reproduce deadlock issue.

          Show
          Sijie Guo added a comment - adding a patch, 1) includes a test case which reproduce the hedwig issue as thread stacks provided in this jira, 2) includes a test case to test ledger recovery with limited permits, which also introduce same deadlock issue as hedwig, 3) add a possible fix to avoid call bookkeeper operation in bookkeeper callback. this patch is based on BOOKKEEPER-173 's patch, because I need to limit the number of worker threads in the test case to 1, so I could let the ledger operations are executed in same thread to reproduce deadlock issue.
          Hide
          Aniruddha added a comment -

          Thanks. The deadlock does not happen after I apply the patch. However, it might be worth looking into moving the lock acquisition out of PendingReadOp and putting it into the application. Is there any reason why the acquire is inside the initiate method and not in an application thread?

          Show
          Aniruddha added a comment - Thanks. The deadlock does not happen after I apply the patch. However, it might be worth looking into moving the lock acquisition out of PendingReadOp and putting it into the application. Is there any reason why the acquire is inside the initiate method and not in an application thread?
          Hide
          Flavio Junqueira added a comment -

          This is the code for asyncReadEntries:

          public void asyncReadEntries(long firstEntry, long lastEntry,
                                           ReadCallback cb, Object ctx) {
                  // Little sanity check
                  if (firstEntry < 0 || lastEntry > lastAddConfirmed
                          || firstEntry > lastEntry) {
                      cb.readComplete(BKException.Code.ReadException, this, null, ctx);
                      return;
                  }
          
                  try {
                      new PendingReadOp(this, firstEntry, lastEntry, cb, ctx).initiate();
                  } catch (InterruptedException e) {
                      cb.readComplete(BKException.Code.InterruptedException, this, null, ctx);
                  }
              }
          

          initiate() is called form the application thread, right?

          Show
          Flavio Junqueira added a comment - This is the code for asyncReadEntries: public void asyncReadEntries(long firstEntry, long lastEntry, ReadCallback cb, Object ctx) { // Little sanity check if (firstEntry < 0 || lastEntry > lastAddConfirmed || firstEntry > lastEntry) { cb.readComplete(BKException.Code.ReadException, this, null, ctx); return; } try { new PendingReadOp(this, firstEntry, lastEntry, cb, ctx).initiate(); } catch (InterruptedException e) { cb.readComplete(BKException.Code.InterruptedException, this, null, ctx); } } initiate() is called form the application thread, right?
          Hide
          Aniruddha added a comment -

          Sorry, should have been clearer. Would it be better if we have a doAsyncReadEntry function inside LedgerHandle, call this from asyncReadEntries and let it acquire the lock and queue the PendingReadOp into a threadpool? This is similar to what is being done for add operations.

          Show
          Aniruddha added a comment - Sorry, should have been clearer. Would it be better if we have a doAsyncReadEntry function inside LedgerHandle, call this from asyncReadEntries and let it acquire the lock and queue the PendingReadOp into a threadpool? This is similar to what is being done for add operations.
          Hide
          Sijie Guo added a comment -

          I don't think we need to do that. from an application they call asyncReadEntries and asyncAddEntry, it has done the same thing to acquire lock. (one thing we might need to add is more documentation to let user know they acquire the lock when calling these two methods)

          Beside that, add operation is quite different from read operation.

          for add operation, bookkeeper needs to guarantee order so it has a PendingAddOps queue to queue all the add operations, so the acquire is better to execute in LedgerHandle before putting it in the pending queue.

          for read operation, we don't need to guarantee the order between different asycReadEntries calls, so we don't has a queue for read ops. beside that, read is actually a range read so we have to acquire the lock per entry for a asyncReadEntries operation, so PendingReadOp is a better place to manage such logic for a read op.

          does it make sense for you?

          BTW, just curious, why the issue is marked as fixed?

          Show
          Sijie Guo added a comment - I don't think we need to do that. from an application they call asyncReadEntries and asyncAddEntry, it has done the same thing to acquire lock. (one thing we might need to add is more documentation to let user know they acquire the lock when calling these two methods) Beside that, add operation is quite different from read operation. for add operation, bookkeeper needs to guarantee order so it has a PendingAddOps queue to queue all the add operations, so the acquire is better to execute in LedgerHandle before putting it in the pending queue. for read operation, we don't need to guarantee the order between different asycReadEntries calls, so we don't has a queue for read ops. beside that, read is actually a range read so we have to acquire the lock per entry for a asyncReadEntries operation, so PendingReadOp is a better place to manage such logic for a read op. does it make sense for you? BTW, just curious, why the issue is marked as fixed?
          Hide
          Sijie Guo added a comment -

          the patch is not passed reviewing, so just reopen it.

          Show
          Sijie Guo added a comment - the patch is not passed reviewing, so just reopen it.
          Hide
          Flavio Junqueira added a comment -

          Hi Sijie, Here are some comments on the patch:

          • One clarification. I understand the call to lh.bk.callbackWorker.submitOrdered in readComplete, but not the one in doRecoveryRead. Why do we need to give it to a worker thread in this case?
          • It is not performance critical in this case, but it sounds like a good ideal in general to have LOG.debug statements wrapped with isDebugEnabled() (LedgerRecoveryOp:86). You may have simply missed this one.
          • Is this change gratuitous or really necessary:
                protected Bookie newBookie(ServerConfiguration conf)
                    throws IOException, KeeperException, InterruptedException, BookieException {
                    return new Bookie(conf);
                }
            
            
          • testRecoveryDeadlockWithLimitedPermits() has no assertion or fail clause. What is it testing?
          • I'm not entirely sure why we need this method:
              /**
               * Add configuration object.
               *
               * @param conf configuration object
               */
              public void addConf(Configuration otherConf) throws ConfigurationException {
                  conf.addConfiguration(otherConf);
              }
          

          Why can't we set the bk client configuration in the constructor?

          • Typo: "... so a scan request need to scan over two ledger" -> "... so a scan request need to scan over two ledgers"
          • In TestDeadlock, if I understand the test correctly, consumeQueue.take() is supposed to hang due to the bug of this jira. Consequently, we have to wait until junit times out the test? I was wondering if there is a way of avoiding the time out.
          • Suggestion for rephrasing comment:
          // it obtains the permit and wait for a response,
          // but the response is delayed and readEntries is called
          // in the readComplete callback to read entries of the 
          // same ledger. since there is no permit, it blocks.
          
          Show
          Flavio Junqueira added a comment - Hi Sijie, Here are some comments on the patch: One clarification. I understand the call to lh.bk.callbackWorker.submitOrdered in readComplete, but not the one in doRecoveryRead. Why do we need to give it to a worker thread in this case? It is not performance critical in this case, but it sounds like a good ideal in general to have LOG.debug statements wrapped with isDebugEnabled() (LedgerRecoveryOp:86). You may have simply missed this one. Is this change gratuitous or really necessary: protected Bookie newBookie(ServerConfiguration conf) throws IOException, KeeperException, InterruptedException, BookieException { return new Bookie(conf); } testRecoveryDeadlockWithLimitedPermits() has no assertion or fail clause. What is it testing? I'm not entirely sure why we need this method: /** * Add configuration object. * * @param conf configuration object */ public void addConf(Configuration otherConf) throws ConfigurationException { conf.addConfiguration(otherConf); } Why can't we set the bk client configuration in the constructor? Typo: "... so a scan request need to scan over two ledger" -> "... so a scan request need to scan over two ledgers" In TestDeadlock, if I understand the test correctly, consumeQueue.take() is supposed to hang due to the bug of this jira. Consequently, we have to wait until junit times out the test? I was wondering if there is a way of avoiding the time out. Suggestion for rephrasing comment: // it obtains the permit and wait for a response, // but the response is delayed and readEntries is called // in the readComplete callback to read entries of the // same ledger. since there is no permit, it blocks.
          Hide
          Flavio Junqueira added a comment -

          Cancelling patch until comments are addressed.

          Show
          Flavio Junqueira added a comment - Cancelling patch until comments are addressed.
          Hide
          Sijie Guo added a comment -

          thanks Flavio for reviewing.

          > but not the one in doRecoveryRead. Why do we need to give it to a worker thread in this case?

          doRecoveryRead() will be executed in #addComplete, which is also in the callback of bookkeeper. so it should do same as what it did in #readComplete.

          > It is not performance critical in this case, but it sounds like a good ideal in general to have LOG.debug statements wrapped with isDebugEnabled() (LedgerRecoveryOp:86). You may have simply missed this one.

          if we use '+' to concatenate string it would introduce the performance issue. but we don't concatenate the string in debug message, so it would not introduce the performance issue. besides that, I remembered that because we use slf4j, Ivan suggested to use such kind of pattern to log debug message.

          > Is this change gratuitous or really necessary:

          actually I used this method in hedwig BookKeeperTestBase class, which extends existed Bookie class to simulate response delay for #readEntry. It helps testing the deadlock in bookkeeper persistence manager.

          > testRecoveryDeadlockWithLimitedPermits() has no assertion or fail clause. What is it testing?

          if we don't apply the patch, the test case will hang due to deadlock. so I am not sure how to add assertion and fail clause. what is your opinion?

          > I'm not entirely sure why we need this method:

          the method is used in TestDeadLock#getServerConfiguration. it used to load a bookkeeper client configuration object. so the hub server could use bookkeeper client settings we provided.

          > In TestDeadlock, if I understand the test correctly, consumeQueue.take() is supposed to hang due to the bug of this jira. Consequently, we have to wait until junit times out the test? I was wondering if there is a way of avoiding the time out.

          yes. we have to wait until timeout if we don't apply this patch. I have no perfect solution to test such hang due to deadlock.

          > typos & comments

          yeah. thanks for fixing them. I would update them to new patch.

          Show
          Sijie Guo added a comment - thanks Flavio for reviewing. > but not the one in doRecoveryRead. Why do we need to give it to a worker thread in this case? doRecoveryRead() will be executed in #addComplete, which is also in the callback of bookkeeper. so it should do same as what it did in #readComplete. > It is not performance critical in this case, but it sounds like a good ideal in general to have LOG.debug statements wrapped with isDebugEnabled() (LedgerRecoveryOp:86). You may have simply missed this one. if we use '+' to concatenate string it would introduce the performance issue. but we don't concatenate the string in debug message, so it would not introduce the performance issue. besides that, I remembered that because we use slf4j, Ivan suggested to use such kind of pattern to log debug message. > Is this change gratuitous or really necessary: actually I used this method in hedwig BookKeeperTestBase class, which extends existed Bookie class to simulate response delay for #readEntry. It helps testing the deadlock in bookkeeper persistence manager. > testRecoveryDeadlockWithLimitedPermits() has no assertion or fail clause. What is it testing? if we don't apply the patch, the test case will hang due to deadlock. so I am not sure how to add assertion and fail clause. what is your opinion? > I'm not entirely sure why we need this method: the method is used in TestDeadLock#getServerConfiguration. it used to load a bookkeeper client configuration object. so the hub server could use bookkeeper client settings we provided. > In TestDeadlock, if I understand the test correctly, consumeQueue.take() is supposed to hang due to the bug of this jira. Consequently, we have to wait until junit times out the test? I was wondering if there is a way of avoiding the time out. yes. we have to wait until timeout if we don't apply this patch. I have no perfect solution to test such hang due to deadlock. > typos & comments yeah. thanks for fixing them. I would update them to new patch.
          Hide
          Sijie Guo added a comment -

          attach a new patch to improve comments according to Flavio's suggestions.

          Show
          Sijie Guo added a comment - attach a new patch to improve comments according to Flavio's suggestions.
          Hide
          Flavio Junqueira added a comment - - edited

          One idea occurred to me and I wanted to run it by you guys. I don't think it is critical that we throttle for recovery, so I was wondering if it makes sense that the calls to read entries from LedgerRecoveryOp do not try to acquire permits. This way we avoid the problem altogether.

          Show
          Flavio Junqueira added a comment - - edited One idea occurred to me and I wanted to run it by you guys. I don't think it is critical that we throttle for recovery, so I was wondering if it makes sense that the calls to read entries from LedgerRecoveryOp do not try to acquire permits. This way we avoid the problem altogether.
          Hide
          Sijie Guo added a comment -

          I think it is OK to not try to acquire permits in LedgerRecoveryOp, it could resolve the issue in LedgerRecoveryOp. but it could not resolve the issue in BookKeeper persistence manager in hedwig. so I am not sure I understand what you said that "we avoid the problem altogether".

          Show
          Sijie Guo added a comment - I think it is OK to not try to acquire permits in LedgerRecoveryOp, it could resolve the issue in LedgerRecoveryOp. but it could not resolve the issue in BookKeeper persistence manager in hedwig. so I am not sure I understand what you said that "we avoid the problem altogether".
          Hide
          Flavio Junqueira added a comment -

          Your patch seems to fix the hedwig issue without changing the flow of regular asyncReadEntries. It only changes LedgerRecoveryOp on the bookkeeper side, right? One option I see is to keep your changes to hedwig and not have recovery acquiring permits.

          One clarification, if you don't mind, Sijie. It is not clear from your patch how your modifications to the persistence manager prevents callbacks from calling asyncReadEntries. Could you give me some more insight, please?

          Show
          Flavio Junqueira added a comment - Your patch seems to fix the hedwig issue without changing the flow of regular asyncReadEntries. It only changes LedgerRecoveryOp on the bookkeeper side, right? One option I see is to keep your changes to hedwig and not have recovery acquiring permits. One clarification, if you don't mind, Sijie. It is not clear from your patch how your modifications to the persistence manager prevents callbacks from calling asyncReadEntries. Could you give me some more insight, please?
          Hide
          Sijie Guo added a comment -

          sure, the code change in bookkeeper persistence manager is to put the asyncReadEntries in opQueue instead of calling asyncReadEntries directly in the callback.

                                   }
                               }
           
          -                    startReadingFrom(imlr.startSeqIdIncluded + entry.getEntryId() + 1);
          -
          +                    // continue scanning messages
          +                    scanMessages(request, imlr.startSeqIdIncluded + entry.getEntryId() + 1);
                           }
                       }, request.ctx);
                   }
          
          

          the detail code change is as above. it call scanMessages, which put a range scan op again in topic op queue, instead of startReadingFrom. so the asyncReadEntries would be executed only the range op is executed, which is in hedwig thread not in bookkeeper thread.

          Show
          Sijie Guo added a comment - sure, the code change in bookkeeper persistence manager is to put the asyncReadEntries in opQueue instead of calling asyncReadEntries directly in the callback. } } - startReadingFrom(imlr.startSeqIdIncluded + entry.getEntryId() + 1); - + // continue scanning messages + scanMessages(request, imlr.startSeqIdIncluded + entry.getEntryId() + 1); } }, request.ctx); } the detail code change is as above. it call scanMessages, which put a range scan op again in topic op queue, instead of startReadingFrom. so the asyncReadEntries would be executed only the range op is executed, which is in hedwig thread not in bookkeeper thread.
          Hide
          Sijie Guo added a comment -

          Flavio and I had a discussion about this jira,

          1) for LedgerRecoveryOp, it would better to acquire permit per recovery op. we don't need to acquire permits for the read/add ops executed in a recovery op. so the issue could be resolved. to do the change, we need a flag in PendingAddOp/PendingReadOp to control whether acquire/release permits when read/add. we leverage PendingAddOp#enableRecoveryAdd() as the flag.

          2) in Bookkeeper SafeOrderExecutor, I proposed to introduce deadlock checking mechanism in runtime. when a callback is submitted to SafeOrderExecutor, we would check whether the thread is blocked on the semaphore or not. If the thread is blocked, we would fail the callback directly. Doing such deadlock checking would affect performance, so it would better to provide a flag to control whether enabling it or not. The benefit of introducing such mechanism, we could avoid using timeout in the test case.

          I would like to come out a draft patch about the above proposal, then we could discuss more.

          Show
          Sijie Guo added a comment - Flavio and I had a discussion about this jira, 1) for LedgerRecoveryOp, it would better to acquire permit per recovery op. we don't need to acquire permits for the read/add ops executed in a recovery op. so the issue could be resolved. to do the change, we need a flag in PendingAddOp/PendingReadOp to control whether acquire/release permits when read/add. we leverage PendingAddOp#enableRecoveryAdd() as the flag. 2) in Bookkeeper SafeOrderExecutor, I proposed to introduce deadlock checking mechanism in runtime. when a callback is submitted to SafeOrderExecutor, we would check whether the thread is blocked on the semaphore or not. If the thread is blocked, we would fail the callback directly. Doing such deadlock checking would affect performance, so it would better to provide a flag to control whether enabling it or not. The benefit of introducing such mechanism, we could avoid using timeout in the test case. I would like to come out a draft patch about the above proposal, then we could discuss more.
          Hide
          Sijie Guo added a comment -

          Attach a prototype of deadlock check executor for discussion.

          the idea for deadlock check executor is used Semaphore#getQueuedThreads() to check whether a thread is blocked when summiting a task to it.

          If a deadlock is found, a RejectedExecutionException is thrown. so the thread should catch such exception to handle it (such as fail the response directly).

          Unfortunately, I found that submitOrdered is used at least 5 places. It would make the source code unmaintainable.

          so I am not sure it worth doing such deadlock checking. Flavio, how is your opinion?

          another thing that I were thinking is the way using Semaphore to throttle the best way. I thought throttle is to not overwhelm a bookie server, so why not throttle in PerChannelBookieClient which talks with a bookie server directly.

          Show
          Sijie Guo added a comment - Attach a prototype of deadlock check executor for discussion. the idea for deadlock check executor is used Semaphore#getQueuedThreads() to check whether a thread is blocked when summiting a task to it. If a deadlock is found, a RejectedExecutionException is thrown. so the thread should catch such exception to handle it (such as fail the response directly). Unfortunately, I found that submitOrdered is used at least 5 places. It would make the source code unmaintainable. so I am not sure it worth doing such deadlock checking. Flavio, how is your opinion? another thing that I were thinking is the way using Semaphore to throttle the best way. I thought throttle is to not overwhelm a bookie server, so why not throttle in PerChannelBookieClient which talks with a bookie server directly.
          Hide
          Sijie Guo added a comment -

          BTW, attach a v3 patch that address 1) in previous comment, not acquire/release permits in ledger recovery op.

          the deadlock checking idea introduced is trying to remove timeout mechanism in testing deadlock. but from my draft patch, seems that is not easy to adapt a deadlock checking mechanism. so how about creating a separated for it?

          Show
          Sijie Guo added a comment - BTW, attach a v3 patch that address 1) in previous comment, not acquire/release permits in ledger recovery op. the deadlock checking idea introduced is trying to remove timeout mechanism in testing deadlock. but from my draft patch, seems that is not easy to adapt a deadlock checking mechanism. so how about creating a separated for it?
          Hide
          Flavio Junqueira added a comment -

          It really feels like we have three issues here: the hedwig deadlock, the changes to recovery, deadlock detection. In this jira, it sounds like we only need to solve the first. The other two we should address in different jiras.

          On the changes to recovery to not acquire/release permits, I don't really like the conditional acquire/permit, and I was thinking if we could remove it by just acquiring and releasing in different places. For adds, we could acquire the permits before doAsyncAddEntry, while for the release part, we may need a different callback in PendingAddOp. What do you think, Sijie?

          Show
          Flavio Junqueira added a comment - It really feels like we have three issues here: the hedwig deadlock, the changes to recovery, deadlock detection. In this jira, it sounds like we only need to solve the first. The other two we should address in different jiras. On the changes to recovery to not acquire/release permits, I don't really like the conditional acquire/permit, and I was thinking if we could remove it by just acquiring and releasing in different places. For adds, we could acquire the permits before doAsyncAddEntry, while for the release part, we may need a different callback in PendingAddOp. What do you think, Sijie?
          Hide
          Sijie Guo added a comment -

          I agreed to separated it into three jiras and we fix the first one.

          > For adds, we could acquire the permits before doAsyncAddEntry, while for the release part, we may need a different callback in PendingAddOp.

          for Add it is OK, but how about readEntries? since releasing permits in PendingReadOp is not in single place, if you want to fix it in a different callback, I think you have to wrap the callback passed by user and do release in the wrapper.

          What I am thinking to resolve recovery deadlock issue is to revisit the throttle mechanism used in bookkeeper.

          Show
          Sijie Guo added a comment - I agreed to separated it into three jiras and we fix the first one. > For adds, we could acquire the permits before doAsyncAddEntry, while for the release part, we may need a different callback in PendingAddOp. for Add it is OK, but how about readEntries? since releasing permits in PendingReadOp is not in single place, if you want to fix it in a different callback, I think you have to wrap the callback passed by user and do release in the wrapper. What I am thinking to resolve recovery deadlock issue is to revisit the throttle mechanism used in bookkeeper.
          Hide
          Sijie Guo added a comment -

          Attach a new patch just resolve the deadlock issue in Hedwig bookkeeper persistence manager.

          for the ledger recovery issue, it would be discussion in BOOKKEEPER-239. And the deadlock checking mechanism would be resolved in BOOKKEEPER-240

          Show
          Sijie Guo added a comment - Attach a new patch just resolve the deadlock issue in Hedwig bookkeeper persistence manager. for the ledger recovery issue, it would be discussion in BOOKKEEPER-239 . And the deadlock checking mechanism would be resolved in BOOKKEEPER-240
          Hide
          Ivan Kelly added a comment -

          lgtm +1,

          committing

          Show
          Ivan Kelly added a comment - lgtm +1, committing
          Hide
          Ivan Kelly added a comment -

          Committed as r1335958

          Show
          Ivan Kelly added a comment - Committed as r1335958
          Hide
          Hudson added a comment -

          Integrated in bookkeeper-trunk #496 (See https://builds.apache.org/job/bookkeeper-trunk/496/)
          BOOKKEEPER-215: Deadlock occurs under high load (sijie via ivank) (Revision 1335958)

          Result = SUCCESS
          ivank :
          Files :

          • /zookeeper/bookkeeper/trunk/CHANGES.txt
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
          • /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java
          • /zookeeper/bookkeeper/trunk/hedwig-client/src/main/java/org/apache/hedwig/conf/AbstractConfiguration.java
          • /zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/persistence/BookkeeperPersistenceManager.java
          • /zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/HedwigHubTestBase.java
          • /zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/BookKeeperTestBase.java
          • /zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestDeadlock.java
          Show
          Hudson added a comment - Integrated in bookkeeper-trunk #496 (See https://builds.apache.org/job/bookkeeper-trunk/496/ ) BOOKKEEPER-215 : Deadlock occurs under high load (sijie via ivank) (Revision 1335958) Result = SUCCESS ivank : Files : /zookeeper/bookkeeper/trunk/CHANGES.txt /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java /zookeeper/bookkeeper/trunk/hedwig-client/src/main/java/org/apache/hedwig/conf/AbstractConfiguration.java /zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/persistence/BookkeeperPersistenceManager.java /zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/HedwigHubTestBase.java /zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/BookKeeperTestBase.java /zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestDeadlock.java

            People

            • Assignee:
              Sijie Guo
              Reporter:
              Aniruddha
            • Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development