Kafka
  1. Kafka
  2. KAFKA-1461

Replica fetcher thread does not implement any back-off behavior

    Details

    • Type: Improvement Improvement
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 0.8.1.1
    • Fix Version/s: 0.8.3
    • Component/s: replication
    • Labels:

      Description

      The current replica fetcher thread will retry in a tight loop if any error occurs during the fetch call. For example, we've seen cases where the fetch continuously throws a connection refused exception leading to several replica fetcher threads that spin in a pretty tight loop.

      To a much lesser degree this is also an issue in the consumer fetcher thread, although the fact that erroring partitions are removed so a leader can be re-discovered helps some.

      1. KAFKA-1461.patch
        19 kB
        Sriharsha Chintalapani
      2. KAFKA-1461.patch
        12 kB
        Sriharsha Chintalapani
      3. KAFKA-1461_2015-04-07_08:41:18.patch
        20 kB
        Sriharsha Chintalapani
      4. KAFKA-1461_2015-04-03_20:48:34.patch
        19 kB
        Sriharsha Chintalapani
      5. KAFKA-1461_2015-03-27_17:02:32.patch
        17 kB
        Sriharsha Chintalapani
      6. KAFKA-1461_2015-03-27_16:56:45.patch
        15 kB
        Sriharsha Chintalapani
      7. KAFKA-1461_2015-03-27_15:31:11.patch
        15 kB
        Sriharsha Chintalapani
      8. KAFKA-1461_2015-03-17_16:03:33.patch
        12 kB
        Sriharsha Chintalapani
      9. KAFKA-1461_2015-03-12_13:54:51.patch
        13 kB
        Sriharsha Chintalapani
      10. KAFKA-1461_2015-03-11_18:17:51.patch
        12 kB
        Sriharsha Chintalapani
      11. KAFKA-1461_2015-03-11_10:41:26.patch
        18 kB
        Sriharsha Chintalapani

        Issue Links

          Activity

          Hide
          Nicolae Marasoiu added a comment -

          Hi,

          So I guess in this block:
          try

          { trace("Issuing to broker %d of fetch request %s".format(sourceBroker.id, fetchRequest)) response = simpleConsumer.fetch(fetchRequest) }

          catch {
          case t: Throwable =>
          if (isRunning.get) {
          warn("Error in fetch %s. Possible cause: %s".format(fetchRequest, t.toString))
          partitionMapLock synchronized

          { partitionsWithError ++= partitionMap.keys }

          }
          }
          I should add a case for the specific scenario of connection timeout/refused/reset and introduce a backoff on that path?

          Show
          Nicolae Marasoiu added a comment - Hi, So I guess in this block: try { trace("Issuing to broker %d of fetch request %s".format(sourceBroker.id, fetchRequest)) response = simpleConsumer.fetch(fetchRequest) } catch { case t: Throwable => if (isRunning.get) { warn("Error in fetch %s. Possible cause: %s".format(fetchRequest, t.toString)) partitionMapLock synchronized { partitionsWithError ++= partitionMap.keys } } } I should add a case for the specific scenario of connection timeout/refused/reset and introduce a backoff on that path?
          Hide
          Guozhang Wang added a comment -

          I did not realize this ticket exist, and created the same one here (KAFKA-1629). It has some more detailed explanation of the issue though.

          Show
          Guozhang Wang added a comment - I did not realize this ticket exist, and created the same one here ( KAFKA-1629 ). It has some more detailed explanation of the issue though.
          Hide
          Joe Stein added a comment -

          Nicolae Cismaru are you working on this patch? If not can we assign it to unassigned so if someone wants to jump in and fix it, sure is annoying when it happens (like waiting on Recovering unflushed segment ) during that time every replica fetching from it spews the error ERROR kafka.server.ReplicaFetcherThread

          Show
          Joe Stein added a comment - Nicolae Cismaru are you working on this patch? If not can we assign it to unassigned so if someone wants to jump in and fix it, sure is annoying when it happens (like waiting on Recovering unflushed segment ) during that time every replica fetching from it spews the error ERROR kafka.server.ReplicaFetcherThread
          Hide
          Nicolae Marasoiu added a comment -

          I agree to give to someone else, did not made progress yet on this

          thank you

          În data de marți, 25 noiembrie 2014, Joe Stein (JIRA) <jira@apache.org> a

          Show
          Nicolae Marasoiu added a comment - I agree to give to someone else, did not made progress yet on this thank you În data de marți, 25 noiembrie 2014, Joe Stein (JIRA) <jira@apache.org> a
          Hide
          Sriharsha Chintalapani added a comment -

          Joe Stein Nicolae Marasoiu I can take this. I am looking at this code for another JIRA.

          Show
          Sriharsha Chintalapani added a comment - Joe Stein Nicolae Marasoiu I can take this. I am looking at this code for another JIRA.
          Hide
          Sriharsha Chintalapani added a comment - - edited

          Guozhang Wang I had the following code in my mind about backoff retries incase of any error. This code will be under ReplicaFetcherThread.handlePartitions.
          I am thinking off maintaining two maps in ReplicaFetcherThread
          private val partitionsWithErrorStandbyMap = new mutable.HashMap[TopicAndPartition, Long] // a (topic, partition) -> offset
          private val partitionsWithErrorMap = new mutable.HashMap[TopicAndPartition, Long] // a (topic, partition) -> timestamp
          one for offset and one for timestamp.
          remove the partitions from the AbstractFetcherThread.partitionsMap and add back to the map once the currentTime > partitionsWithErrorMap.timestamp + replicaFetcherRetryBackoffMs .
          I am not quite sure about maintaining these two maps . If its look ok to you , I'll send a patch or if you have any other approach please let me know.

            def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) {
          
              //add to the partitionsWithErrorMap with currentTime.
              for (partition <- partitions) {
                if(!partitionsWithErrorMap.contains(partition)) {
                  partitionsWithErrorMap.put(partition, System.currentTimeMillis())
                  currentOffset(partition) match {
                    case Some(offset: Long) =>  partitionsWithErrorStandbyMap.put(partition, offset)
                  }
                }
              }
              removePartitions(partitions.toSet)
              val partitionsToBeAdded = new mutable.HashMap[TopicAndPartition, Long]
              // process partitionsWithErrorMap and add partitions back if the backoff time elapsed.
              partitionsWithErrorMap.foreach {
                case((topicAndPartition, timeMs)) =>
                  if(System.currentTimeMillis() > timeMs + brokerConfig.replicaFetcherRetryBackoffMs) {
                    partitionsWithErrorStandbyMap.get(topicAndPartition) match {
                      case Some(offset: Long) => partitionsToBeAdded.put(topicAndPartition, offset)
                    }
                    partitionsWithErrorStandbyMap.remove(topicAndPartition)
                  }
              }
              addPartitions(partitionsToBeAdded)
            }
          
          Show
          Sriharsha Chintalapani added a comment - - edited Guozhang Wang I had the following code in my mind about backoff retries incase of any error. This code will be under ReplicaFetcherThread.handlePartitions. I am thinking off maintaining two maps in ReplicaFetcherThread private val partitionsWithErrorStandbyMap = new mutable.HashMap [TopicAndPartition, Long] // a (topic, partition) -> offset private val partitionsWithErrorMap = new mutable.HashMap [TopicAndPartition, Long] // a (topic, partition) -> timestamp one for offset and one for timestamp. remove the partitions from the AbstractFetcherThread.partitionsMap and add back to the map once the currentTime > partitionsWithErrorMap.timestamp + replicaFetcherRetryBackoffMs . I am not quite sure about maintaining these two maps . If its look ok to you , I'll send a patch or if you have any other approach please let me know. def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) { //add to the partitionsWithErrorMap with currentTime. for (partition <- partitions) { if (!partitionsWithErrorMap.contains(partition)) { partitionsWithErrorMap.put(partition, System .currentTimeMillis()) currentOffset(partition) match { case Some(offset: Long ) => partitionsWithErrorStandbyMap.put(partition, offset) } } } removePartitions(partitions.toSet) val partitionsToBeAdded = new mutable.HashMap[TopicAndPartition, Long ] // process partitionsWithErrorMap and add partitions back if the backoff time elapsed. partitionsWithErrorMap.foreach { case ((topicAndPartition, timeMs)) => if ( System .currentTimeMillis() > timeMs + brokerConfig.replicaFetcherRetryBackoffMs) { partitionsWithErrorStandbyMap.get(topicAndPartition) match { case Some(offset: Long ) => partitionsToBeAdded.put(topicAndPartition, offset) } partitionsWithErrorStandbyMap.remove(topicAndPartition) } } addPartitions(partitionsToBeAdded) }
          Hide
          Guozhang Wang added a comment -

          Sriharsha Chintalapani Sorry for the late reply.

          This fix looks good to me overall, except that we cannot potentially add partitions back only in the handlePartitionsWithErrors() call, since it will only be triggered when the next error happens. We can probably move this piece of code to processPartitionData().

          Another way to do this could be:

          1. Make the partitionMap in AbstractFetcherThread of a map from TopicAndPartition to OffsetAndState, where OffsetAndState contains the Offset (Long) and the State (active, inactive-with-delay). For simplicity we can just use Int here, and "active" would be 0, inactive would be the delay time.

          2. Adding another function called "delayPartitions" in AbstractFetcherThread, which set State to inactive with the delay time.

          3. In AbstractFetcherThread doWork() only include partitions with State 0 to send the fetch request, and also update the state values for non-zero partitions.

          Show
          Guozhang Wang added a comment - Sriharsha Chintalapani Sorry for the late reply. This fix looks good to me overall, except that we cannot potentially add partitions back only in the handlePartitionsWithErrors() call, since it will only be triggered when the next error happens. We can probably move this piece of code to processPartitionData(). Another way to do this could be: 1. Make the partitionMap in AbstractFetcherThread of a map from TopicAndPartition to OffsetAndState, where OffsetAndState contains the Offset (Long) and the State (active, inactive-with-delay). For simplicity we can just use Int here, and "active" would be 0, inactive would be the delay time. 2. Adding another function called "delayPartitions" in AbstractFetcherThread, which set State to inactive with the delay time. 3. In AbstractFetcherThread doWork() only include partitions with State 0 to send the fetch request, and also update the state values for non-zero partitions.
          Hide
          Idcmp added a comment - - edited

          This issue can be tickled on a multi-broker configuration by having brokers advertise host names that do not exist. (Say for example you're running Kafka in docker containers with custom hostnames

          Show
          Idcmp added a comment - - edited This issue can be tickled on a multi-broker configuration by having brokers advertise host names that do not exist. (Say for example you're running Kafka in docker containers with custom hostnames
          Hide
          Sriharsha Chintalapani added a comment -

          Created reviewboard https://reviews.apache.org/r/31366/diff/
          against branch origin/trunk

          Show
          Sriharsha Chintalapani added a comment - Created reviewboard https://reviews.apache.org/r/31366/diff/ against branch origin/trunk
          Hide
          Sriharsha Chintalapani added a comment -

          Guozhang Wang thanks for the pointers. Can you please take a look at the patch when you get a chance.

          Show
          Sriharsha Chintalapani added a comment - Guozhang Wang thanks for the pointers. Can you please take a look at the patch when you get a chance.
          Hide
          Jun Rao added a comment -

          Sriharsha Chintalapani, thanks for the patch. Managing the backoff per partition is a bit more complicated than I was expecting. The most common case that we want to handle here is that the fetcher is trying to fetch from a broker that's already down. In this case, the simplest approach is to just back off the fetcher (for all partitions) a bit.

          Another common case is that we are doing a controlled shutdown by moving the leaders off a broker one at the time. The fetcher may get a NotLeader error code for some partitions. In this case, it's less critical to remove those partitions from the fetcher since those partitions will be removed from the fetcher quickly by the leaderAndIsrRequests from the controller.

          My concern with managing the backoff at the partition level is that if the backoff is out of sync among the partitions, it may happen that different partitions become active at slightly different times and the fetcher doesn't actually back off. Also, the code becomes more complicated.

          So, my recommendation is the following.
          (1) Add the backoff config for the replica fetcher.
          (2) In AbstractFetcherThread, simply backoff based on the configured time, if it hits an exception when doing a fetch.
          (3) In order to shut down AbstractFetcherThread quickly, the backoff can be implemented on waiting on a new condition. We will signal that new condition during the shutdown.

          Show
          Jun Rao added a comment - Sriharsha Chintalapani , thanks for the patch. Managing the backoff per partition is a bit more complicated than I was expecting. The most common case that we want to handle here is that the fetcher is trying to fetch from a broker that's already down. In this case, the simplest approach is to just back off the fetcher (for all partitions) a bit. Another common case is that we are doing a controlled shutdown by moving the leaders off a broker one at the time. The fetcher may get a NotLeader error code for some partitions. In this case, it's less critical to remove those partitions from the fetcher since those partitions will be removed from the fetcher quickly by the leaderAndIsrRequests from the controller. My concern with managing the backoff at the partition level is that if the backoff is out of sync among the partitions, it may happen that different partitions become active at slightly different times and the fetcher doesn't actually back off. Also, the code becomes more complicated. So, my recommendation is the following. (1) Add the backoff config for the replica fetcher. (2) In AbstractFetcherThread, simply backoff based on the configured time, if it hits an exception when doing a fetch. (3) In order to shut down AbstractFetcherThread quickly, the backoff can be implemented on waiting on a new condition. We will signal that new condition during the shutdown.
          Hide
          Jun Rao added a comment -

          Sriharsha Chintalapani, since this affects the unit test performance (KAFKA-2010), would you have time to work on this in the next day or two? If not, I can pick it up.

          Show
          Jun Rao added a comment - Sriharsha Chintalapani , since this affects the unit test performance ( KAFKA-2010 ), would you have time to work on this in the next day or two? If not, I can pick it up.
          Hide
          Sriharsha Chintalapani added a comment -

          Jun Rao I'll work tomorrow and finish up the patch.
          Few questions on your recommendations
          " In AbstractFetcherThread, simply backoff based on the configured time, if it hits an exception when doing a fetch."
          so instead of handling partitions errors if there is an exception while fetching we will just backoff the AbstractFetcherThread wait until configured time elapsed or a condition is met?

          Show
          Sriharsha Chintalapani added a comment - Jun Rao I'll work tomorrow and finish up the patch. Few questions on your recommendations " In AbstractFetcherThread, simply backoff based on the configured time, if it hits an exception when doing a fetch." so instead of handling partitions errors if there is an exception while fetching we will just backoff the AbstractFetcherThread wait until configured time elapsed or a condition is met?
          Hide
          Jun Rao added a comment -

          Sriharsha Chintalapani, thanks. Your understanding is correct. We can probably just expose an awaitShutdown(timeout) method in ShutdownableThread and call it in AbstractFetcherThread.

          Show
          Jun Rao added a comment - Sriharsha Chintalapani , thanks. Your understanding is correct. We can probably just expose an awaitShutdown(timeout) method in ShutdownableThread and call it in AbstractFetcherThread.
          Hide
          Guozhang Wang added a comment -

          Jun Rao Could you elaborate a bit on "different partitions become active at slightly different times and the fetcher doesn't actually back off"? Not sure I understand why the fetcher does not actually back off.

          I agree that upon IOException thrown in SimpleConsumer.fetch, we should back off the thread as a whole for common case #1 you mentioned above; but at the same time we should still consider backing off for partition-specific error codes, as otherwise the broker logs will be kind of polluted with all error messages from continuous retries we have seen before. Do you agree?

          Show
          Guozhang Wang added a comment - Jun Rao Could you elaborate a bit on "different partitions become active at slightly different times and the fetcher doesn't actually back off"? Not sure I understand why the fetcher does not actually back off. I agree that upon IOException thrown in SimpleConsumer.fetch, we should back off the thread as a whole for common case #1 you mentioned above; but at the same time we should still consider backing off for partition-specific error codes, as otherwise the broker logs will be kind of polluted with all error messages from continuous retries we have seen before. Do you agree?
          Hide
          Jun Rao added a comment -

          Guozhang Wang, my concern is on the implementation of the DelayedItem. If you create a bunch of DelayedItems with the same timeout, they may timeout slightly differently since the calculation depends on the current time, which can change. In the second case when the leaders are moved one at time, what's going to happen is that the controller will tell the broker to move to the right leader right away. This typically happens within a few milli seconds. We could optimize this case, but I am not sure if it's worth the extra complexity in the code. In the first case, the remaining shutdown process could take seconds after the socket server is shut down. So backing off will definitely help.

          Perhaps we can just do a simple experiment with controlled shutdown and see how serious the issue is w/o backing off.

          Show
          Jun Rao added a comment - Guozhang Wang , my concern is on the implementation of the DelayedItem. If you create a bunch of DelayedItems with the same timeout, they may timeout slightly differently since the calculation depends on the current time, which can change. In the second case when the leaders are moved one at time, what's going to happen is that the controller will tell the broker to move to the right leader right away. This typically happens within a few milli seconds. We could optimize this case, but I am not sure if it's worth the extra complexity in the code. In the first case, the remaining shutdown process could take seconds after the socket server is shut down. So backing off will definitely help. Perhaps we can just do a simple experiment with controlled shutdown and see how serious the issue is w/o backing off.
          Hide
          Sriharsha Chintalapani added a comment -

          Created reviewboard https://reviews.apache.org/r/31927/diff/
          against branch origin/trunk

          Show
          Sriharsha Chintalapani added a comment - Created reviewboard https://reviews.apache.org/r/31927/diff/ against branch origin/trunk
          Hide
          Sriharsha Chintalapani added a comment -

          Jun Rao Guozhang Wang please take a look at the above patch . Let me know if that's what you have in mind. I also added "replica.fetch.backoff.ms" and "controller.socket.timeout.ms" to TestUtils.createBrokerConfig this reduced the total test run time from 15mins to under 10mins on my machine.

          Show
          Sriharsha Chintalapani added a comment - Jun Rao Guozhang Wang please take a look at the above patch . Let me know if that's what you have in mind. I also added "replica.fetch.backoff.ms" and "controller.socket.timeout.ms" to TestUtils.createBrokerConfig this reduced the total test run time from 15mins to under 10mins on my machine.
          Hide
          Sriharsha Chintalapani added a comment -

          Updated reviewboard https://reviews.apache.org/r/31927/diff/
          against branch origin/trunk

          Show
          Sriharsha Chintalapani added a comment - Updated reviewboard https://reviews.apache.org/r/31927/diff/ against branch origin/trunk
          Hide
          Jun Rao added a comment -

          Sriharsha Chintalapani and Guozhang Wang, thinking about this a bit more. There are really two types of states that we need to manage in AbstractFetcherThread. The first one is the connection state, i.e., if a connection breaks, we want to backoff the reconnection. The second one is the partition state, i.e., if the partition hits an exception, we want to backoff that particular partition a bit.

          The first one is what Sriharsha Chintalapani's current RB is addressing. How about let's complete this first since it affects the performance of the unit tests? Once that's committed, we can address the second one, which is in Sriharsha Chintalapani's initial patch.

          Show
          Jun Rao added a comment - Sriharsha Chintalapani and Guozhang Wang , thinking about this a bit more. There are really two types of states that we need to manage in AbstractFetcherThread. The first one is the connection state, i.e., if a connection breaks, we want to backoff the reconnection. The second one is the partition state, i.e., if the partition hits an exception, we want to backoff that particular partition a bit. The first one is what Sriharsha Chintalapani 's current RB is addressing. How about let's complete this first since it affects the performance of the unit tests? Once that's committed, we can address the second one, which is in Sriharsha Chintalapani 's initial patch.
          Hide
          Sriharsha Chintalapani added a comment -

          Updated reviewboard https://reviews.apache.org/r/31927/diff/
          against branch origin/trunk

          Show
          Sriharsha Chintalapani added a comment - Updated reviewboard https://reviews.apache.org/r/31927/diff/ against branch origin/trunk
          Hide
          Joe Stein added a comment -

          Shouldn't there be a KIP for this?

          Show
          Joe Stein added a comment - Shouldn't there be a KIP for this?
          Hide
          Sriharsha Chintalapani added a comment -

          Joe Stein since there aren't any interface changes I am not sure if a KIP is necessary. Ofcourse we added a new config for replica.fetch.backoff.ms If this warrants a KIP than I can write up one.

          Show
          Sriharsha Chintalapani added a comment - Joe Stein since there aren't any interface changes I am not sure if a KIP is necessary. Ofcourse we added a new config for replica.fetch.backoff.ms If this warrants a KIP than I can write up one.
          Hide
          Joe Stein added a comment -

          I personally think it is over kill but i bring it up because it seems to be required for other changes so I am just asking a question. If we are using the KIP to help folks understand the reason behind changes then we should do that and be complete or not.

          Show
          Joe Stein added a comment - I personally think it is over kill but i bring it up because it seems to be required for other changes so I am just asking a question. If we are using the KIP to help folks understand the reason behind changes then we should do that and be complete or not.
          Hide
          Jun Rao added a comment -

          Joe Stein, I don't think we need a KIP for this either. What other changes require this fix?

          Show
          Jun Rao added a comment - Joe Stein , I don't think we need a KIP for this either. What other changes require this fix?
          Hide
          Joe Stein added a comment -

          Ignore last comment, deleted... wrong JIRA

          Show
          Joe Stein added a comment - Ignore last comment, deleted... wrong JIRA
          Hide
          Jun Rao added a comment -

          Hmm, I don't think we removed or changed any config. Which ones are you referring to?

          Show
          Jun Rao added a comment - Hmm, I don't think we removed or changed any config. Which ones are you referring to?
          Hide
          Sriharsha Chintalapani added a comment -

          Updated reviewboard https://reviews.apache.org/r/31927/diff/
          against branch origin/trunk

          Show
          Sriharsha Chintalapani added a comment - Updated reviewboard https://reviews.apache.org/r/31927/diff/ against branch origin/trunk
          Hide
          Jun Rao added a comment -

          Sriharsha Chintalapani, thanks for the patch. +1 on RB 31927 after fixing the issue in my last comment. Committed to trunk. Now, unit test times went down to 8.5 mins from 12 mins.

          Show
          Jun Rao added a comment - Sriharsha Chintalapani , thanks for the patch. +1 on RB 31927 after fixing the issue in my last comment. Committed to trunk. Now, unit test times went down to 8.5 mins from 12 mins.
          Hide
          Jun Rao added a comment -

          Sriharsha Chintalapani, for the changes in RB 31366. We need to think through the case when all partitions are inactive. Presumably when that happens, we need to back off the fetching a bit.

          Show
          Jun Rao added a comment - Sriharsha Chintalapani , for the changes in RB 31366. We need to think through the case when all partitions are inactive. Presumably when that happens, we need to back off the fetching a bit.
          Hide
          Jun Rao added a comment -

          Also, in AbstractFetcherThread, we probably should use the configured backoff time, instead of the constant below.

          partitionMapCond.await(200L, TimeUnit.MILLISECONDS)

          Show
          Jun Rao added a comment - Also, in AbstractFetcherThread, we probably should use the configured backoff time, instead of the constant below. partitionMapCond.await(200L, TimeUnit.MILLISECONDS)
          Hide
          Sriharsha Chintalapani added a comment -

          Thanks Jun Rao I'll incorporate your and Guozhang Wang feedback for RB 31366. Yes we can add that condition at partitionMapCond.await(200L) and use the fetchBackoffMs . I'll send updated pr for it.

          Show
          Sriharsha Chintalapani added a comment - Thanks Jun Rao I'll incorporate your and Guozhang Wang feedback for RB 31366. Yes we can add that condition at partitionMapCond.await(200L) and use the fetchBackoffMs . I'll send updated pr for it.
          Hide
          Sriharsha Chintalapani added a comment -

          Updated reviewboard https://reviews.apache.org/r/31366/diff/
          against branch origin/trunk

          Show
          Sriharsha Chintalapani added a comment - Updated reviewboard https://reviews.apache.org/r/31366/diff/ against branch origin/trunk
          Hide
          Sriharsha Chintalapani added a comment -

          Guozhang Wang updated the PR as per your review suggestions. Please take a look when you get a chance.

          Show
          Sriharsha Chintalapani added a comment - Guozhang Wang updated the PR as per your review suggestions. Please take a look when you get a chance.
          Hide
          Guozhang Wang added a comment -

          Sorry for the delay, I will take a look at 31366 today.

          Show
          Guozhang Wang added a comment - Sorry for the delay, I will take a look at 31366 today.
          Hide
          Sriharsha Chintalapani added a comment -

          Guozhang Wang Thanks for the review. Can you please take a look at my reply to your comment.

          Show
          Sriharsha Chintalapani added a comment - Guozhang Wang Thanks for the review. Can you please take a look at my reply to your comment.
          Hide
          Sriharsha Chintalapani added a comment -

          Updated reviewboard https://reviews.apache.org/r/31366/diff/
          against branch origin/trunk

          Show
          Sriharsha Chintalapani added a comment - Updated reviewboard https://reviews.apache.org/r/31366/diff/ against branch origin/trunk
          Hide
          Sriharsha Chintalapani added a comment -

          Guozhang Wang addressed your last review. Please take a look. Thanks.

          Show
          Sriharsha Chintalapani added a comment - Guozhang Wang addressed your last review. Please take a look. Thanks.
          Hide
          Sriharsha Chintalapani added a comment -

          Updated reviewboard https://reviews.apache.org/r/31366/diff/
          against branch origin/trunk

          Show
          Sriharsha Chintalapani added a comment - Updated reviewboard https://reviews.apache.org/r/31366/diff/ against branch origin/trunk
          Hide
          Sriharsha Chintalapani added a comment -

          Updated reviewboard https://reviews.apache.org/r/31366/diff/
          against branch origin/trunk

          Show
          Sriharsha Chintalapani added a comment - Updated reviewboard https://reviews.apache.org/r/31366/diff/ against branch origin/trunk
          Hide
          Sriharsha Chintalapani added a comment -

          Updated reviewboard https://reviews.apache.org/r/31366/diff/
          against branch origin/trunk

          Show
          Sriharsha Chintalapani added a comment - Updated reviewboard https://reviews.apache.org/r/31366/diff/ against branch origin/trunk
          Hide
          Sriharsha Chintalapani added a comment -

          Updated reviewboard https://reviews.apache.org/r/31366/diff/
          against branch origin/trunk

          Show
          Sriharsha Chintalapani added a comment - Updated reviewboard https://reviews.apache.org/r/31366/diff/ against branch origin/trunk
          Hide
          Guozhang Wang added a comment -

          Thanks for the patch, committed to trunk.

          Show
          Guozhang Wang added a comment - Thanks for the patch, committed to trunk.

            People

            • Assignee:
              Sriharsha Chintalapani
              Reporter:
              Sam Meder
            • Votes:
              2 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development