Re: Changing hash algorithm to LogCleaner offset map

2016-07-22 Thread Shikhar Bhushan
Not sure I understand the motivation to use a FIPS-compliant hash function for log compaction -- what are the security ramifications? On Fri, Jul 22, 2016 at 2:56 PM Luciano Afranllie wrote: > A little bit of background first. > > We are trying to make a deployment of

Re: Changing hash algorithm to LogCleaner offset map

2016-07-22 Thread Luciano Afranllie
A little bit of background first. We are trying to make a deployment of Kafka that is FIPS 140-2 ( https://en.wikipedia.org/wiki/FIPS_140-2) complaint and one of the requirements is not to use MD5. As far as we could see, Kafka is using MD5 only to hash message keys in a offset map

[jira] [Commented] (KAFKA-3977) KafkaConsumer swallows exceptions raised from message deserializers

2016-07-22 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3977?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15390218#comment-15390218 ] ASF GitHub Bot commented on KAFKA-3977: --- GitHub user hachikuji opened a pull request:

[GitHub] kafka pull request #1656: KAFKA-3977: Defer fetch parsing for space efficien...

2016-07-22 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1656 KAFKA-3977: Defer fetch parsing for space efficiency and to ensure exceptions are raised to the user You can merge this pull request into a Git repository by running: $ git pull

[jira] [Work started] (KAFKA-3973) Investigate feasibility of caching bytes vs. records

2016-07-22 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3973?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3973 started by Bill Bejeck. -- > Investigate feasibility of caching bytes vs. records >

[jira] [Updated] (KAFKA-3982) Issue with processing order of consumer properties in console consumer

2016-07-22 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-3982: --- Status: Patch Available (was: Open) > Issue with processing order of consumer properties in

Changing hash algorithm to LogCleaner offset map

2016-07-22 Thread Luciano Afranllie
Hi We are evaluating to change the hash algorithm used by the SkimpyOffsetMap used by the LogCleaner from MD5 to SHA-1. Besides the impact in performance (more memory, more cpu usage) is there anything that may be impacted? Regards Luciano

[GitHub] kafka pull request #1655: KAFKA-3982: Fix processing order of some of the co...

2016-07-22 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/1655 KAFKA-3982: Fix processing order of some of the consumer properties This PR updates processing of console consumer's input properties. For both old and new consumer, the value

[jira] [Commented] (KAFKA-3982) Issue with processing order of consumer properties in console consumer

2016-07-22 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3982?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15390181#comment-15390181 ] ASF GitHub Bot commented on KAFKA-3982: --- GitHub user vahidhashemian opened a pull request:

[jira] [Work started] (KAFKA-3777) Extract the existing LRU cache out of RocksDBStore

2016-07-22 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3777?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3777 started by Anna Povzner. --- > Extract the existing LRU cache out of RocksDBStore >

Re: [DISCUSS] KIP-70: Revise Partition Assignment Semantics on New Consumer's Subscription Change

2016-07-22 Thread Dana Powers
This is a nice change. Great KIP write up. -Dana On Fri, Jul 22, 2016 at 10:07 AM, Vahid S Hashemian wrote: > Thanks Ismael. > > What do you think is the best way to check with Storm / Spark users? Their > mailing list? > > Thanks. > > Regards, > --Vahid > > > > >

Build failed in Jenkins: kafka-trunk-jdk7 #1437

2016-07-22 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Increase default `waitTime` in `waitUntilTrue` to 15 seconds [ismael] KAFKA-3983; Add additional information to Acceptor debug message -- [...truncated 6752

[jira] [Commented] (KAFKA-1650) Mirror Maker could lose data on unclean shutdown.

2016-07-22 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1650?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15390055#comment-15390055 ] ASF GitHub Bot commented on KAFKA-1650: --- GitHub user ottomata opened a pull request:

[GitHub] kafka pull request #1654: MINOR: Update MirrorMaker docs to remove multiple ...

2016-07-22 Thread ottomata
GitHub user ottomata opened a pull request: https://github.com/apache/kafka/pull/1654 MINOR: Update MirrorMaker docs to remove multiple --consumer.config options See: - https://issues.apache.org/jira/browse/KAFKA-1650 -

Jenkins build is back to normal : kafka-trunk-jdk8 #772

2016-07-22 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-2932) Adjust importance level of Kafka Connect configs

2016-07-22 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2932?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15389961#comment-15389961 ] ASF GitHub Bot commented on KAFKA-2932: --- GitHub user cotedm opened a pull request:

[GitHub] kafka pull request #1653: KAFKA-2932: Adjust importance level of Kafka Conne...

2016-07-22 Thread cotedm
GitHub user cotedm opened a pull request: https://github.com/apache/kafka/pull/1653 KAFKA-2932: Adjust importance level of Kafka Connect configs @ewencp I went down the list of connect configs and it looks like only the internal converter configs are mismarked. It looks like the

[jira] [Assigned] (KAFKA-2932) Adjust importance level of Kafka Connect configs

2016-07-22 Thread Dustin Cote (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2932?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dustin Cote reassigned KAFKA-2932: -- Assignee: Dustin Cote (was: Ewen Cheslack-Postava) > Adjust importance level of Kafka Connect

[jira] [Commented] (KAFKA-2932) Adjust importance level of Kafka Connect configs

2016-07-22 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2932?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15389872#comment-15389872 ] Ewen Cheslack-Postava commented on KAFKA-2932: -- [~cotedm] Yes please! Just FYI, almost

Re: [DISCUSS] KIP-70: Revise Partition Assignment Semantics on New Consumer's Subscription Change

2016-07-22 Thread Vahid S Hashemian
Thanks Jason / Ewen for your feedback. I agree that this is more like a bug than anything else and should have little impact on the users. Regards, --Vahid From: Ewen Cheslack-Postava To: dev@kafka.apache.org Date: 07/21/2016 10:59 PM Subject:Re:

[jira] [Resolved] (KAFKA-3983) It would be helpful if SocketServer's Acceptors logged both the SocketChannel port and the processor ID upon registra

2016-07-22 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3983?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-3983. Resolution: Fixed > It would be helpful if SocketServer's Acceptors logged both the SocketChannel

[jira] [Commented] (KAFKA-3983) It would be helpful if SocketServer's Acceptors logged both the SocketChannel port and the processor ID upon registra

2016-07-22 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15389844#comment-15389844 ] ASF GitHub Bot commented on KAFKA-3983: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #1648: KAFKA-3983 - Add additional information to debug

2016-07-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1648 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

Re: [DISCUSS] KIP-70: Revise Partition Assignment Semantics on New Consumer's Subscription Change

2016-07-22 Thread Vahid S Hashemian
Thanks Ismael. What do you think is the best way to check with Storm / Spark users? Their mailing list? Thanks. Regards, --Vahid From: Ismael Juma To: dev@kafka.apache.org Date: 07/22/2016 01:44 AM Subject:Re: [DISCUSS] KIP-70: Revise Partition

[GitHub] kafka pull request #1589: MINOR: Increase default `waitTime` in `waitUntilTr...

2016-07-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1589 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Created] (KAFKA-3985) Transient system test failure ZooKeeperSecurityUpgradeTest.test_zk_security_upgrade.security_protocol

2016-07-22 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-3985: -- Summary: Transient system test failure ZooKeeperSecurityUpgradeTest.test_zk_security_upgrade.security_protocol Key: KAFKA-3985 URL:

[jira] [Commented] (KAFKA-2932) Adjust importance level of Kafka Connect configs

2016-07-22 Thread Dustin Cote (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2932?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15389722#comment-15389722 ] Dustin Cote commented on KAFKA-2932: [~ewencp] mind if I pick this one up? > Adjust importance level

[jira] [Commented] (KAFKA-2394) Use RollingFileAppender by default in log4j.properties

2016-07-22 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2394?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15389617#comment-15389617 ] ASF GitHub Bot commented on KAFKA-2394: --- GitHub user cotedm opened a pull request:

[jira] [Commented] (KAFKA-2394) Use RollingFileAppender by default in log4j.properties

2016-07-22 Thread Dustin Cote (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2394?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15389618#comment-15389618 ] Dustin Cote commented on KAFKA-2394: had to change the PR to come from a different branch so I could

[GitHub] kafka pull request #1652: KAFKA-2394: move to RollingFileAppender by default...

2016-07-22 Thread cotedm
GitHub user cotedm opened a pull request: https://github.com/apache/kafka/pull/1652 KAFKA-2394: move to RollingFileAppender by default for log4j You can merge this pull request into a Git repository by running: $ git pull https://github.com/cotedm/kafka KAFKA-2394

[jira] [Commented] (KAFKA-2394) Use RollingFileAppender by default in log4j.properties

2016-07-22 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2394?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15389514#comment-15389514 ] ASF GitHub Bot commented on KAFKA-2394: --- Github user cotedm closed the pull request at:

[GitHub] kafka pull request #1434: KAFKA-2394: move to RollingFileAppender by default...

2016-07-22 Thread cotedm
Github user cotedm closed the pull request at: https://github.com/apache/kafka/pull/1434 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[GitHub] kafka pull request #1651: MINOR: Fix typos in security section

2016-07-22 Thread ssaamm
GitHub user ssaamm opened a pull request: https://github.com/apache/kafka/pull/1651 MINOR: Fix typos in security section 1. I think the instructions in step 2 of the security section which describe adding the CA to server/client truststores are swapped. That is, the instruction

Re: [DISCUSS] KIP-4 ACL Admin Schema

2016-07-22 Thread Jim Jagielski
> On Jul 21, 2016, at 10:57 PM, Ismael Juma wrote: > > Hi Grant, > > Thanks for the KIP. A few questions and comments: > > 1. My main concern is that we are skipping the discussion on the desired > model for controlling ACL access and updates. I understand the desire to >

Re: [DISCUSS] Optimise memory used by replication process by using adaptive fetch message size

2016-07-22 Thread Andrey L. Neporada
Hi! Thanks for feedback - I agree that the proper way to fix this issue is to provide per-request data limit. Will try to do it. Thanks, Andrey. > On 21 Jul 2016, at 18:57, Jay Kreps wrote: > > I think the memory usage for consumers can be improved a lot, but I think >

[jira] [Commented] (KAFKA-3916) Connection from controller to broker disconnects

2016-07-22 Thread Andrey Konyaev (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3916?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15389188#comment-15389188 ] Andrey Konyaev commented on KAFKA-3916: --- I have this problem with 0.10 vers. > Connection from

[jira] [Resolved] (KAFKA-3167) Use local to the workspace Gradle cache and recreate it on every build

2016-07-22 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3167?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-3167. Resolution: Fixed > Use local to the workspace Gradle cache and recreate it on every build >

Re: [DISCUSS] KIP-70: Revise Partition Assignment Semantics on New Consumer's Subscription Change

2016-07-22 Thread Ismael Juma
Thanks for the KIP Vahid. The change makes sense. On the compatibility front, could we check some of the advanced Kafka users like Storm and Spark in order to verify if they would be affected? Ismael On Wed, Jul 20, 2016 at 1:55 AM, Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > Hi

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-07-22 Thread Shikhar Bhushan
flatMap() / supporting 1->n feels nice and general since filtering is just the case of going from 1->0 I'm not sure why we'd need to do any more granular offset tracking (like sub-offsets) for source connectors: after transformation of a given record to n records, all of those n should map to

Jenkins build is back to normal : kafka-trunk-jdk7 #1436

2016-07-22 Thread Apache Jenkins Server
See