Re: Review Request 35867: Patch for KAFKA-1901

2015-06-26 Thread Manikumar Reddy O


> On June 25, 2015, 7:01 p.m., Joel Koshy wrote:
> > build.gradle, line 57
> > 
> >
> > Why was this change made?

This is related to followup patch of KAFKA-2199. Will remove after KAFKA-2199 
checkin.


> On June 25, 2015, 7:01 p.m., Joel Koshy wrote:
> > build.gradle, line 373
> > 
> >
> > Isn't it sufficient to just get the hash from the filesystem 
> > (https://gist.github.com/JonasGroeger/7620911) instead of git log? Why do 
> > we need both approaches?

yes, it should be sufficient. I just felt using git cmd is more cleaner than 
reading from filesystem.

BTW,  these gradle tasks are borrowed from gradle project itself.
https://github.com/gradle/gradle/blob/master/gradle/buildReceipt.gradle


> On June 25, 2015, 7:01 p.m., Joel Koshy wrote:
> > core/src/main/scala/kafka/common/AppInfo.scala, line 17
> > 
> >
> > Do we need this AppInfo anymore? KafkaServer now initializes 
> > JmxReporter - so if you browse your mbeans you will see two AppInfo mbeans 
> > - one under kafka.common and another under kafka
> > 
> > That said, I commented further above that AppInfo should probably be 
> > directly instantiated and registered separately from JmxReporter.

Curently AppInfo is also used for old producer and consumer. We can remove the 
AppInfo usage from KafkaServer. But, I think we can retain this till we migrate 
kafka server to new kafka metrics.


> On June 25, 2015, 7:01 p.m., Joel Koshy wrote:
> > clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java, 
> > line 58
> > 
> >
> > Are you concerned about collisions with multiple clients in the same VM?

yes, this will help us to uniquely identify/register/de-register these Mbeans


> On June 25, 2015, 7:01 p.m., Joel Koshy wrote:
> > clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java, 
> > line 146
> > 
> >
> > Actually, why are we doing this in JmxReporter? Can we just instantiate 
> > an AppInfo mbean directly in the producer/consumer/server?

Yes,  we can instantiate directly. I dont want to duplicate the code. 
JmxReporter is getting instantiated in producer,consumer and server and also 
metrics.close() will deregister these mbeans.

I can move these utility methods (registerAppInfo, unregisterAppInfo) to 
Utils.java and call directly from KafkaProducer,Consumer and Server. Let me 
know your preference. i will change accordingly.


> On June 25, 2015, 7:01 p.m., Joel Koshy wrote:
> > build.gradle, line 386
> > 
> >
> > I was originally interested in this because it would be a quick way to 
> > determine what version someone is running/testing with. However, it is 
> > obviously not foolproof since you could have local changes that are 
> > staged/unstaged but not committed.
> > 
> > This is probably good enough, but can you think about whether it is 
> > possible to easily add a "tainted" boolean field? i.e., if there are any 
> > additional source files that are untracked or staged but not committed?

It should be posible with some git commands. But do we really need this?  most 
of us will be running stable release or some trunk point.


- Manikumar Reddy


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/35867/#review89379
---


On June 25, 2015, 10:11 a.m., Manikumar Reddy O wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/35867/
> ---
> 
> (Updated June 25, 2015, 10:11 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1901
> https://issues.apache.org/jira/browse/KAFKA-1901
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> generated a properties file (kafka-version.properties) with version, 
> git-commitId in it. This props file is used to create AppInfo MBean.
> 
> 
> Diffs
> -
> 
>   build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 
>   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
> 9be8fbc648369ad9db1a7eea94bc1b9edbfdbfd7 
>   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
> 5671a3fbeea8cb9a9ffeeb41aa1b132b92c0cae8 
>   clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java 
> 6b9590c418aedd2727544c5dd23c017b4b72467a 
>   clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java 
> PRE-CREAT

[jira] [Updated] (KAFKA-1901) Move Kafka version to be generated in code by build (instead of in manifest)

2015-06-26 Thread Manikumar Reddy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1901?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Manikumar Reddy updated KAFKA-1901:
---
Attachment: KAFKA-1901_2015-06-26_13:16:29.patch

> Move Kafka version to be generated in code by build (instead of in manifest)
> 
>
> Key: KAFKA-1901
> URL: https://issues.apache.org/jira/browse/KAFKA-1901
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.0
>Reporter: Jason Rosenberg
>Assignee: Manikumar Reddy
> Attachments: KAFKA-1901.patch, KAFKA-1901_2015-06-26_13:16:29.patch
>
>
> With 0.8.2 (rc2), I've started seeing this warning in the logs of apps 
> deployed to our staging (both server and client):
> {code}
> 2015-01-23 00:55:25,273  WARN [async-message-sender-0] common.AppInfo$ - 
> Can't read Kafka version from MANIFEST.MF. Possible cause: 
> java.lang.NullPointerException
> {code}
> The issues is that in our deployment, apps are deployed with single 'shaded' 
> jars (e.g. using the maven shade plugin).  This means the MANIFEST.MF file 
> won't have a kafka version.  Instead, suggest the kafka build generate the 
> proper version in code, as part of the build.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: Review Request 35867: Patch for KAFKA-1901

2015-06-26 Thread Manikumar Reddy O

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/35867/
---

(Updated June 26, 2015, 7:49 a.m.)


Review request for kafka.


Bugs: KAFKA-1901
https://issues.apache.org/jira/browse/KAFKA-1901


Repository: kafka


Description (updated)
---

Addresing Joel's comments


Diffs (updated)
-

  build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 
  clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
9be8fbc648369ad9db1a7eea94bc1b9edbfdbfd7 
  clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
5671a3fbeea8cb9a9ffeeb41aa1b132b92c0cae8 
  clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java 
6b9590c418aedd2727544c5dd23c017b4b72467a 
  clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java 
PRE-CREATION 
  clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java 
07b1b60d3a9cb1a399a2fe95b87229f64f539f3b 
  clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java 
544e120594de78c43581a980b1e4087b4fb98ccb 
  core/src/main/scala/kafka/common/AppInfo.scala 
d642ca555f83c41451d4fcaa5c01a1f86eff0a1c 
  core/src/main/scala/kafka/server/KafkaServer.scala 
52dc728bb1ab4b05e94dc528da1006040e2f28c9 

Diff: https://reviews.apache.org/r/35867/diff/


Testing
---


Thanks,

Manikumar Reddy O



[jira] [Commented] (KAFKA-1901) Move Kafka version to be generated in code by build (instead of in manifest)

2015-06-26 Thread Manikumar Reddy (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1901?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14602524#comment-14602524
 ] 

Manikumar Reddy commented on KAFKA-1901:


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

> Move Kafka version to be generated in code by build (instead of in manifest)
> 
>
> Key: KAFKA-1901
> URL: https://issues.apache.org/jira/browse/KAFKA-1901
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.0
>Reporter: Jason Rosenberg
>Assignee: Manikumar Reddy
> Attachments: KAFKA-1901.patch, KAFKA-1901_2015-06-26_13:16:29.patch
>
>
> With 0.8.2 (rc2), I've started seeing this warning in the logs of apps 
> deployed to our staging (both server and client):
> {code}
> 2015-01-23 00:55:25,273  WARN [async-message-sender-0] common.AppInfo$ - 
> Can't read Kafka version from MANIFEST.MF. Possible cause: 
> java.lang.NullPointerException
> {code}
> The issues is that in our deployment, apps are deployed with single 'shaded' 
> jars (e.g. using the maven shade plugin).  This means the MANIFEST.MF file 
> won't have a kafka version.  Instead, suggest the kafka build generate the 
> proper version in code, as part of the build.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (KAFKA-1100) metrics shouldn't have generation/timestamp specific names

2015-06-26 Thread Manikumar Reddy (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1100?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Manikumar Reddy resolved KAFKA-1100.

   Resolution: Fixed
Fix Version/s: 0.8.2.1

This got fixed in KAFKA-1481. Hence closing the issue.

> metrics shouldn't have generation/timestamp specific names
> --
>
> Key: KAFKA-1100
> URL: https://issues.apache.org/jira/browse/KAFKA-1100
> Project: Kafka
>  Issue Type: Bug
>Reporter: Jason Rosenberg
> Fix For: 0.8.2.1
>
>
> I've noticed that there are several metrics that seem useful for monitoring 
> overtime, but which contain generational timestamps in the metric name.
> We are using yammer metrics libraries to send metrics data in a background 
> thread every 10 seconds (to kafka actually), and then they eventually end up 
> in a metrics database (graphite, opentsdb).  The metrics then get graphed via 
> UI, and we can see metrics going way back, etc.
> Unfortunately, many of the metrics coming from kafka seem to have metric 
> names that change any time the server or consumer is restarted, which makes 
> it hard to easily create graphs over long periods of time (spanning app 
> restarts).
> For example:
> names like: 
> kafka.consumer.FetchRequestAndResponseMetricssquare-1371718712833-e9bb4d10-0-508818741-AllBrokersFetchRequestRateAndTimeMs
> or: 
> kafka.consumer.ZookeeperConsumerConnector...topicName.square-1373476779391-78aa2e83-0-FetchQueueSize
> In our staging environment, we have our servers on regular auto-deploy cycles 
> (they restart every few hours).  So just not longitudinally usable to have 
> metric names constantly changing like this.
> Is there something that can easily be done?  Is it really necessary to have 
> so much cryptic info in the metric name?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2302) Currently iterated chunk is not cleared during consumer shutdown

2015-06-26 Thread Manikumar Reddy (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2302?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14602910#comment-14602910
 ] 

Manikumar Reddy commented on KAFKA-2302:


After calling shutdown method, we will get at-most  current/last read chunk 
messages. shutdown() does not guarantee the immediate  shutdown of consumer 
iterator. This behavior can be changed.

> Currently iterated chunk is not cleared during consumer shutdown
> 
>
> Key: KAFKA-2302
> URL: https://issues.apache.org/jira/browse/KAFKA-2302
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.8.1.1
>Reporter: Matthieu Sprunck
>Assignee: Joel Koshy
>
> During consumer connector shutdown, fetch queues are cleared, but the 
> currently iterated chunk is not cleared.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2302) Currently iterated chunk is not cleared during consumer shutdown

2015-06-26 Thread Matthieu Sprunck (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2302?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14603115#comment-14603115
 ] 

Matthieu Sprunck commented on KAFKA-2302:
-

After calling the shutdown() method there is no way to commit offsets of the 
current read chunk messages. The connection to zookeeper is closed.
The current workflow for the shutdown is :
# Shutdown of Wildcard topic watcher
# Shutdown of Auto commit scheduler
# Closing of fetcher connections
# Clearing of the chunk messages queue and sending of the shutdown command
# Commit of last offsets (only for auto commit)
# Closing of the zookeeper connection

>From my comprehension, the fourth step is not sufficient. The consumer 
>iterator is only able to read the shutdown command in the queue after all 
>messages of the current chunk have been consumed. By calling the close() 
>method of the Kafka stream the current stream would be cleared.

> Currently iterated chunk is not cleared during consumer shutdown
> 
>
> Key: KAFKA-2302
> URL: https://issues.apache.org/jira/browse/KAFKA-2302
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.8.1.1
>Reporter: Matthieu Sprunck
>Assignee: Joel Koshy
>
> During consumer connector shutdown, fetch queues are cleared, but the 
> currently iterated chunk is not cleared.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Help Us Nominate Apache Kafka for a 2015 Bossie (Best of OSS) Award - Due June 30th

2015-06-26 Thread Neha Narkhede
Hello Kafka community members,

We appreciate your use and support of Kafka and all the feedback you’ve
provided to us along the way.  If all is still going well with Kafka and
you’re realizing great value from it, we’d like your support in nominating
Kafka for a 2015 InfoWorld Bossie award, which is an annual award where
InfoWorld honors the best of open source software.


As a reminder, Kafka

was
selected as one of "InfoWorld's top picks in distributed data processing,
data analytics, machine learning, NoSQL databases, and the Hadoop
ecosystem." A technology can win consecutive years, so there's nothing
stopping Kafka from making the list again.

Nominations for this award are very simple and require you to simply
deliver an email to InfoWorld's executive editor Doug Dineley (
doug_dine...@infoworld.com) with the following information:

   -

   The name of your software, or your use case
   -

   A link to Kafka's website: http://kafka.apache.org/
   -

   A few sentences on how you or your customers are using the software and
   why it is important and award-worthy.

