[jira] [Commented] (KAFKA-1112) broker can not start itself after kafka is killed with -9
[ https://issues.apache.org/jira/browse/KAFKA-1112?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13825487#comment-13825487 ] Jun Rao commented on KAFKA-1112: Thinking about this a bit more. The end state is that we want to only recover the portion of the log segment from the recovery point, instead of recovering the whole log segment. The dilemma is that we are not sure what portion of the index is valid. Scanning from the beginning of the log segment defeats the purpose of incremental recovery. One possible solution is to checkpoint an index recovery point, in addition to the recovery offset per log. The index recovery point is the # of valid index entries in the segment to which the recovery offset belongs. This way, on startup, we will be sure that the data in the last valid index entry is not corrupted and we can use it to quickly locate the recovery offset in the log file. > broker can not start itself after kafka is killed with -9 > - > > Key: KAFKA-1112 > URL: https://issues.apache.org/jira/browse/KAFKA-1112 > Project: Kafka > Issue Type: Bug > Components: log >Affects Versions: 0.8, 0.8.1 >Reporter: Kane Kim >Assignee: Jay Kreps >Priority: Critical > Attachments: KAFKA-1112-v1.patch, KAFKA-1112-v2.patch, > KAFKA-1112-v3.patch, KAFKA-1112.out > > > When I kill kafka with -9, broker cannot start itself because of corrupted > index logs. I think kafka should try to delete/rebuild indexes itself without > manual intervention. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (KAFKA-1134) onControllerFailover function should be synchronized with other functions
[ https://issues.apache.org/jira/browse/KAFKA-1134?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1134: - Affects Version/s: 0.8.1 0.8 > onControllerFailover function should be synchronized with other functions > - > > Key: KAFKA-1134 > URL: https://issues.apache.org/jira/browse/KAFKA-1134 > Project: Kafka > Issue Type: Bug >Affects Versions: 0.8, 0.8.1 >Reporter: Guozhang Wang > > Otherwise race conditions could happen. For example, handleNewSession will > close all sockets with brokers while the handleStateChange in > onControllerFailover tries to send requests to them. -- This message was sent by Atlassian JIRA (v6.1#6144)
Re: Review Request 15201: address Neha's review comments
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/15201/ --- (Updated Nov. 18, 2013, 5:57 p.m.) Review request for kafka. Summary (updated) - address Neha's review comments Bugs: KAFKA-1117 https://issues.apache.org/jira/browse/KAFKA-1117 Repository: kafka Description (updated) --- kafka-1117; fix 5 kafka-1117; fix 4 kafka-1117; fix 3 kafka-1117; fix 2 kafka-1117; fix 1 kafka-1117 Diffs (updated) - core/src/main/scala/kafka/api/OffsetResponse.scala 08dc3cd3d166efba6b2b43f6e148f636b175affe core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala PRE-CREATION Diff: https://reviews.apache.org/r/15201/diff/ Testing --- Thanks, Jun Rao
[jira] [Commented] (KAFKA-1117) tool for checking the consistency among replicas
[ https://issues.apache.org/jira/browse/KAFKA-1117?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13825532#comment-13825532 ] Jun Rao commented on KAFKA-1117: Updated reviewboard https://reviews.apache.org/r/15201/ against branch origin/trunk > tool for checking the consistency among replicas > > > Key: KAFKA-1117 > URL: https://issues.apache.org/jira/browse/KAFKA-1117 > Project: Kafka > Issue Type: New Feature > Components: core >Affects Versions: 0.8.1 >Reporter: Jun Rao >Assignee: Jun Rao > Fix For: 0.8.1 > > Attachments: KAFKA-1117.patch, KAFKA-1117_2013-11-11_08:44:25.patch, > KAFKA-1117_2013-11-12_08:34:53.patch, KAFKA-1117_2013-11-14_08:24:41.patch, > KAFKA-1117_2013-11-18_09:58:23.patch > > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (KAFKA-1117) tool for checking the consistency among replicas
[ https://issues.apache.org/jira/browse/KAFKA-1117?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1117: --- Attachment: KAFKA-1117_2013-11-18_09:58:23.patch > tool for checking the consistency among replicas > > > Key: KAFKA-1117 > URL: https://issues.apache.org/jira/browse/KAFKA-1117 > Project: Kafka > Issue Type: New Feature > Components: core >Affects Versions: 0.8.1 >Reporter: Jun Rao >Assignee: Jun Rao > Fix For: 0.8.1 > > Attachments: KAFKA-1117.patch, KAFKA-1117_2013-11-11_08:44:25.patch, > KAFKA-1117_2013-11-12_08:34:53.patch, KAFKA-1117_2013-11-14_08:24:41.patch, > KAFKA-1117_2013-11-18_09:58:23.patch > > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (KAFKA-1112) broker can not start itself after kafka is killed with -9
[ https://issues.apache.org/jira/browse/KAFKA-1112?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13825588#comment-13825588 ] Guozhang Wang commented on KAFKA-1112: -- Did some research on network about "fsync", and it seems fsync can be reliable even with disk's block-write behavior since it is sequential, which means even file system crashed during fsync we will not expect random behavior. > broker can not start itself after kafka is killed with -9 > - > > Key: KAFKA-1112 > URL: https://issues.apache.org/jira/browse/KAFKA-1112 > Project: Kafka > Issue Type: Bug > Components: log >Affects Versions: 0.8, 0.8.1 >Reporter: Kane Kim >Assignee: Jay Kreps >Priority: Critical > Attachments: KAFKA-1112-v1.patch, KAFKA-1112-v2.patch, > KAFKA-1112-v3.patch, KAFKA-1112.out > > > When I kill kafka with -9, broker cannot start itself because of corrupted > index logs. I think kafka should try to delete/rebuild indexes itself without > manual intervention. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (KAFKA-1112) broker can not start itself after kafka is killed with -9
[ https://issues.apache.org/jira/browse/KAFKA-1112?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13825620#comment-13825620 ] Neha Narkhede commented on KAFKA-1112: -- [~junrao] This approach seems reasonable unless I'm missing any caveats in Log. [~jkreps] what do you think? > broker can not start itself after kafka is killed with -9 > - > > Key: KAFKA-1112 > URL: https://issues.apache.org/jira/browse/KAFKA-1112 > Project: Kafka > Issue Type: Bug > Components: log >Affects Versions: 0.8, 0.8.1 >Reporter: Kane Kim >Assignee: Jay Kreps >Priority: Critical > Attachments: KAFKA-1112-v1.patch, KAFKA-1112-v2.patch, > KAFKA-1112-v3.patch, KAFKA-1112.out > > > When I kill kafka with -9, broker cannot start itself because of corrupted > index logs. I think kafka should try to delete/rebuild indexes itself without > manual intervention. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] Subscription: outstanding kafka patches
Issue Subscription Filter: outstanding kafka patches (75 issues) The list of outstanding kafka patches Subscriber: kafka-mailing-list Key Summary KAFKA-1131 copy some more files into the release tar and zip that are needed/desired https://issues.apache.org/jira/browse/KAFKA-1131 KAFKA-1130 "log.dirs" is a confusing property name https://issues.apache.org/jira/browse/KAFKA-1130 KAFKA-1127 kafka and zookeeper server should start in daemon mode and log to correct position https://issues.apache.org/jira/browse/KAFKA-1127 KAFKA-1117 tool for checking the consistency among replicas https://issues.apache.org/jira/browse/KAFKA-1117 KAFKA-1116 Need to upgrade sbt-assembly to compile on scala 2.10.2 https://issues.apache.org/jira/browse/KAFKA-1116 KAFKA-1112 broker can not start itself after kafka is killed with -9 https://issues.apache.org/jira/browse/KAFKA-1112 KAFKA-1110 Unable to produce messages with snappy/gzip compression https://issues.apache.org/jira/browse/KAFKA-1110 KAFKA-1109 Need to fix GC log configuration code, not able to override KAFKA_GC_LOG_OPTS https://issues.apache.org/jira/browse/KAFKA-1109 KAFKA-1106 HighwaterMarkCheckpoint failure puting broker into a bad state https://issues.apache.org/jira/browse/KAFKA-1106 KAFKA-1093 Log.getOffsetsBefore(t, …) does not return the last confirmed offset before t https://issues.apache.org/jira/browse/KAFKA-1093 KAFKA-1086 Improve GetOffsetShell to find metadata automatically https://issues.apache.org/jira/browse/KAFKA-1086 KAFKA-1082 zkclient dies after UnknownHostException in zk reconnect https://issues.apache.org/jira/browse/KAFKA-1082 KAFKA-1079 Liars in PrimitiveApiTest that promise to test api in compression mode, but don't do this actually https://issues.apache.org/jira/browse/KAFKA-1079 KAFKA-1049 Encoder implementations are required to provide an undocumented constructor. https://issues.apache.org/jira/browse/KAFKA-1049 KAFKA-1032 Messages sent to the old leader will be lost on broker GC resulted failure https://issues.apache.org/jira/browse/KAFKA-1032 KAFKA-1020 Remove getAllReplicasOnBroker from KafkaController https://issues.apache.org/jira/browse/KAFKA-1020 KAFKA-1012 Implement an Offset Manager and hook offset requests to it https://issues.apache.org/jira/browse/KAFKA-1012 KAFKA-1011 Decompression and re-compression on MirrorMaker could result in messages being dropped in the pipeline https://issues.apache.org/jira/browse/KAFKA-1011 KAFKA-1005 kafka.perf.ConsumerPerformance not shutting down consumer https://issues.apache.org/jira/browse/KAFKA-1005 KAFKA-1004 Handle topic event for trivial whitelist topic filters https://issues.apache.org/jira/browse/KAFKA-1004 KAFKA-998 Producer should not retry on non-recoverable error codes https://issues.apache.org/jira/browse/KAFKA-998 KAFKA-997 Provide a strict verification mode when reading configuration properties https://issues.apache.org/jira/browse/KAFKA-997 KAFKA-996 Capitalize first letter for log entries https://issues.apache.org/jira/browse/KAFKA-996 KAFKA-984 Avoid a full rebalance in cases when a new topic is discovered but container/broker set stay the same https://issues.apache.org/jira/browse/KAFKA-984 KAFKA-976 Order-Preserving Mirror Maker Testcase https://issues.apache.org/jira/browse/KAFKA-976 KAFKA-967 Use key range in ProducerPerformance https://issues.apache.org/jira/browse/KAFKA-967 KAFKA-917 Expose zk.session.timeout.ms in console consumer https://issues.apache.org/jira/browse/KAFKA-917 KAFKA-885 sbt package builds two kafka jars https://issues.apache.org/jira/browse/KAFKA-885 KAFKA-881 Kafka broker not respecting log.roll.hours https://issues.apache.org/jira/browse/KAFKA-881 KAFKA-873 Consider replacing zkclient with curator (with zkclient-bridge) https://issues.apache.org/jira/browse/KAFKA-873 KAFKA-868 System Test - add test case for rolling controlled shutdown https://issues.apache.org/jira/browse/KAFKA-868 KAFKA-863 System Test - update 0.7 version of kafka-run-class.sh for Migration Tool test cases https://issues.apache.org/jira/browse/KAFKA-863 KAFKA-859 support basic auth protection of mx4j console https://issues.apache.org/jira/browse/KAFKA-859 KAFKA-855 Ant+Ivy build for Kafka https://issues.apache.org/jira/browse/KAFKA-855 KAFKA-854 Upgrade dependencies for 0.8 https://issues.apache.org/jira/browse/KAFKA-854 KAFKA-815 Improve SimpleConsumerShell to take in a max messages config option https://issues.apache.org/jira/browse/KAFKA-815 KAFKA-745 Remove getShutdownReceive() and othe
Re: Review Request 15201: address Neha's review comments
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/15201/#review29067 --- Ship it! Ship It! - Neha Narkhede On Nov. 18, 2013, 5:57 p.m., Jun Rao wrote: > > --- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/15201/ > --- > > (Updated Nov. 18, 2013, 5:57 p.m.) > > > Review request for kafka. > > > Bugs: KAFKA-1117 > https://issues.apache.org/jira/browse/KAFKA-1117 > > > Repository: kafka > > > Description > --- > > kafka-1117; fix 5 > > > kafka-1117; fix 4 > > > kafka-1117; fix 3 > > > kafka-1117; fix 2 > > > kafka-1117; fix 1 > > > kafka-1117 > > > Diffs > - > > core/src/main/scala/kafka/api/OffsetResponse.scala > 08dc3cd3d166efba6b2b43f6e148f636b175affe > core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala PRE-CREATION > > Diff: https://reviews.apache.org/r/15201/diff/ > > > Testing > --- > > > Thanks, > > Jun Rao > >
[jira] [Commented] (KAFKA-1112) broker can not start itself after kafka is killed with -9
[ https://issues.apache.org/jira/browse/KAFKA-1112?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13825708#comment-13825708 ] Jay Kreps commented on KAFKA-1112: -- Yeah at a high-level there are a couple of things we could do: 1. Non-incremental a. Harden the current approach (what the attached patches do) b. Use the clean shutdown file 2. Implement incremental recovery (what Jun is proposing) All of these are good. 1a is implemented, but is arguably gross. I am open to 1b or 2 or a short-term/long-term thing. For 2 I think the details to figure out would be 1. OffsetCheckpoint is shared so adding the position to that file will impact other use cases how will that be handled? 2. I suspect that if we want to move to positions we should do something like (file, log_position, index_position) rather than a mixture of logical and physical. 3. We need to ensure that log compaction is thought through. This could cause the physical position to change. That could be fine but we need to reason through it. 4. We need to ensure that we handle truncation which implies that a position X could be stable, then deleted, then rewritten differently without flush. This may be fine we just have to think it through. > broker can not start itself after kafka is killed with -9 > - > > Key: KAFKA-1112 > URL: https://issues.apache.org/jira/browse/KAFKA-1112 > Project: Kafka > Issue Type: Bug > Components: log >Affects Versions: 0.8, 0.8.1 >Reporter: Kane Kim >Assignee: Jay Kreps >Priority: Critical > Attachments: KAFKA-1112-v1.patch, KAFKA-1112-v2.patch, > KAFKA-1112-v3.patch, KAFKA-1112.out > > > When I kill kafka with -9, broker cannot start itself because of corrupted > index logs. I think kafka should try to delete/rebuild indexes itself without > manual intervention. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (KAFKA-1112) broker can not start itself after kafka is killed with -9
[ https://issues.apache.org/jira/browse/KAFKA-1112?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1112: --- Attachment: KAFKA-1112-v4.patch Ok, so it seems that the end state is not that simple and may need some more thoughts. I took patch v3 , removed the recovery part in LogSegment and replaced it with the simpler approach using the clean shutdown file. > broker can not start itself after kafka is killed with -9 > - > > Key: KAFKA-1112 > URL: https://issues.apache.org/jira/browse/KAFKA-1112 > Project: Kafka > Issue Type: Bug > Components: log >Affects Versions: 0.8, 0.8.1 >Reporter: Kane Kim >Assignee: Jay Kreps >Priority: Critical > Attachments: KAFKA-1112-v1.patch, KAFKA-1112-v2.patch, > KAFKA-1112-v3.patch, KAFKA-1112-v4.patch, KAFKA-1112.out > > > When I kill kafka with -9, broker cannot start itself because of corrupted > index logs. I think kafka should try to delete/rebuild indexes itself without > manual intervention. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (KAFKA-1103) Consumer uses two zkclients
[ https://issues.apache.org/jira/browse/KAFKA-1103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13825863#comment-13825863 ] Guozhang Wang commented on KAFKA-1103: -- Since this is closely related to KAFKA-1004, will merge the two here. > Consumer uses two zkclients > --- > > Key: KAFKA-1103 > URL: https://issues.apache.org/jira/browse/KAFKA-1103 > Project: Kafka > Issue Type: Bug >Reporter: Joel Koshy >Assignee: Guozhang Wang > Fix For: 0.8.1 > > > .. which is very confusing when debugging consumer logs. I don't remember any > good reason for this, and we should get rid of the one instantiated in > ZookeeperTopicEventWatcher if possible. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (KAFKA-1112) broker can not start itself after kafka is killed with -9
[ https://issues.apache.org/jira/browse/KAFKA-1112?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13825893#comment-13825893 ] Neha Narkhede commented on KAFKA-1112: -- Thanks for the patch, Jun! Overall, looks good (+1). Few minor comments that you can address on checkin - 1. Log - okay we need to actually recovery this log => okay we need to actually recover this log 2. OffsetIndex - In sanityCheck(), in one error message, we print the index file's absolute path and in another, we print only the name. Can we standardize on one? It is better to print the entire path since we can have more than one data directories. > broker can not start itself after kafka is killed with -9 > - > > Key: KAFKA-1112 > URL: https://issues.apache.org/jira/browse/KAFKA-1112 > Project: Kafka > Issue Type: Bug > Components: log >Affects Versions: 0.8, 0.8.1 >Reporter: Kane Kim >Assignee: Jay Kreps >Priority: Critical > Attachments: KAFKA-1112-v1.patch, KAFKA-1112-v2.patch, > KAFKA-1112-v3.patch, KAFKA-1112-v4.patch, KAFKA-1112.out > > > When I kill kafka with -9, broker cannot start itself because of corrupted > index logs. I think kafka should try to delete/rebuild indexes itself without > manual intervention. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (KAFKA-1112) broker can not start itself after kafka is killed with -9
[ https://issues.apache.org/jira/browse/KAFKA-1112?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13825922#comment-13825922 ] Jay Kreps commented on KAFKA-1112: -- +1 lgtm. > broker can not start itself after kafka is killed with -9 > - > > Key: KAFKA-1112 > URL: https://issues.apache.org/jira/browse/KAFKA-1112 > Project: Kafka > Issue Type: Bug > Components: log >Affects Versions: 0.8, 0.8.1 >Reporter: Kane Kim >Assignee: Jay Kreps >Priority: Critical > Attachments: KAFKA-1112-v1.patch, KAFKA-1112-v2.patch, > KAFKA-1112-v3.patch, KAFKA-1112-v4.patch, KAFKA-1112.out > > > When I kill kafka with -9, broker cannot start itself because of corrupted > index logs. I think kafka should try to delete/rebuild indexes itself without > manual intervention. -- This message was sent by Atlassian JIRA (v6.1#6144)
Review Request 15659: Patch for KAFKA-1103
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/15659/ --- Review request for kafka. Bugs: KAFKA-1103 https://issues.apache.org/jira/browse/KAFKA-1103 Repository: kafka Description --- KAFKA-1103.v1 Diffs - core/src/main/scala/kafka/consumer/TopicFilter.scala cf3853b223095e1fe0921175c407a906828b8113 core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala c0350cd05cf1b59866a1fedccbeb700b3e828d44 core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala a67c193df9f7cbfc52f75dc1b71dc017de1b5fe2 core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala 40a2bf7a9277eb5f94bc07b40d7726d81860cefc Diff: https://reviews.apache.org/r/15659/diff/ Testing --- Thanks, Guozhang Wang
[jira] [Updated] (KAFKA-1103) Consumer uses two zkclients
[ https://issues.apache.org/jira/browse/KAFKA-1103?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1103: - Attachment: KAFKA-1103.patch > Consumer uses two zkclients > --- > > Key: KAFKA-1103 > URL: https://issues.apache.org/jira/browse/KAFKA-1103 > Project: Kafka > Issue Type: Bug >Reporter: Joel Koshy >Assignee: Guozhang Wang > Fix For: 0.8.1 > > Attachments: KAFKA-1103.patch > > > .. which is very confusing when debugging consumer logs. I don't remember any > good reason for this, and we should get rid of the one instantiated in > ZookeeperTopicEventWatcher if possible. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (KAFKA-1103) Consumer uses two zkclients
[ https://issues.apache.org/jira/browse/KAFKA-1103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13825942#comment-13825942 ] Guozhang Wang commented on KAFKA-1103: -- Created reviewboard https://reviews.apache.org/r/15659/ against branch origin/trunk > Consumer uses two zkclients > --- > > Key: KAFKA-1103 > URL: https://issues.apache.org/jira/browse/KAFKA-1103 > Project: Kafka > Issue Type: Bug >Reporter: Joel Koshy >Assignee: Guozhang Wang > Fix For: 0.8.1 > > Attachments: KAFKA-1103.patch > > > .. which is very confusing when debugging consumer logs. I don't remember any > good reason for this, and we should get rid of the one instantiated in > ZookeeperTopicEventWatcher if possible. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Resolved] (KAFKA-1112) broker can not start itself after kafka is killed with -9
[ https://issues.apache.org/jira/browse/KAFKA-1112?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-1112. Resolution: Fixed Fix Version/s: 0.8.1 Thanks for the reviews. Committed to trunk after addressing Neha's comments. > broker can not start itself after kafka is killed with -9 > - > > Key: KAFKA-1112 > URL: https://issues.apache.org/jira/browse/KAFKA-1112 > Project: Kafka > Issue Type: Bug > Components: log >Affects Versions: 0.8, 0.8.1 >Reporter: Kane Kim >Assignee: Jay Kreps >Priority: Critical > Fix For: 0.8.1 > > Attachments: KAFKA-1112-v1.patch, KAFKA-1112-v2.patch, > KAFKA-1112-v3.patch, KAFKA-1112-v4.patch, KAFKA-1112.out > > > When I kill kafka with -9, broker cannot start itself because of corrupted > index logs. I think kafka should try to delete/rebuild indexes itself without > manual intervention. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Created] (KAFKA-1135) Code cleanup - use Json.encode() to write json data to zk
Swapnil Ghike created KAFKA-1135: Summary: Code cleanup - use Json.encode() to write json data to zk Key: KAFKA-1135 URL: https://issues.apache.org/jira/browse/KAFKA-1135 Project: Kafka Issue Type: Bug Reporter: Swapnil Ghike Assignee: Swapnil Ghike Fix For: 0.8.1 -- This message was sent by Atlassian JIRA (v6.1#6144)
Review Request 15665: Code clean: use Json.encode() to write json data to zookeeper
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/15665/ --- Review request for kafka. Bugs: KAFKA-1135 https://issues.apache.org/jira/browse/KAFKA-1135 Repository: kafka Description --- json.encode Diffs - core/src/main/scala/kafka/admin/AdminUtils.scala 8ff4bd5a5f6ea1a51df926c31155251bcc109238 core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala 26beb9698422ceb6cc682b86913b4f9d2d4f1307 core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala 981d2bbecf2fa11f1d2c423535c7c30851d2d7bb core/src/main/scala/kafka/consumer/TopicCount.scala a3eb53e8262115d1184cd1c7a2b47f21c22c077b core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala c0350cd05cf1b59866a1fedccbeb700b3e828d44 core/src/main/scala/kafka/controller/KafkaController.scala 88792c2b2a360e928ab9cd00de151e5d5f94452d core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala 33b73609b1178c56e692fb60e35aca04ad1af586 core/src/main/scala/kafka/utils/Utils.scala c9ca95f1937d0ef2e64c70e4d811a0d4f358d9db core/src/main/scala/kafka/utils/ZkUtils.scala 856d13605b0b4bf86010571eacbacc0fb0ba7950 Diff: https://reviews.apache.org/r/15665/diff/ Testing --- Verified Thanks, Swapnil Ghike
[jira] [Commented] (KAFKA-1135) Code cleanup - use Json.encode() to write json data to zk
[ https://issues.apache.org/jira/browse/KAFKA-1135?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13826116#comment-13826116 ] Swapnil Ghike commented on KAFKA-1135: -- Created reviewboard https://reviews.apache.org/r/15665/ against branch origin/trunk > Code cleanup - use Json.encode() to write json data to zk > - > > Key: KAFKA-1135 > URL: https://issues.apache.org/jira/browse/KAFKA-1135 > Project: Kafka > Issue Type: Bug >Reporter: Swapnil Ghike >Assignee: Swapnil Ghike > Fix For: 0.8.1 > > Attachments: KAFKA-1135.patch > > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (KAFKA-1135) Code cleanup - use Json.encode() to write json data to zk
[ https://issues.apache.org/jira/browse/KAFKA-1135?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Swapnil Ghike updated KAFKA-1135: - Attachment: KAFKA-1135.patch > Code cleanup - use Json.encode() to write json data to zk > - > > Key: KAFKA-1135 > URL: https://issues.apache.org/jira/browse/KAFKA-1135 > Project: Kafka > Issue Type: Bug >Reporter: Swapnil Ghike >Assignee: Swapnil Ghike > Fix For: 0.8.1 > > Attachments: KAFKA-1135.patch > > -- This message was sent by Atlassian JIRA (v6.1#6144)
Re: Review Request 15665: Code clean: use Json.encode() to write json data to zookeeper
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/15665/ --- (Updated Nov. 19, 2013, 3:16 a.m.) Review request for kafka. Bugs: KAFKA-1135 https://issues.apache.org/jira/browse/KAFKA-1135 Repository: kafka Description --- json.encode Diffs - core/src/main/scala/kafka/admin/AdminUtils.scala 8ff4bd5a5f6ea1a51df926c31155251bcc109238 core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala 26beb9698422ceb6cc682b86913b4f9d2d4f1307 core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala 981d2bbecf2fa11f1d2c423535c7c30851d2d7bb core/src/main/scala/kafka/consumer/TopicCount.scala a3eb53e8262115d1184cd1c7a2b47f21c22c077b core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala c0350cd05cf1b59866a1fedccbeb700b3e828d44 core/src/main/scala/kafka/controller/KafkaController.scala 88792c2b2a360e928ab9cd00de151e5d5f94452d core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala 33b73609b1178c56e692fb60e35aca04ad1af586 core/src/main/scala/kafka/utils/Utils.scala c9ca95f1937d0ef2e64c70e4d811a0d4f358d9db core/src/main/scala/kafka/utils/ZkUtils.scala 856d13605b0b4bf86010571eacbacc0fb0ba7950 Diff: https://reviews.apache.org/r/15665/diff/ Testing (updated) --- Verified that zookeeper data looks like the structures defined in https://cwiki.apache.org/confluence/display/KAFKA/Kafka+data+structures+in+Zookeeper Thanks, Swapnil Ghike
[jira] [Updated] (KAFKA-1135) Code cleanup - use Json.encode() to write json data to zk
[ https://issues.apache.org/jira/browse/KAFKA-1135?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Swapnil Ghike updated KAFKA-1135: - Attachment: KAFKA-1135_2013-11-18_19:17:54.patch > Code cleanup - use Json.encode() to write json data to zk > - > > Key: KAFKA-1135 > URL: https://issues.apache.org/jira/browse/KAFKA-1135 > Project: Kafka > Issue Type: Bug >Reporter: Swapnil Ghike >Assignee: Swapnil Ghike > Fix For: 0.8.1 > > Attachments: KAFKA-1135.patch, KAFKA-1135_2013-11-18_19:17:54.patch > > -- This message was sent by Atlassian JIRA (v6.1#6144)
Re: Review Request 15665: Patch for KAFKA-1135
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/15665/ --- (Updated Nov. 19, 2013, 3:17 a.m.) Review request for kafka. Summary (updated) - Patch for KAFKA-1135 Bugs: KAFKA-1135 https://issues.apache.org/jira/browse/KAFKA-1135 Repository: kafka Description --- json.encode Diffs (updated) - core/src/main/scala/kafka/admin/AdminUtils.scala 8ff4bd5a5f6ea1a51df926c31155251bcc109238 core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala 26beb9698422ceb6cc682b86913b4f9d2d4f1307 core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala 981d2bbecf2fa11f1d2c423535c7c30851d2d7bb core/src/main/scala/kafka/consumer/TopicCount.scala a3eb53e8262115d1184cd1c7a2b47f21c22c077b core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala c0350cd05cf1b59866a1fedccbeb700b3e828d44 core/src/main/scala/kafka/controller/KafkaController.scala 88792c2b2a360e928ab9cd00de151e5d5f94452d core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala 33b73609b1178c56e692fb60e35aca04ad1af586 core/src/main/scala/kafka/utils/Utils.scala c9ca95f1937d0ef2e64c70e4d811a0d4f358d9db core/src/main/scala/kafka/utils/ZkUtils.scala 856d13605b0b4bf86010571eacbacc0fb0ba7950 Diff: https://reviews.apache.org/r/15665/diff/ Testing --- Verified that zookeeper data looks like the structures defined in https://cwiki.apache.org/confluence/display/KAFKA/Kafka+data+structures+in+Zookeeper Thanks, Swapnil Ghike
[jira] [Commented] (KAFKA-1135) Code cleanup - use Json.encode() to write json data to zk
[ https://issues.apache.org/jira/browse/KAFKA-1135?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13826128#comment-13826128 ] Swapnil Ghike commented on KAFKA-1135: -- Updated reviewboard https://reviews.apache.org/r/15665/ against branch origin/trunk > Code cleanup - use Json.encode() to write json data to zk > - > > Key: KAFKA-1135 > URL: https://issues.apache.org/jira/browse/KAFKA-1135 > Project: Kafka > Issue Type: Bug >Reporter: Swapnil Ghike >Assignee: Swapnil Ghike > Fix For: 0.8.1 > > Attachments: KAFKA-1135.patch, KAFKA-1135_2013-11-18_19:17:54.patch > > -- This message was sent by Atlassian JIRA (v6.1#6144)
Re: Review Request 15665: Patch for KAFKA-1135
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/15665/ --- (Updated Nov. 19, 2013, 3:21 a.m.) Review request for kafka. Bugs: KAFKA-1135 https://issues.apache.org/jira/browse/KAFKA-1135 Repository: kafka Description (updated) --- iteration 2 json.encode Diffs (updated) - core/src/main/scala/kafka/admin/AdminUtils.scala 8ff4bd5a5f6ea1a51df926c31155251bcc109238 core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala 26beb9698422ceb6cc682b86913b4f9d2d4f1307 core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala 981d2bbecf2fa11f1d2c423535c7c30851d2d7bb core/src/main/scala/kafka/consumer/TopicCount.scala a3eb53e8262115d1184cd1c7a2b47f21c22c077b core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala c0350cd05cf1b59866a1fedccbeb700b3e828d44 core/src/main/scala/kafka/controller/KafkaController.scala 88792c2b2a360e928ab9cd00de151e5d5f94452d core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala 33b73609b1178c56e692fb60e35aca04ad1af586 core/src/main/scala/kafka/utils/Utils.scala c9ca95f1937d0ef2e64c70e4d811a0d4f358d9db core/src/main/scala/kafka/utils/ZkUtils.scala 856d13605b0b4bf86010571eacbacc0fb0ba7950 Diff: https://reviews.apache.org/r/15665/diff/ Testing --- Verified that zookeeper data looks like the structures defined in https://cwiki.apache.org/confluence/display/KAFKA/Kafka+data+structures+in+Zookeeper Thanks, Swapnil Ghike
[jira] [Updated] (KAFKA-1135) Code cleanup - use Json.encode() to write json data to zk
[ https://issues.apache.org/jira/browse/KAFKA-1135?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Swapnil Ghike updated KAFKA-1135: - Attachment: KAFKA-1135_2013-11-18_19:20:58.patch > Code cleanup - use Json.encode() to write json data to zk > - > > Key: KAFKA-1135 > URL: https://issues.apache.org/jira/browse/KAFKA-1135 > Project: Kafka > Issue Type: Bug >Reporter: Swapnil Ghike >Assignee: Swapnil Ghike > Fix For: 0.8.1 > > Attachments: KAFKA-1135.patch, KAFKA-1135_2013-11-18_19:17:54.patch, > KAFKA-1135_2013-11-18_19:20:58.patch > > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (KAFKA-1135) Code cleanup - use Json.encode() to write json data to zk
[ https://issues.apache.org/jira/browse/KAFKA-1135?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13826133#comment-13826133 ] Swapnil Ghike commented on KAFKA-1135: -- Updated reviewboard https://reviews.apache.org/r/15665/ against branch origin/trunk > Code cleanup - use Json.encode() to write json data to zk > - > > Key: KAFKA-1135 > URL: https://issues.apache.org/jira/browse/KAFKA-1135 > Project: Kafka > Issue Type: Bug >Reporter: Swapnil Ghike >Assignee: Swapnil Ghike > Fix For: 0.8.1 > > Attachments: KAFKA-1135.patch, KAFKA-1135_2013-11-18_19:17:54.patch, > KAFKA-1135_2013-11-18_19:20:58.patch > > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (KAFKA-1127) kafka and zookeeper server should start in daemon mode and log to correct position
[ https://issues.apache.org/jira/browse/KAFKA-1127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1127: --- Resolution: Fixed Fix Version/s: 0.8.1 Assignee: Raymond Liu Status: Resolved (was: Patch Available) Thanks for patch v3. +1. Committed to trunk. > kafka and zookeeper server should start in daemon mode and log to correct > position > -- > > Key: KAFKA-1127 > URL: https://issues.apache.org/jira/browse/KAFKA-1127 > Project: Kafka > Issue Type: Improvement > Components: core >Affects Versions: 0.9 >Reporter: Raymond Liu >Assignee: Raymond Liu >Priority: Minor > Fix For: 0.8.1 > > Attachments: kafka-1127-v2.patch, kafka-1127-v3.patch, > kafka-1127.patch > > > Hi > At present, the kafka-server-start.sh and zookeeper-server-start.sh will > start kafka broker and zookeeper in fg mode, though "daemon" parameter is > added. you need to add & to make it bg. But the console log will still output > in fg. > Should make it fully run in bg and redirect console log to files. And log > file position should not depends on where the cmd is running. Thus it will be > more friendly for scripts to launch server on nodes by batch. -- This message was sent by Atlassian JIRA (v6.1#6144)