[jira] [Commented] (KAFKA-1196) java.lang.IllegalArgumentException Buffer.limit on FetchResponse.scala + 33
[ https://issues.apache.org/jira/browse/KAFKA-1196?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13869381#comment-13869381 ] Gerrit Jansen van Vuuren commented on KAFKA-1196: - If I throw an error then what? How can I consume the data in the first place? I'm using all the defaults and the provided stock consumer/producer. The problem here is that I've not setup anywhere in the configuration to consume more than 2GB, its the way that the consumer does its fetch that causes the data to go over 2GB. So it means that: If you use the broker and consumer and for some reason the consumer does a fetch over 2GB at any time, I'll be unable to consume the data ever, even though no single message is even over the 100-200 mb. The only solution left to me is then either write my own client, or delete the topic data every time I see this, which is about 5 seconds after I removed and recreated the topic. java.lang.IllegalArgumentException Buffer.limit on FetchResponse.scala + 33 --- Key: KAFKA-1196 URL: https://issues.apache.org/jira/browse/KAFKA-1196 Project: Kafka Issue Type: Bug Components: consumer Affects Versions: 0.8.0 Environment: running java 1.7, linux and kafka compiled against scala 2.9.2 Reporter: Gerrit Jansen van Vuuren Assignee: Neha Narkhede Priority: Blocker Labels: newbie Fix For: 0.9.0 I have 6 topics each with 8 partitions spread over 4 kafka servers. the servers are 24 core 72 gig ram. While consuming from the topics I get an IlegalArgumentException and all consumption stops, the error keeps on throwing. I've tracked it down to FectchResponse.scala line 33 The error happens when the FetchResponsePartitionData object's readFrom method calls: messageSetBuffer.limit(messageSetSize) I put in some debug code the the messageSetSize is 671758648, while the buffer.capacity() gives 155733313, for some reason the buffer is smaller than the required message size. I don't know the consumer code enough to debug this. It doesn't matter if compression is used or not. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (KAFKA-1200) inconsistent log levels when consumed offset is reset
[ https://issues.apache.org/jira/browse/KAFKA-1200?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dima Pekar updated KAFKA-1200: -- Attachment: KAFKA-1200.patch Provided patch that changes log level (several occurrences) to from warn to error inside kafka.server.AbstractFetcherThread#processFetchRequest method. inconsistent log levels when consumed offset is reset - Key: KAFKA-1200 URL: https://issues.apache.org/jira/browse/KAFKA-1200 Project: Kafka Issue Type: Bug Components: consumer Affects Versions: 0.8.0 Reporter: Jason Rosenberg Assignee: Neha Narkhede Fix For: 0.8.1 Attachments: KAFKA-1200.patch I've recently been dealing with the issue where my consumer falls behind and essentially loses data when the broker deletes data, due to it's retention policy. On the broker, this is logged as an ERROR: 2013-12-23 05:02:08,456 ERROR [kafka-request-handler-2] server.KafkaApis - [KafkaApi-45] Error when processing fetch request for partition [mytopic,0] offset 204243601 from consumer with correlation id 130341 kafka.common.OffsetOutOfRangeException: Request for offset 204243601 but we only have log segments in the range 204343397 to 207423640. But on the consumer, this same event is logged as a WARN: 2013-12-23 05:02:08,797 WARN [ConsumerFetcherThread-myconsumergroup-1387353494862-7aa0c61d-0-45] consumer.ConsumerFetcherThread - [ConsumerFetcherThread-myconsumergroup-1387353494862-7aa0c61d-0-45], Current offset 204243601 for partition [mytopic,0] out of range; reset offset to 204343397 It seems this should also be an ERROR condition (it would seem the consumer would care more about this than the broker, at least!). Also, sometimes (but not always) there is also this log message on the consumer, which does log as an ERROR (I'm not sure why this log line doesn't always appear after the above WARN?): 2014-01-08 02:31:47,681 ERROR [myconsumerthread-0] consumer.ConsumerIterator - consumed offset: 16163904970 doesn't match fetch offset: 16175326044 for mytopic:0: fetched offset = 16175330598: consumed offset = 16163904970; Consumer may lose data In this message, there is the Consumer may lose data message, which makes sense. Seems the fetcher thread above should also log something like that, and be an ERROR. This would allow for more consistent alerting, in this case. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (KAFKA-1200) inconsistent log levels when consumed offset is reset
[ https://issues.apache.org/jira/browse/KAFKA-1200?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dima Pekar updated KAFKA-1200: -- Status: Patch Available (was: Open) inconsistent log levels when consumed offset is reset - Key: KAFKA-1200 URL: https://issues.apache.org/jira/browse/KAFKA-1200 Project: Kafka Issue Type: Bug Components: consumer Affects Versions: 0.8.0 Reporter: Jason Rosenberg Assignee: Neha Narkhede Fix For: 0.8.1 Attachments: KAFKA-1200.patch I've recently been dealing with the issue where my consumer falls behind and essentially loses data when the broker deletes data, due to it's retention policy. On the broker, this is logged as an ERROR: 2013-12-23 05:02:08,456 ERROR [kafka-request-handler-2] server.KafkaApis - [KafkaApi-45] Error when processing fetch request for partition [mytopic,0] offset 204243601 from consumer with correlation id 130341 kafka.common.OffsetOutOfRangeException: Request for offset 204243601 but we only have log segments in the range 204343397 to 207423640. But on the consumer, this same event is logged as a WARN: 2013-12-23 05:02:08,797 WARN [ConsumerFetcherThread-myconsumergroup-1387353494862-7aa0c61d-0-45] consumer.ConsumerFetcherThread - [ConsumerFetcherThread-myconsumergroup-1387353494862-7aa0c61d-0-45], Current offset 204243601 for partition [mytopic,0] out of range; reset offset to 204343397 It seems this should also be an ERROR condition (it would seem the consumer would care more about this than the broker, at least!). Also, sometimes (but not always) there is also this log message on the consumer, which does log as an ERROR (I'm not sure why this log line doesn't always appear after the above WARN?): 2014-01-08 02:31:47,681 ERROR [myconsumerthread-0] consumer.ConsumerIterator - consumed offset: 16163904970 doesn't match fetch offset: 16175326044 for mytopic:0: fetched offset = 16175330598: consumed offset = 16163904970; Consumer may lose data In this message, there is the Consumer may lose data message, which makes sense. Seems the fetcher thread above should also log something like that, and be an ERROR. This would allow for more consistent alerting, in this case. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (KAFKA-1197) Count of bytes or messages of a topic stored in kafka
[ https://issues.apache.org/jira/browse/KAFKA-1197?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13869538#comment-13869538 ] Kostya Golikov commented on KAFKA-1197: --- Is it really an issue to sum up offsets for topic's partitions? This will give you bytes (pre-0.8.x) / messages (post-0.8.x) count. Count of bytes or messages of a topic stored in kafka - Key: KAFKA-1197 URL: https://issues.apache.org/jira/browse/KAFKA-1197 Project: Kafka Issue Type: New Feature Components: core Affects Versions: 0.7.2, 0.8.0 Reporter: Hanish Bansal Priority: Minor There should be direct way of measuring count of messages or bytes for a topic stored in Kafka. There are already some very useful metrics like byteRate and messageRate using what we can see count of bytes/messages coming into Kafka broker. I was looking for some jmx metrics that can give count of messages/bytes stored in kafka. If we look into data stores like hbase we can see how many messages are stored in hbase or if we look into search engine like elasticsearch then also we can see how many messages are stored/indexed in elasticsearch. In similar way i was expecting that there should be some way to see count of messages or bytes for a topic stored in kafka without using any external tool. It will be really helpful if there is some support for this using some jmx metric or by script. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (KAFKA-1197) Count of bytes or messages of a topic stored in kafka
[ https://issues.apache.org/jira/browse/KAFKA-1197?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13869555#comment-13869555 ] Hanish Bansal commented on KAFKA-1197: -- Not an issue. In case kafka is not restarted we can get idea of message count using MessageIn metrix. But all metrics are reset to zero when kafka is restarted so its not possible to see message count stored in kafka after restart kafka node. So there can be some script which can give count of messages stored in a topic. Count of bytes or messages of a topic stored in kafka - Key: KAFKA-1197 URL: https://issues.apache.org/jira/browse/KAFKA-1197 Project: Kafka Issue Type: New Feature Components: core Affects Versions: 0.7.2, 0.8.0 Reporter: Hanish Bansal Priority: Minor There should be direct way of measuring count of messages or bytes for a topic stored in Kafka. There are already some very useful metrics like byteRate and messageRate using what we can see count of bytes/messages coming into Kafka broker. I was looking for some jmx metrics that can give count of messages/bytes stored in kafka. If we look into data stores like hbase we can see how many messages are stored in hbase or if we look into search engine like elasticsearch then also we can see how many messages are stored/indexed in elasticsearch. In similar way i was expecting that there should be some way to see count of messages or bytes for a topic stored in kafka without using any external tool. It will be really helpful if there is some support for this using some jmx metric or by script. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
Re: Review Request 16814: Patch for KAFKA-1202
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/16814/#review31626 --- core/src/main/scala/kafka/controller/KafkaController.scala https://reviews.apache.org/r/16814/#comment60298 the summary of steps that we had before is helpful, in addition to the comments inside the API. It is helpful to move it back here. core/src/main/scala/kafka/controller/KafkaController.scala https://reviews.apache.org/r/16814/#comment60297 It is a bit confusing to read the code that starts with step #4. Makes you wonder what about steps 1-3. I think it is useful to have the summary of what it does in the API description, in addition to comments inside the API core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala https://reviews.apache.org/r/16814/#comment60301 new - New core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala https://reviews.apache.org/r/16814/#comment60303 receiving replicas - Replicas that receive the LeaderAndIsr request sent by the leader selector core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala https://reviews.apache.org/r/16814/#comment60305 Like the comment above says The reassigned replicas are ALL in the in sync replicas set before this API is invoked. Hence, this check seems unnecessary right? core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala https://reviews.apache.org/r/16814/#comment60306 Same changes as above core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala https://reviews.apache.org/r/16814/#comment60308 and here - Neha Narkhede On Jan. 13, 2014, 5:57 a.m., Jun Rao wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/16814/ --- (Updated Jan. 13, 2014, 5:57 a.m.) Review request for kafka. Bugs: KAFKA-1202 https://issues.apache.org/jira/browse/KAFKA-1202 Repository: kafka Description --- 1. Changed ununcessary ZK reads to reading from the controller cache. 2. ReassignedPartitionLeaderSelector: Make sure the new leader is in the current isr. 3. ControlledShutdownLeaderSelector: Only make sure that the new leader is not being shutting down. This means that if the leader is already moved off the shutting down broker, we will let leader election succeed (even though it's not necessary). This increases the chance of a successful controlled shutdown. 4. Added various comments. Diffs - core/src/main/scala/kafka/controller/KafkaController.scala 03ef9cf00602cfb6c6485c5a818a2f54a9e499b7 core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala a47b142940fd8aaebbecebf67eb6d74a4dc871de core/src/main/scala/kafka/controller/PartitionStateMachine.scala 5859ce7abaf0b675693a025f867ae9cef59f3bf7 core/src/main/scala/kafka/controller/ReplicaStateMachine.scala ad4ee53a9516e7e6ea14a9ec72a5aab511810f83 core/src/main/scala/kafka/utils/ZkUtils.scala 73902b2bd18b95e00e610fca86d6ed8b4af30e9a Diff: https://reviews.apache.org/r/16814/diff/ Testing --- Thanks, Jun Rao
[jira] [Updated] (KAFKA-1202) optimize ZK access in KafkaController
[ https://issues.apache.org/jira/browse/KAFKA-1202?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1202: - Assignee: Jun Rao optimize ZK access in KafkaController - Key: KAFKA-1202 URL: https://issues.apache.org/jira/browse/KAFKA-1202 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1 Reporter: Jun Rao Assignee: Jun Rao Attachments: KAFKA-1202.patch In KafkaController, we access ZK in the following places. Those accesses are not necessary since we can read from the cache in the controller. In onBrokerFailure(deadBrokers: Seq[Int]), replicaStateMachine.handleStateChanges(getAllReplicasOnBroker(zkClient, controllerContext.allTopics.toSeq, deadBrokers), OfflineReplica) } In onBrokerStartup(newBrokers: Seq[Int]) replicaStateMachine.handleStateChanges(getAllReplicasOnBroker(zkClient, controllerContext.allTopics.toSeq, newBrokers), OnlineReplica) } In shutdownBroker(), getPartitionsAssignedToBroker(zkClient, controllerContext.allTopics.toSeq, id).map { } -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] Subscription: outstanding kafka patches
Issue Subscription Filter: outstanding kafka patches (81 issues) The list of outstanding kafka patches Subscriber: kafka-mailing-list Key Summary KAFKA-1202 optimize ZK access in KafkaController https://issues.apache.org/jira/browse/KAFKA-1202 KAFKA-1200 inconsistent log levels when consumed offset is reset https://issues.apache.org/jira/browse/KAFKA-1200 KAFKA-1199 Add a reduced access log level https://issues.apache.org/jira/browse/KAFKA-1199 KAFKA-1194 The kafka broker cannot delete the old log files after the configured time https://issues.apache.org/jira/browse/KAFKA-1194 KAFKA-1190 create a draw performance graph script https://issues.apache.org/jira/browse/KAFKA-1190 KAFKA-1188 Stale LeaderAndIsr request could be handled by the broker on Controller failover https://issues.apache.org/jira/browse/KAFKA-1188 KAFKA-1180 WhiteList topic filter gets a NullPointerException on complex Regex https://issues.apache.org/jira/browse/KAFKA-1180 KAFKA-1173 Using Vagrant to get up and running with Apache Kafka https://issues.apache.org/jira/browse/KAFKA-1173 KAFKA-1171 Gradle build for Kafka https://issues.apache.org/jira/browse/KAFKA-1171 KAFKA-1147 Consumer socket timeout should be greater than fetch max wait https://issues.apache.org/jira/browse/KAFKA-1147 KAFKA-1145 Broker fail to sync after restart https://issues.apache.org/jira/browse/KAFKA-1145 KAFKA-1144 commitOffsets can be passed the offsets to commit https://issues.apache.org/jira/browse/KAFKA-1144 KAFKA-1130 log.dirs is a confusing property name https://issues.apache.org/jira/browse/KAFKA-1130 KAFKA-1116 Need to upgrade sbt-assembly to compile on scala 2.10.2 https://issues.apache.org/jira/browse/KAFKA-1116 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-1025 Producer.send should provide recoverability info on failiure https://issues.apache.org/jira/browse/KAFKA-1025 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-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)
Re: Review Request 16759: Patch for KAFKA-1199
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/16759/#review31635 --- Ship it! Ship It! - Jun Rao On Jan. 12, 2014, 11:10 p.m., Neha Narkhede wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/16759/ --- (Updated Jan. 12, 2014, 11:10 p.m.) Review request for kafka. Bugs: KAFKA-1199 https://issues.apache.org/jira/browse/KAFKA-1199 Repository: kafka Description --- Removed the default implementation for describe() from RequestOrResponse Renamed describeRequest to describe and added describe for responses, but with the default toString. We can add the details when we find a use case for it Included DEBUG and TRACE levels for access logging. At the DEBUG level, all parts of the request description that are proportional to the number of partitions or topics is omitted Diffs - core/src/main/scala/kafka/api/ControlledShutdownRequest.scala ad6a20d43f9bba937bc700d40c13945a01f007e8 core/src/main/scala/kafka/api/ControlledShutdownResponse.scala b7c844892a9750b5824e2dd66fe928f75e8d039c core/src/main/scala/kafka/api/FetchRequest.scala d41a7056e2ad810974018a79ee696f1a6385b721 core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala 3401afa05269b402c2d2166a2a3427d46234534e core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala 378b2b371377a1f5688da146e51a5c56358539f0 core/src/main/scala/kafka/api/OffsetCommitRequest.scala 1cbe6e8de5a83d79b75249e661ccdacadacbb77c core/src/main/scala/kafka/api/OffsetCommitResponse.scala ad54bd63334b92e3ee3996da94586d39f560c057 core/src/main/scala/kafka/api/OffsetFetchRequest.scala a4c5623dbd48d9a0f21b87e39d63cde3604c64a0 core/src/main/scala/kafka/api/OffsetFetchResponse.scala ce03a13fc218bf67904a33bc3468d644ec7b5a14 core/src/main/scala/kafka/api/OffsetRequest.scala 0a94a6c352581774474d32f0539a3055e8fb1bb4 core/src/main/scala/kafka/api/OffsetResponse.scala fca76a209347737d81c45bf5dfd901edbd8e148b core/src/main/scala/kafka/api/ProducerRequest.scala c6063511173649dd8806c78e850f293419b3dc7a core/src/main/scala/kafka/api/ProducerResponse.scala d59c5bb8b8d735fcf8ffe0b4efad5bdf4f79af81 core/src/main/scala/kafka/api/RequestOrResponse.scala ba59c31a93519eebd24b7a8ad80feea750851025 core/src/main/scala/kafka/api/StopReplicaRequest.scala efd704697024456b547f8344fcc535d37fc2ffb8 core/src/main/scala/kafka/api/StopReplicaResponse.scala c82eadd6a9f7c2054543a6877ede75c80d6ef036 core/src/main/scala/kafka/api/TopicMetadataRequest.scala c5221c4e31a0e18ba435b7801ca0dbed01439677 core/src/main/scala/kafka/api/TopicMetadataResponse.scala 290f263dfdceba9208fba9b6117fa7b66f04d88b core/src/main/scala/kafka/api/UpdateMetadataRequest.scala 2ead3644eb865be02c52a03ef7671d7466fc4a71 core/src/main/scala/kafka/api/UpdateMetadataResponse.scala b1e42c39c8801fc93a8156d3ae7ad8d22d0f40e7 core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala 05757a13ff149801ecbe3d886aaed48ae455993a core/src/main/scala/kafka/network/RequestChannel.scala 330d3a0df280e2016de254619a0987b6ebee101c Diff: https://reviews.apache.org/r/16759/diff/ Testing --- Thanks, Neha Narkhede
kafka pull request: Update README
GitHub user ailzhang opened a pull request: https://github.com/apache/kafka/pull/12 Update README You can merge this pull request into a Git repository by running: $ git pull https://github.com/ailzhang/kafka patch-1 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/12.patch commit d3f5d9f2df6d5b3d5397d370103c3f689abb9683 Author: ailzhang ailzh...@users.noreply.github.com Date: 2014-01-13T19:31:50Z Update README
[jira] [Created] (KAFKA-1203) Step 3 of QuickStart guide is wrong
J Potter created KAFKA-1203: --- Summary: Step 3 of QuickStart guide is wrong Key: KAFKA-1203 URL: https://issues.apache.org/jira/browse/KAFKA-1203 Project: Kafka Issue Type: Bug Components: website Reporter: J Potter Priority: Minor Step 3 of the quick start guide (http://kafka.apache.org/07/quickstart.html) incorrectly lists --zookeeper localhost:2181, when it should be --broker-list localhost:9092 -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Created] (KAFKA-1204) Add KAFKA_LOG_DIR to allow LOG_DIR to be outside of code dir
J Potter created KAFKA-1204: --- Summary: Add KAFKA_LOG_DIR to allow LOG_DIR to be outside of code dir Key: KAFKA-1204 URL: https://issues.apache.org/jira/browse/KAFKA-1204 Project: Kafka Issue Type: Improvement Components: core Reporter: J Potter The kafka-run-class.sh script currently generates a LOG_DIR variable and expects the path under base_dir/logs to be writeable by the process, without a way of overriding it. I think a simple change as below would allow ops folks to fire it up and point the LOG_DIR elsewhere, as desired. bin/kafka-run-class.sh @@ -23,9 +23,11 @@ fi base_dir=$(dirname $0)/.. # create logs directory -LOG_DIR=$base_dir/logs -if [ ! -d $LOG_DIR ]; then - mkdir $LOG_DIR +if [ -z KAFKA_LOG_DIR ]; then +KAFKA_LOG_DIR=$base_dir/logs +fi +if [ ! -d $KAFKA_LOG_DIR ]; then + mkdir $KAFKA_LOG_DIR fi if [ -z $SCALA_VERSION ]; then @@ -98,7 +100,7 @@ if [ $1 = daemon ] [ -z $KAFKA_GC_LOG_OPTS] ; then shift GC_LOG_FILE_NAME=$1$GC_FILE_SUFFIX shift - KAFKA_GC_LOG_OPTS=-Xloggc:$LOG_DIR/$GC_LOG_FILE_NAME -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps + KAFKA_GC_LOG_OPTS=-Xloggc:$KAFKA_LOG_DIR/$GC_LOG_FILE_NAME -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps fi exec $JAVA $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_OPTS -cp $CLASSPATH $KAFKA_OPTS $@ -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (KAFKA-1203) Step 3 of QuickStart guide is wrong
[ https://issues.apache.org/jira/browse/KAFKA-1203?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13870107#comment-13870107 ] Joe Stein commented on KAFKA-1203: -- You might be thinking of the 0.8 quick start http://kafka.apache.org/documentation.html#quickstart which in 0.8 uses the broker-list instead of zookeeper and has the --broker-list param. 0.7 uses zookeeper for producing messages and is a required param for 0.7 as per the quick start Step 3 of QuickStart guide is wrong --- Key: KAFKA-1203 URL: https://issues.apache.org/jira/browse/KAFKA-1203 Project: Kafka Issue Type: Bug Components: website Reporter: J Potter Priority: Minor Labels: documentation Original Estimate: 5m Remaining Estimate: 5m Step 3 of the quick start guide (http://kafka.apache.org/07/quickstart.html) incorrectly lists --zookeeper localhost:2181, when it should be --broker-list localhost:9092 -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (KAFKA-1203) Step 3 of QuickStart guide is wrong
[ https://issues.apache.org/jira/browse/KAFKA-1203?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13870128#comment-13870128 ] J Potter commented on KAFKA-1203: - Ahh. That would explain a lot! My mistake; apologies for the noise on this one. Step 3 of QuickStart guide is wrong --- Key: KAFKA-1203 URL: https://issues.apache.org/jira/browse/KAFKA-1203 Project: Kafka Issue Type: Bug Components: website Reporter: J Potter Priority: Minor Labels: documentation Original Estimate: 5m Remaining Estimate: 5m Step 3 of the quick start guide (http://kafka.apache.org/07/quickstart.html) incorrectly lists --zookeeper localhost:2181, when it should be --broker-list localhost:9092 -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Resolved] (KAFKA-1203) Step 3 of QuickStart guide is wrong
[ https://issues.apache.org/jira/browse/KAFKA-1203?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein resolved KAFKA-1203. -- Resolution: Won't Fix Step 3 of QuickStart guide is wrong --- Key: KAFKA-1203 URL: https://issues.apache.org/jira/browse/KAFKA-1203 Project: Kafka Issue Type: Bug Components: website Reporter: J Potter Priority: Minor Labels: documentation Original Estimate: 5m Remaining Estimate: 5m Step 3 of the quick start guide (http://kafka.apache.org/07/quickstart.html) incorrectly lists --zookeeper localhost:2181, when it should be --broker-list localhost:9092 -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (KAFKA-1204) Add KAFKA_LOG_DIR to allow LOG_DIR to be outside of code dir
[ https://issues.apache.org/jira/browse/KAFKA-1204?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13870290#comment-13870290 ] Joe Stein commented on KAFKA-1204: -- Hi J, that is not a bad idea and can see it being helpful for folks to not have to change the script to use it differently, sure. can you modify it though to work like the JMX_PORT does so folks can have a consistent use of the scripts. So, if KAFKA_LOG_DIR is not specified in the environment variable then it would default to use what exists now without any changes but then they could specify KAFKA_LOG_DIR as an env variable. Also take a look at the git work flow https://cwiki.apache.org/confluence/display/KAFKA/Git+Workflow and patch review tool for https://cwiki.apache.org/confluence/display/KAFKA/Kafka+patch+review+tool for updates/changes to the project, thanks! Add KAFKA_LOG_DIR to allow LOG_DIR to be outside of code dir Key: KAFKA-1204 URL: https://issues.apache.org/jira/browse/KAFKA-1204 Project: Kafka Issue Type: Improvement Components: core Reporter: J Potter Labels: patch Original Estimate: 10m Remaining Estimate: 10m The kafka-run-class.sh script currently generates a LOG_DIR variable and expects the path under base_dir/logs to be writeable by the process, without a way of overriding it. I think a simple change as below would allow ops folks to fire it up and point the LOG_DIR elsewhere, as desired. bin/kafka-run-class.sh @@ -23,9 +23,11 @@ fi base_dir=$(dirname $0)/.. # create logs directory -LOG_DIR=$base_dir/logs -if [ ! -d $LOG_DIR ]; then - mkdir $LOG_DIR +if [ -z KAFKA_LOG_DIR ]; then +KAFKA_LOG_DIR=$base_dir/logs +fi +if [ ! -d $KAFKA_LOG_DIR ]; then + mkdir $KAFKA_LOG_DIR fi if [ -z $SCALA_VERSION ]; then @@ -98,7 +100,7 @@ if [ $1 = daemon ] [ -z $KAFKA_GC_LOG_OPTS] ; then shift GC_LOG_FILE_NAME=$1$GC_FILE_SUFFIX shift - KAFKA_GC_LOG_OPTS=-Xloggc:$LOG_DIR/$GC_LOG_FILE_NAME -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps + KAFKA_GC_LOG_OPTS=-Xloggc:$KAFKA_LOG_DIR/$GC_LOG_FILE_NAME -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps fi exec $JAVA $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_OPTS -cp $CLASSPATH $KAFKA_OPTS $@ -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (KAFKA-1200) inconsistent log levels when consumed offset is reset
[ https://issues.apache.org/jira/browse/KAFKA-1200?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1200: - Resolution: Fixed Status: Resolved (was: Patch Available) inconsistent log levels when consumed offset is reset - Key: KAFKA-1200 URL: https://issues.apache.org/jira/browse/KAFKA-1200 Project: Kafka Issue Type: Bug Components: consumer Affects Versions: 0.8.0 Reporter: Jason Rosenberg Assignee: Neha Narkhede Fix For: 0.8.1 Attachments: KAFKA-1200.patch I've recently been dealing with the issue where my consumer falls behind and essentially loses data when the broker deletes data, due to it's retention policy. On the broker, this is logged as an ERROR: 2013-12-23 05:02:08,456 ERROR [kafka-request-handler-2] server.KafkaApis - [KafkaApi-45] Error when processing fetch request for partition [mytopic,0] offset 204243601 from consumer with correlation id 130341 kafka.common.OffsetOutOfRangeException: Request for offset 204243601 but we only have log segments in the range 204343397 to 207423640. But on the consumer, this same event is logged as a WARN: 2013-12-23 05:02:08,797 WARN [ConsumerFetcherThread-myconsumergroup-1387353494862-7aa0c61d-0-45] consumer.ConsumerFetcherThread - [ConsumerFetcherThread-myconsumergroup-1387353494862-7aa0c61d-0-45], Current offset 204243601 for partition [mytopic,0] out of range; reset offset to 204343397 It seems this should also be an ERROR condition (it would seem the consumer would care more about this than the broker, at least!). Also, sometimes (but not always) there is also this log message on the consumer, which does log as an ERROR (I'm not sure why this log line doesn't always appear after the above WARN?): 2014-01-08 02:31:47,681 ERROR [myconsumerthread-0] consumer.ConsumerIterator - consumed offset: 16163904970 doesn't match fetch offset: 16175326044 for mytopic:0: fetched offset = 16175330598: consumed offset = 16163904970; Consumer may lose data In this message, there is the Consumer may lose data message, which makes sense. Seems the fetcher thread above should also log something like that, and be an ERROR. This would allow for more consistent alerting, in this case. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Closed] (KAFKA-1199) Add a reduced access log level
[ https://issues.apache.org/jira/browse/KAFKA-1199?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede closed KAFKA-1199. Add a reduced access log level -- Key: KAFKA-1199 URL: https://issues.apache.org/jira/browse/KAFKA-1199 Project: Kafka Issue Type: Improvement Components: network Reporter: Neha Narkhede Assignee: Neha Narkhede Attachments: KAFKA-1199.patch, KAFKA-1199_2014-01-12_14:58:25.patch, KAFKA-1199_2014-01-12_15:10:50.patch Today, access logs are either all or none. Sometimes access logging with all the information can get space intensive and overwhelming. It is worth adding some top level and important information at DEBUG and the rest at TRACE, so that it can be configured as required. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Resolved] (KAFKA-1199) Add a reduced access log level
[ https://issues.apache.org/jira/browse/KAFKA-1199?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede resolved KAFKA-1199. -- Resolution: Fixed Add a reduced access log level -- Key: KAFKA-1199 URL: https://issues.apache.org/jira/browse/KAFKA-1199 Project: Kafka Issue Type: Improvement Components: network Reporter: Neha Narkhede Assignee: Neha Narkhede Attachments: KAFKA-1199.patch, KAFKA-1199_2014-01-12_14:58:25.patch, KAFKA-1199_2014-01-12_15:10:50.patch Today, access logs are either all or none. Sometimes access logging with all the information can get space intensive and overwhelming. It is worth adding some top level and important information at DEBUG and the rest at TRACE, so that it can be configured as required. -- This message was sent by Atlassian JIRA (v6.1.5#6160)