Submissions must be sent to Doug  by June 30,
2015. Please let us know if you have any questions or if we can help in any
way.


Thank you for being part of the Kafka community!


-- 
Best,
Neha


Re: Help Us Nominate Apache Kafka for a 2015 Bossie (Best of OSS) Award - Due June 30th

2015-06-26 Thread Gwen Shapira
Sent! Thanks for letting us know of this opportunity to promote our
favorite Apache project :)

For inspiration, here's what I wrote:

Apache Kafka revolutionized stream processing for big data. There is a
dramatic growth in the number and variety of data sources an
organization has to track as well as new business requirements to
process and analyze the data in real time.
Not only does Kafka made stream processing a practical reality - by
integrating all data sources into a scalable and reliable platforms
that provides consistent streams of data for stream processing systems
like Storm, Spark and Flink to work with. Stream processing is
probably the most exciting big data innovation of the year, and it
would not be possible without Kafka to provide those streams of data.

Please don't copy-paste, but in case you want to help and feel
writer-blocked, I hope you get some inspiration :)

Gwen

On Fri, Jun 26, 2015 at 9:47 AM, Neha Narkhede  wrote:
> Hello Kafka community members,
>
> We appreciate your use and support of Kafka and all the feedback you’ve
> provided to us along the way.  If all is still going well with Kafka and
> you’re realizing great value from it, we’d like your support in nominating
> Kafka for a 2015 InfoWorld Bossie award, which is an annual award where
> InfoWorld honors the best of open source software.
>
>
> As a reminder, Kafka
> 
> was
> selected as one of "InfoWorld's top picks in distributed data processing,
> data analytics, machine learning, NoSQL databases, and the Hadoop
> ecosystem." A technology can win consecutive years, so there's nothing
> stopping Kafka from making the list again.
>
> Nominations for this award are very simple and require you to simply
> deliver an email to InfoWorld's executive editor Doug Dineley (
> doug_dine...@infoworld.com) with the following information:
>
>-
>
>The name of your software, or your use case
>-
>
>A link to Kafka's website: http://kafka.apache.org/
>-
>
>A few sentences on how you or your customers are using the software and
>why it is important and award-worthy.
>
> Submissions must be sent to Doug  by June 30,
> 2015. Please let us know if you have any questions or if we can help in any
> way.
>
>
> Thank you for being part of the Kafka community!
>
>
> --
> Best,
> Neha


