Kafka
  1. Kafka
  2. KAFKA-903

[0.8.0 - windows] FATAL - [highwatermark-checkpoint-thread1] (Logging.scala:109) - Attempt to swap the new high watermark file with the old one failed

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Blocker Blocker
    • Resolution: Fixed
    • Affects Version/s: 0.8.0
    • Fix Version/s: 0.8.0
    • Component/s: core
    • Labels:
      None
    • Environment:

      Description

      This FATAL shuts down both brokers on windows,

      {2013-05-10 18:23:57,636}

      DEBUG [local-vat] (Logging.scala:51) - Sending 1
      messages with no compression to [robert_v_2x0,0]

      {2013-05-10 18:23:57,637}

      DEBUG [local-vat] (Logging.scala:51) - Producer
      sending messages with correlation id 178 for topics [robert_v_2x0,0] to
      broker 1 on 192.168.1.100:9093

      {2013-05-10 18:23:57,689}

      FATAL [highwatermark-checkpoint-thread1]
      (Logging.scala:109) - Attempt to swap the new high watermark file with the
      old one failed

      {2013-05-10 18:23:57,739}

      INFO [Thread-4] (Logging.scala:67) - [Kafka
      Server 0], shutting down

      Furthermore, attempts to restart them fail, with the following log:

      {2013-05-10 19:14:52,156}

      INFO [Thread-1] (Logging.scala:67) - [Kafka Server 0], started

      {2013-05-10 19:14:52,157}

      INFO [ZkClient-EventThread-32-localhost:2181] (Logging.scala:67) - New leader is 0

      {2013-05-10 19:14:52,193}

      DEBUG [ZkClient-EventThread-32-localhost:2181] (ZkEventThread.java:79) - Delivering event #1 done

      {2013-05-10 19:14:52,193}

      DEBUG [ZkClient-EventThread-32-localhost:2181] (ZkEventThread.java:69) - Delivering event #4 ZkEvent[Data of /controller_epoch changed sent to kafka.controller.ControllerEpochListener@5cb88f42]

      {2013-05-10 19:14:52,210}

      DEBUG [SyncThread:0] (FinalRequestProcessor.java:78) - Processing request:: sessionid:0x13e9127882e0001 type:exists cxid:0x1d zxid:0xfffffffffffffffe txntype:unknown reqpath:/controller_epoch

      {2013-05-10 19:14:52,210}

      DEBUG [SyncThread:0] (FinalRequestProcessor.java:160) - sessionid:0x13e9127882e0001 type:exists cxid:0x1d zxid:0xfffffffffffffffe txntype:unknown reqpath:/controller_epoch

      {2013-05-10 19:14:52,213}

      DEBUG [Thread-1-SendThread(localhost:2181)] (ClientCnxn.java:838) - Reading reply sessionid:0x13e9127882e0001, packet:: clientPath:null serverPath:null finished:false header:: 29,3 replyHeader:: 29,37,0 request:: '/controller_epoch,T response:: s

      {16,36,1368231712816,1368234889961,1,0,0,0,1,0,16}

      {2013-05-10 19:14:52,219}

      INFO [Thread-5] (Logging.scala:67) - [Kafka Server 0], shutting down

      1. kafka-903_v3.patch
        0.9 kB
        Jun Rao
      2. kafka_2.8.0-0.8.0-SNAPSHOT.jar
        2.26 MB
        Jun Rao
      3. kafka-903_v2.patch
        2 kB
        Jun Rao
      4. kafka-903.patch
        0.8 kB
        Jun Rao

        Issue Links

          Activity

          Hide
          Sean Glover added a comment - - edited

          I still see this exception on Windows 7 Pro SP1 with kafka 0.8.0 beta 1.

          [2013-11-26 13:20:29,374] FATAL Attempt to swap the new high watermark file with the old one failed (kafka.server.HighwaterMarkCheckpoint)
          [2013-11-26 13:20:29,391] INFO [Kafka Server 0], shutting down (kafka.server.KafkaServer)
          [2013-11-26 13:20:29,393] INFO [Socket Server on Broker 0], shutting down (kafka.network.SocketServer)
          [2013-11-26 13:20:29,399] INFO [Socket Server on Broker 0], shut down completely (kafka.network.SocketServer)
          [2013-11-26 13:20:29,401] INFO [Kafka Request Handler on Broker 0], shutting down (kafka.server.KafkaRequestHandlerPool)
          [2013-11-26 13:20:29,403] INFO [Kafka Request Handler on Broker 0], shutted down completely (kafka.server.KafkaRequestHandlerPool)
          [2013-11-26 13:20:29,405] INFO Shutdown Kafka scheduler (kafka.utils.KafkaScheduler)
          [2013-11-26 13:20:29,648] INFO Closing zookeeper client... (kafka.server.KafkaZooKeeper)
          [2013-11-26 13:20:29,660] INFO [Replica Manager on Broker 0]: Shut down (kafka.server.ReplicaManager)
          [2013-11-26 13:20:29,671] INFO [ReplicaFetcherManager on broker 0] shutting down (kafka.server.ReplicaFetcherManager)
          [2013-11-26 13:20:29,674] INFO [ReplicaFetcherManager on broker 0] shutdown completed (kafka.server.ReplicaFetcherManager)

          EDIT:

          I think it may have been due to the fact that I set log.dir in my kafka server.properties file to a relative directory: "..\\data
          kafka-logs". I did this just to keep everything related to kafka in one place. We only use kafka on windows on development machines.

          When I updated log.dir to an absolute path I have not seen the Exception.

          EDIT 3: Removed EDIT 2, irrelevant.

          Show
          Sean Glover added a comment - - edited I still see this exception on Windows 7 Pro SP1 with kafka 0.8.0 beta 1. [2013-11-26 13:20:29,374] FATAL Attempt to swap the new high watermark file with the old one failed (kafka.server.HighwaterMarkCheckpoint) [2013-11-26 13:20:29,391] INFO [Kafka Server 0] , shutting down (kafka.server.KafkaServer) [2013-11-26 13:20:29,393] INFO [Socket Server on Broker 0] , shutting down (kafka.network.SocketServer) [2013-11-26 13:20:29,399] INFO [Socket Server on Broker 0] , shut down completely (kafka.network.SocketServer) [2013-11-26 13:20:29,401] INFO [Kafka Request Handler on Broker 0] , shutting down (kafka.server.KafkaRequestHandlerPool) [2013-11-26 13:20:29,403] INFO [Kafka Request Handler on Broker 0] , shutted down completely (kafka.server.KafkaRequestHandlerPool) [2013-11-26 13:20:29,405] INFO Shutdown Kafka scheduler (kafka.utils.KafkaScheduler) [2013-11-26 13:20:29,648] INFO Closing zookeeper client... (kafka.server.KafkaZooKeeper) [2013-11-26 13:20:29,660] INFO [Replica Manager on Broker 0] : Shut down (kafka.server.ReplicaManager) [2013-11-26 13:20:29,671] INFO [ReplicaFetcherManager on broker 0] shutting down (kafka.server.ReplicaFetcherManager) [2013-11-26 13:20:29,674] INFO [ReplicaFetcherManager on broker 0] shutdown completed (kafka.server.ReplicaFetcherManager) EDIT: I think it may have been due to the fact that I set log.dir in my kafka server.properties file to a relative directory: "..\\data kafka-logs". I did this just to keep everything related to kafka in one place. We only use kafka on windows on development machines. When I updated log.dir to an absolute path I have not seen the Exception. EDIT 3: Removed EDIT 2, irrelevant.
          Jun Rao made changes -
          Link This issue duplicates KAFKA-876 [ KAFKA-876 ]
          Jun Rao made changes -
          Resolution Fixed [ 1 ]
          Status Patch Available [ 10002 ] Resolved [ 5 ]
          Assignee Jun Rao [ junrao ]
          Fix Version/s 0.8 [ 12317244 ]
          Hide
          Jun Rao added a comment -

          Thanks for the review. Committed v3 to 0.8.

          Show
          Jun Rao added a comment - Thanks for the review. Committed v3 to 0.8.
          Hide
          Jay Kreps added a comment -

          +1

          Show
          Jay Kreps added a comment - +1
          Hide
          Neha Narkhede added a comment -

          +1 on v3

          Show
          Neha Narkhede added a comment - +1 on v3
          Jun Rao made changes -
          Attachment kafka-903_v3.patch [ 12585154 ]
          Jun Rao made changes -
          Attachment kafka-903_v3.patch [ 12585155 ]
          Jun Rao made changes -
          Attachment kafka-903_v3.patch [ 12585154 ]
          Hide
          Jun Rao added a comment -

          Attach patch v3.

          To address Jay's concern, instead of using a generic renameTo util, only falls back to the non-atomic renameTo in checkpointing the high watermark file. Since both files are in the same dir and we control the naming, those other causes you listed that can fail renameTo won't happen. I didn't do the os level checking since I am not sure it that works well for environments like cygwin. We could guard this under a broker config parameter, but I am not sure if it's worth it.

          For Sriram's concern, this seems to be at least a problem for some versions of java on Windows since other projects like Hadoop (https://issues.apache.org/jira/browse/HADOOP-959) have also seen this before.

          Show
          Jun Rao added a comment - Attach patch v3. To address Jay's concern, instead of using a generic renameTo util, only falls back to the non-atomic renameTo in checkpointing the high watermark file. Since both files are in the same dir and we control the naming, those other causes you listed that can fail renameTo won't happen. I didn't do the os level checking since I am not sure it that works well for environments like cygwin. We could guard this under a broker config parameter, but I am not sure if it's worth it. For Sriram's concern, this seems to be at least a problem for some versions of java on Windows since other projects like Hadoop ( https://issues.apache.org/jira/browse/HADOOP-959 ) have also seen this before.
          Hide
          Timothy Chen added a comment -

          Do you know when this is going to be pushed to 0.8 branch?

          Show
          Timothy Chen added a comment - Do you know when this is going to be pushed to 0.8 branch?
          Hide
          Sriram Subramanian added a comment -

          There are two claims in this JIRA -

          1. "this may have to do with Windows not supporting file.renameTo() if the target file already exists"
          2. renameTo is not atomic in windows

          Claim 1 is wrong. MoveFileEx is the native api that helps you to rename an existing file. "MOVEFILE_REPLACE_EXISTING" is the flag you would use. This might be a bug in the java api or as the SO link indicates, does not work for non empty directories. (http://msdn.microsoft.com/en-us/library/windows/desktop/aa365240(v=vs.85).aspx)

          Claim 2 is possible depending on the OS settings. The caller of MoveFileEx is supposed to handle the failure.

          Show
          Sriram Subramanian added a comment - There are two claims in this JIRA - 1. "this may have to do with Windows not supporting file.renameTo() if the target file already exists" 2. renameTo is not atomic in windows Claim 1 is wrong. MoveFileEx is the native api that helps you to rename an existing file. "MOVEFILE_REPLACE_EXISTING" is the flag you would use. This might be a bug in the java api or as the SO link indicates, does not work for non empty directories. ( http://msdn.microsoft.com/en-us/library/windows/desktop/aa365240(v=vs.85).aspx ) Claim 2 is possible depending on the OS settings. The caller of MoveFileEx is supposed to handle the failure.
          Hide
          Jay Kreps added a comment -

          I don't think this rename functionality is such a good idea. There are many ways a rename can fail: (1) permissions, (2) disk errors, (3) bad target name, (4) rename is to another volume, etc. Java doesn't differentiate these, so in any of these cases we would then try to delete the file. Not sure this is a good idea.

          Another approach would be to add a Utils.IsWindows = System.getProperty("os.name").startsWith("Windows") and using this to fall back to the funky non-atomic behavior.

          Show
          Jay Kreps added a comment - I don't think this rename functionality is such a good idea. There are many ways a rename can fail: (1) permissions, (2) disk errors, (3) bad target name, (4) rename is to another volume, etc. Java doesn't differentiate these, so in any of these cases we would then try to delete the file. Not sure this is a good idea. Another approach would be to add a Utils.IsWindows = System.getProperty("os.name").startsWith("Windows") and using this to fall back to the funky non-atomic behavior.
          Hide
          Rob Withers added a comment -

          Jun, it looks like this works. I have 23 MB in the log file, on broker0. No replication and I guess I am only sending to partition 0. So, please close this issue.

          I am still having trouble with consumption, but given it is a weekend, I am busy with other stuff. I will dig in to it later and report my progress.

          Thanks for all your help,
          rob

          Show
          Rob Withers added a comment - Jun, it looks like this works. I have 23 MB in the log file, on broker0. No replication and I guess I am only sending to partition 0. So, please close this issue. I am still having trouble with consumption, but given it is a weekend, I am busy with other stuff. I will dig in to it later and report my progress. Thanks for all your help, rob
          Jun Rao made changes -
          Attachment kafka_2.8.0-0.8.0-SNAPSHOT.jar [ 12582803 ]
          Hide
          Jun Rao added a comment -

          Deleted the old jar and a attach a new jar built with patch v2.

          Show
          Jun Rao added a comment - Deleted the old jar and a attach a new jar built with patch v2.
          Jun Rao made changes -
          Attachment kafka_2.8.0-0.8.0-SNAPSHOT.jar [ 12582791 ]
          Jun Rao made changes -
          Attachment kafka-903_v2.patch [ 12582802 ]
          Hide
          Jun Rao added a comment -

          Attach patch v2. If a file can't be renamed, it deletes the target file first. The implication is that during a hard crash, if the high watermark file is missing, one has to manually rename it from the temporary high watermark file.

          Show
          Jun Rao added a comment - Attach patch v2. If a file can't be renamed, it deletes the target file first. The implication is that during a hard crash, if the high watermark file is missing, one has to manually rename it from the temporary high watermark file.
          Hide
          Rob Withers added a comment -

          I am on jdk 7.0.17, That thread mentions apache.commons.io.FileUtils.moveFile(). Also, my suggestion to be done with the issue: I know there is a way to detect platform, just write a small strategy pattern for the general platform and for windows and plug her in.

          Show
          Rob Withers added a comment - I am on jdk 7.0.17, That thread mentions apache.commons.io.FileUtils.moveFile(). Also, my suggestion to be done with the issue: I know there is a way to detect platform, just write a small strategy pattern for the general platform and for windows and plug her in.
          Hide
          Jun Rao added a comment -

          Hmm, this may have to do with Windows not supporting file.renameTo() if the target file already exists (http://stackoverflow.com/questions/1000183/reliable-file-renameto-alternative-on-windows). We periodically checkpoint high watermarks to disk. The way we do this is to first write all values to a tmp file replication-offset-checkpoint.tmp and then rename the tmp file to replication-offset-checkpoint. This way, if there is any I/O error during checkpointing, we still have the old checkpoint file for use. Not sure what the best way to do this in Windows.

          Could you try java 7 and see if it still has the same issue (the above link suggests that it's fixed in jdk 7)?

          Show
          Jun Rao added a comment - Hmm, this may have to do with Windows not supporting file.renameTo() if the target file already exists ( http://stackoverflow.com/questions/1000183/reliable-file-renameto-alternative-on-windows ). We periodically checkpoint high watermarks to disk. The way we do this is to first write all values to a tmp file replication-offset-checkpoint.tmp and then rename the tmp file to replication-offset-checkpoint. This way, if there is any I/O error during checkpointing, we still have the old checkpoint file for use. Not sure what the best way to do this in Windows. Could you try java 7 and see if it still has the same issue (the above link suggests that it's fixed in jdk 7)?
          Hide
          Rob Withers added a comment -

          It still fails. Where should I look in the jar to ensure the patch is in there? Which class method? Also, could you generate a jar for me with sources?

          thanks a lot and happy saturday!

          {2013-05-11 10:00:58,630}

          DEBUG [local-vat] (Logging.scala:51) - Producer sent messages with correlation id 232 for topics [robert_v_2x0,0] to broker 0 on 192.168.1.100:9092

          {2013-05-11 10:00:58,637}

          DEBUG [local-vat] (Logging.scala:51) - Getting broker partition info for topic robert_v_2x0

          {2013-05-11 10:00:58,638}

          DEBUG [local-vat] (Logging.scala:51) - Partition [robert_v_2x0,0] has leader 0

          {2013-05-11 10:00:58,639}

          DEBUG [local-vat] (Logging.scala:51) - Broker partitions registered for topic: robert_v_2x0 are 0

          {2013-05-11 10:00:58,639}

          DEBUG [local-vat] (Logging.scala:51) - Sending 1 messages with no compression to [robert_v_2x0,0]

          {2013-05-11 10:00:58,640}

          DEBUG [local-vat] (Logging.scala:51) - Producer sending messages with correlation id 234 for topics [robert_v_2x0,0] to broker 0 on 192.168.1.100:9092

          {2013-05-11 10:00:58,648}

          DEBUG [kafka-request-handler-1] (Logging.scala:51) - [Kafka Request Handler 1 on Broker 0], handles request Request(1,sun.nio.ch.SelectionKeyImpl@2b12cf49,null,1368288058646,/192.168.1.100:60622)

          {2013-05-11 10:00:58,649}

          DEBUG [kafka-request-handler-1] (Logging.scala:51) - Adding index entry 115 => 5220900 to 00000000000000000000.index.

          {2013-05-11 10:00:58,650}

          DEBUG [kafka-request-handler-1] (Logging.scala:51) - Partition [robert_v_2x0,0] on broker 0: Highwatermark for partition [robert_v_2x0,0] updated to 116

          {2013-05-11 10:00:58,650}

          DEBUG [kafka-request-handler-1] (Logging.scala:51) - [KafkaApi-0] Produce to local log in 2 ms

          {2013-05-11 10:00:58,656}

          DEBUG [local-vat] (Logging.scala:51) - Producer sent messages with correlation id 234 for topics [robert_v_2x0,0] to broker 0 on 192.168.1.100:9092

          {2013-05-11 10:00:58,666}

          DEBUG [local-vat] (Logging.scala:51) - Getting broker partition info for topic robert_v_2x0

          {2013-05-11 10:00:58,667}

          DEBUG [local-vat] (Logging.scala:51) - Partition [robert_v_2x0,0] has leader 0

          {2013-05-11 10:00:58,667}

          DEBUG [local-vat] (Logging.scala:51) - Broker partitions registered for topic: robert_v_2x0 are 0

          {2013-05-11 10:00:58,669}

          DEBUG [local-vat] (Logging.scala:51) - Sending 1 messages with no compression to [robert_v_2x0,0]

          {2013-05-11 10:00:58,670}

          DEBUG [local-vat] (Logging.scala:51) - Producer sending messages with correlation id 236 for topics [robert_v_2x0,0] to broker 0 on 192.168.1.100:9092

          {2013-05-11 10:00:58,716}

          DEBUG [kafka-request-handler-0] (Logging.scala:51) - [Kafka Request Handler 0 on Broker 0], handles request Request(1,sun.nio.ch.SelectionKeyImpl@2b12cf49,null,1368288058674,/192.168.1.100:60622)

          {2013-05-11 10:00:58,717}

          DEBUG [kafka-request-handler-0] (Logging.scala:51) - Adding index entry 116 => 5232005 to 00000000000000000000.index.

          {2013-05-11 10:00:58,718}

          DEBUG [kafka-request-handler-0] (Logging.scala:51) - Partition [robert_v_2x0,0] on broker 0: Highwatermark for partition [robert_v_2x0,0] updated to 117

          {2013-05-11 10:00:58,718}

          DEBUG [kafka-request-handler-0] (Logging.scala:51) - [KafkaApi-0] Produce to local log in 2 ms

          {2013-05-11 10:00:58,726}

          FATAL [highwatermark-checkpoint-thread1] (Logging.scala:109) - Attempt to swap the new high watermark file with the old one failed

          Show
          Rob Withers added a comment - It still fails. Where should I look in the jar to ensure the patch is in there? Which class method? Also, could you generate a jar for me with sources? thanks a lot and happy saturday! {2013-05-11 10:00:58,630} DEBUG [local-vat] (Logging.scala:51) - Producer sent messages with correlation id 232 for topics [robert_v_2x0,0] to broker 0 on 192.168.1.100:9092 {2013-05-11 10:00:58,637} DEBUG [local-vat] (Logging.scala:51) - Getting broker partition info for topic robert_v_2x0 {2013-05-11 10:00:58,638} DEBUG [local-vat] (Logging.scala:51) - Partition [robert_v_2x0,0] has leader 0 {2013-05-11 10:00:58,639} DEBUG [local-vat] (Logging.scala:51) - Broker partitions registered for topic: robert_v_2x0 are 0 {2013-05-11 10:00:58,639} DEBUG [local-vat] (Logging.scala:51) - Sending 1 messages with no compression to [robert_v_2x0,0] {2013-05-11 10:00:58,640} DEBUG [local-vat] (Logging.scala:51) - Producer sending messages with correlation id 234 for topics [robert_v_2x0,0] to broker 0 on 192.168.1.100:9092 {2013-05-11 10:00:58,648} DEBUG [kafka-request-handler-1] (Logging.scala:51) - [Kafka Request Handler 1 on Broker 0] , handles request Request(1,sun.nio.ch.SelectionKeyImpl@2b12cf49,null,1368288058646,/192.168.1.100:60622) {2013-05-11 10:00:58,649} DEBUG [kafka-request-handler-1] (Logging.scala:51) - Adding index entry 115 => 5220900 to 00000000000000000000.index. {2013-05-11 10:00:58,650} DEBUG [kafka-request-handler-1] (Logging.scala:51) - Partition [robert_v_2x0,0] on broker 0: Highwatermark for partition [robert_v_2x0,0] updated to 116 {2013-05-11 10:00:58,650} DEBUG [kafka-request-handler-1] (Logging.scala:51) - [KafkaApi-0] Produce to local log in 2 ms {2013-05-11 10:00:58,656} DEBUG [local-vat] (Logging.scala:51) - Producer sent messages with correlation id 234 for topics [robert_v_2x0,0] to broker 0 on 192.168.1.100:9092 {2013-05-11 10:00:58,666} DEBUG [local-vat] (Logging.scala:51) - Getting broker partition info for topic robert_v_2x0 {2013-05-11 10:00:58,667} DEBUG [local-vat] (Logging.scala:51) - Partition [robert_v_2x0,0] has leader 0 {2013-05-11 10:00:58,667} DEBUG [local-vat] (Logging.scala:51) - Broker partitions registered for topic: robert_v_2x0 are 0 {2013-05-11 10:00:58,669} DEBUG [local-vat] (Logging.scala:51) - Sending 1 messages with no compression to [robert_v_2x0,0] {2013-05-11 10:00:58,670} DEBUG [local-vat] (Logging.scala:51) - Producer sending messages with correlation id 236 for topics [robert_v_2x0,0] to broker 0 on 192.168.1.100:9092 {2013-05-11 10:00:58,716} DEBUG [kafka-request-handler-0] (Logging.scala:51) - [Kafka Request Handler 0 on Broker 0] , handles request Request(1,sun.nio.ch.SelectionKeyImpl@2b12cf49,null,1368288058674,/192.168.1.100:60622) {2013-05-11 10:00:58,717} DEBUG [kafka-request-handler-0] (Logging.scala:51) - Adding index entry 116 => 5232005 to 00000000000000000000.index. {2013-05-11 10:00:58,718} DEBUG [kafka-request-handler-0] (Logging.scala:51) - Partition [robert_v_2x0,0] on broker 0: Highwatermark for partition [robert_v_2x0,0] updated to 117 {2013-05-11 10:00:58,718} DEBUG [kafka-request-handler-0] (Logging.scala:51) - [KafkaApi-0] Produce to local log in 2 ms {2013-05-11 10:00:58,726} FATAL [highwatermark-checkpoint-thread1] (Logging.scala:109) - Attempt to swap the new high watermark file with the old one failed
          Jun Rao made changes -
          Attachment kafka_2.8.0-0.8.0-SNAPSHOT.jar [ 12582791 ]
          Hide
          Jun Rao added a comment -

          Attach the right jar this time.

          Show
          Jun Rao added a comment - Attach the right jar this time.
          Jun Rao made changes -
          Attachment kafka_2.8.0-0.8.0-SNAPSHOT.jar [ 12582772 ]
          Hide
          Rob Withers added a comment - - edited

          That jar is missing a lot - it only has a MANIFEST. Should I be able to do anything with it?

          Show
          Rob Withers added a comment - - edited That jar is missing a lot - it only has a MANIFEST. Should I be able to do anything with it?
          Jun Rao made changes -
          Attachment kafka_2.8.0-0.8.0-SNAPSHOT.jar [ 12582772 ]
          Hide
          Jun Rao added a comment -

          Attach the kafka jar with the patch.

          Show
          Jun Rao added a comment - Attach the kafka jar with the patch.
          Hide
          Rob Withers added a comment -

          I'd love too. Could you build me a jar and email it too me, please?

          Show
          Rob Withers added a comment - I'd love too. Could you build me a jar and email it too me, please?
          Jun Rao made changes -
          Status Open [ 1 ] Patch Available [ 10002 ]
          Jun Rao made changes -
          Field Original Value New Value
          Attachment kafka-903.patch [ 12582769 ]
          Hide
          Jun Rao added a comment -

          Attach a patch. Rob, could you give it a try?

          Show
          Jun Rao added a comment - Attach a patch. Rob, could you give it a try?
          Rob Withers created issue -

            People

            • Assignee:
              Jun Rao
              Reporter:
              Rob Withers
            • Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development