Re: [jira] [Commented] (KAFKA-2132) Move Log4J appender to a separate module

2015-06-26 Thread Stevo Slavić
Are changes for KAFKA-2132 ticket supposed also to fix bug that core
dependent libraries (core/build/dependant-libs) for all different supported
Scala version, contain two versions of slf4j-log4j12
(slf4j-log4j12-1.6.1.jar leaking from zookeeper 3.4.6 dependency, and test
scoped slf4j-log4j12-1.7.6.jar dependency - latter is explicitly added for
some reason in copyDependantLibs task, but it does not override
slf4j-log4j12 leak from zookeeper) ?

It's the source of annoying:

SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in
[jar:file:/Users/foo/kafka/core/build/dependant-libs-2.10.5/slf4j-log4j12-1.6.1.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in
[jar:file:/Users/foo/kafka/core/build/dependant-libs-2.10.5/slf4j-log4j12-1.7.6.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an
explanation.
SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory]

whenever a script like bin/kafka-topics.sh is run.

Or should separate ticket be filed for this issue?

Kind regard,
Stevo Slavic.

On Wed, Jun 24, 2015 at 7:26 PM, Ashish K Singh (JIRA) 
wrote:

>
> [
> https://issues.apache.org/jira/browse/KAFKA-2132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14599778#comment-14599778
> ]
>
> Ashish K Singh commented on KAFKA-2132:
> ---
>
> Updated reviewboard https://reviews.apache.org/r/33614/
>  against branch trunk
>
> > Move Log4J appender to a separate module
> > 
> >
> > Key: KAFKA-2132
> > URL: https://issues.apache.org/jira/browse/KAFKA-2132
> > Project: Kafka
> >  Issue Type: Improvement
> >Reporter: Gwen Shapira
> >Assignee: Ashish K Singh
> > Attachments: KAFKA-2132.patch,
> KAFKA-2132_2015-04-27_19:59:46.patch, KAFKA-2132_2015-04-30_12:22:02.patch,
> KAFKA-2132_2015-04-30_15:53:17.patch, KAFKA-2132_2015-06-13_21:18:59.patch,
> KAFKA-2132_2015-06-24_10:19:56.patch, KAFKA-2132_2015-06-24_10:25:43.patch
> >
> >
> > Log4j appender is just a producer.
> > Since we have a new producer in the clients module, no need to keep
> Log4J appender in "core" and force people to package all of Kafka with
> their apps.
> > Lets move the Log4jAppender to clients module.
>
>
>
> --
> This message was sent by Atlassian JIRA
> (v6.3.4#6332)
>


[jira] [Commented] (KAFKA-2292) failed fetch request logging doesn't indicate source of request

2015-06-26 Thread Ted Malaska (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14603771#comment-14603771
 ] 

Ted Malaska commented on KAFKA-2292:


Sorry this is taking me a long time.  I was having trouble finding the code 
that outputted this error, until Gwen S helped me see that this code has been 
completely refactored from 0.8.2 to Trunk.  

I will do some digging to see if we can get consumer information in the message 
or not.

Since things are so different I'm going to have to trace everything from 
KafkaAPis.handleFetchRequest to see where to add this functionality it.

I will do it in the next couple of days.  This is turning out to be a great 
learning Jira.  :)

> failed fetch request logging doesn't indicate source of request
> ---
>
> Key: KAFKA-2292
> URL: https://issues.apache.org/jira/browse/KAFKA-2292
> Project: Kafka
>  Issue Type: Bug
>Reporter: Jason Rosenberg
>
> I am trying to figure out the source of a consumer client that is issuing out 
> of range offset requests for a topic, on one for our brokers (we are running 
> 0.8.2.1).
> I see log lines like this:
> {code}
> 2015-06-20 06:17:24,718 ERROR [kafka-request-handler-4] server.ReplicaManager 
> - [Replica Manager on Broker 123]: Error when processing fetch request for 
> partition [mytopic,0] offset 82754176 from consumer with correlation id 596. 
> Possible cause: Request for offset 82754176 but we only have log segments in 
> the range 82814171 to 83259786.
> {code}
> Successive log lines are similar, but with the correlation id incremented, 
> etc.
> Unfortunately, the correlation id is not particularly useful here in the 
> logging, because I have nothing to trace it back to to understand which 
> connected consumer is issuing this request.  It would be useful if the 
> logging included an ip address, or a clientId.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (KAFKA-2292) failed fetch request logging doesn't indicate source of request

2015-06-26 Thread Gwen Shapira (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Gwen Shapira updated KAFKA-2292:

Assignee: Ted Malaska

> failed fetch request logging doesn't indicate source of request
> ---
>
> Key: KAFKA-2292
> URL: https://issues.apache.org/jira/browse/KAFKA-2292
> Project: Kafka
>  Issue Type: Bug
>Reporter: Jason Rosenberg
>Assignee: Ted Malaska
>
> I am trying to figure out the source of a consumer client that is issuing out 
> of range offset requests for a topic, on one for our brokers (we are running 
> 0.8.2.1).
> I see log lines like this:
> {code}
> 2015-06-20 06:17:24,718 ERROR [kafka-request-handler-4] server.ReplicaManager 
> - [Replica Manager on Broker 123]: Error when processing fetch request for 
> partition [mytopic,0] offset 82754176 from consumer with correlation id 596. 
> Possible cause: Request for offset 82754176 but we only have log segments in 
> the range 82814171 to 83259786.
> {code}
> Successive log lines are similar, but with the correlation id incremented, 
> etc.
> Unfortunately, the correlation id is not particularly useful here in the 
> logging, because I have nothing to trace it back to to understand which 
> connected consumer is issuing this request.  It would be useful if the 
> logging included an ip address, or a clientId.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (KAFKA-2303) Fix for KAFKA-2235 LogCleaner offset map overflow causes another compaction failures

2015-06-26 Thread Alexander Demidko (JIRA)
Alexander Demidko created KAFKA-2303:


 Summary: Fix for KAFKA-2235 LogCleaner offset map overflow causes 
another compaction failures
 Key: KAFKA-2303
 URL: https://issues.apache.org/jira/browse/KAFKA-2303
 Project: Kafka
  Issue Type: Bug
  Components: core, log
Affects Versions: 0.8.2.1
Reporter: Alexander Demidko
Assignee: Jay Kreps


We have rolled out the patch for KAFKA-2235 to our kafka cluster, and recently 
instead of 
{code}
"kafka.log.LogCleaner - [kafka-log-cleaner-thread-0], Error due to
java.lang.IllegalArgumentException: requirement failed: Attempt to add a new 
entry to a full offset map." 
{code}
we started to see 
{code}
kafka.log.LogCleaner - [kafka-log-cleaner-thread-0], Error due to
java.lang.IllegalArgumentException: requirement failed: 131390902 messages in 
segment -cgstate-8/79840768.log but offset map can fit 
only 80530612. You can increase log.cleaner.dedupe.buffer.size or decrease 
log.cleaner.threads
{code}
So, we had to roll it back to avoid disk depletion although I'm not sure if it 
needs to be rolled back in trunk. This patch applies more strict checks than 
were in place before: even if there is only one unique key for a segment, 
cleanup will fail if this segment is too big. 


Does it make sense to eliminate a limit for the offset map slots count, for 
example to use an offset map backed by a memory mapped file?

The limit of 80530612 slots comes from memory / bytesPerEntry, where memory is 
Int.MaxValue (we use only one cleaner thread) and bytesPerEntry is 8 + digest 
hash size. Might be wrong, but it seems if the overall number of unique keys 
per partition is more than 80M slots in an OffsetMap, compaction will always 
fail and cleaner thread will die. 




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)