Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-06 Thread Henry Cai
It wasn't quite clear to me how the user program interacts with the discovery API, especially on the user supplied listener part, how does the user program supply that listener to KafkaStreams and how does KafkaStreams know which port the user listener is running, maybe a more complete end-to-end

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-06 Thread Guozhang Wang
+1 On Wed, Jul 6, 2016 at 12:44 PM, Damian Guy wrote: > Hi all, > > I'd like to initiate the voting process for KIP-67 > < > https://cwiki.apache.org/confluence/display/KAFKA/KIP-67%3A+Queryable+state+for+Kafka+Streams > > > > KAFKA-3909

[jira] [Updated] (KAFKA-3931) kafka.api.PlaintextConsumerTest.testPatternUnsubscription transient failure

2016-07-06 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3931?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-3931: --- Status: Patch Available (was: In Progress) >

Error after restarting kafka broker

2016-07-06 Thread Subhash Agrawal
Hi All, I am running kafka broker (0.10.0 version) in standalone mode (Single kafka broker in the cluster). After couple of restart, I see this error during embedded kafka startup, even though it does not seem to be causing any problem. Is there any way I can avoid this error? Thanks Subhash

[jira] [Comment Edited] (KAFKA-3705) Support non-key joining in KTable

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3705?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15365365#comment-15365365 ] Guozhang Wang edited comment on KAFKA-3705 at 7/7/16 12:09 AM: --- [~jfilipiak]

[jira] [Commented] (KAFKA-3705) Support non-key joining in KTable

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3705?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15365365#comment-15365365 ] Guozhang Wang commented on KAFKA-3705: -- [~jfilipiak] I am convinced that this combo-key is necessary

[jira] [Commented] (KAFKA-1543) Changing replication factor

2016-07-06 Thread Richard Lee (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1543?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15365357#comment-15365357 ] Richard Lee commented on KAFKA-1543: This jira item seems orphaned. Anyway, I took another stab at

[GitHub] kafka pull request #1596: Change replication factor during partition map gen...

2016-07-06 Thread llamahunter
GitHub user llamahunter opened a pull request: https://github.com/apache/kafka/pull/1596 Change replication factor during partition map generation If the topic-to-move-json-file contains a new replication-factor for a topic, it is used when assigning partitions to brokers. If

[GitHub] kafka pull request #1595: MINOR: Typo fix in comments

2016-07-06 Thread naferx
GitHub user naferx opened a pull request: https://github.com/apache/kafka/pull/1595 MINOR: Typo fix in comments You can merge this pull request into a Git repository by running: $ git pull https://github.com/naferx/kafka minor-typo Alternatively you can review and apply

[jira] [Commented] (KAFKA-3894) Log Cleaner thread crashes and never restarts

2016-07-06 Thread Tim Carey-Smith (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15365233#comment-15365233 ] Tim Carey-Smith commented on KAFKA-3894: Woohoo, more metrics is so excellent! Regarding the

Re: Parallelisation factor in kafka streams

2016-07-06 Thread Jeyhun Karimov
I meant restarting the underlying KAFKA on which Kafka Streams library is running. So, with restarting Streams application (and not the underlying KAFKA) one can change the number of threads within the application. On Wed, Jul 6, 2016 at 11:22 PM Matthias J. Sax wrote: >

[GitHub] kafka pull request #1594: KAFKA-3931: WIP

2016-07-06 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/1594 KAFKA-3931: WIP You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian/kafka KAFKA-3931 Alternatively you can review and apply

[jira] [Commented] (KAFKA-3931) kafka.api.PlaintextConsumerTest.testPatternUnsubscription transient failure

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

Re: Parallelisation factor in kafka streams

2016-07-06 Thread Matthias J. Sax
Jeyhun, you cannot change the number of threads within an application instance, but you can start new application instances. Internal Kafka Consumer re-balance with re-assign the partitions over all running application instances. Not sure what you mean by "restart the cluster"? For sure, you do

Build failed in Jenkins: kafka-trunk-jdk8 #738

2016-07-06 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3836: KStreamReduce and KTableReduce should not pass nulls to [wangguoz] KAFKA-3926: Fix transient test failure in RegexSourceIntegrationTest -- [...truncated 3394

Build failed in Jenkins: kafka-0.10.0-jdk7 #144

2016-07-06 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3836: KStreamReduce and KTableReduce should not pass nulls to -- [...truncated 1674 lines...] kafka.api.PlaintextConsumerTest > testPartitionPauseAndResume PASSED

[jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15365046#comment-15365046 ] Jun Rao commented on KAFKA-3919: [~BigAndy], yes, you are right. We actually do index based on the first

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

2016-07-06 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3836: KStreamReduce and KTableReduce should not pass nulls to [wangguoz] KAFKA-3926: Fix transient test failure in RegexSourceIntegrationTest -- [...truncated

Re: Parallelisation factor in kafka streams

2016-07-06 Thread Jeyhun Karimov
Thank you for your answer Matthias. Is it possible to change the parallelism in runtime? Or do we have to restart the cluster? On Wed, 6 Jul 2016 at 19:08, Matthias J. Sax wrote: > Hi Jeyhun, > > the number of partitions determine the number of tasks within a Kafka >

[jira] [Commented] (KAFKA-3894) Log Cleaner thread crashes and never restarts

2016-07-06 Thread Kiran Pillarisetty (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15365026#comment-15365026 ] Kiran Pillarisetty commented on KAFKA-3894: --- Regarding Log cleaner JMX metrics, I just submitted

[jira] [Updated] (KAFKA-3857) Additional log cleaner metrics

2016-07-06 Thread Kiran Pillarisetty (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3857?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kiran Pillarisetty updated KAFKA-3857: -- Reviewer: Ismael Juma Status: Patch Available (was: Open) [~ijuma] Since there

[jira] [Commented] (KAFKA-3857) Additional log cleaner metrics

2016-07-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15365013#comment-15365013 ] ASF GitHub Bot commented on KAFKA-3857: --- GitHub user kiranptivo reopened a pull request:

[GitHub] kafka pull request #1593: KAFKA-3857 Additional log cleaner metrics

2016-07-06 Thread kiranptivo
GitHub user kiranptivo reopened a pull request: https://github.com/apache/kafka/pull/1593 KAFKA-3857 Additional log cleaner metrics Fixes KAFKA-3857 Changes proposed in this pull request: The following additional log cleaner metrics have been added. 1.

[jira] [Commented] (KAFKA-3857) Additional log cleaner metrics

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

[GitHub] kafka pull request #1593: KAFKA-3857 Additional log cleaner metrics

2016-07-06 Thread kiranptivo
Github user kiranptivo closed the pull request at: https://github.com/apache/kafka/pull/1593 --- 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

[VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-06 Thread Damian Guy
Hi all, I'd like to initiate the voting process for KIP-67 KAFKA-3909 is the top level JIRA for this effort. Initial PRs for Step 1 of the process

[jira] [Resolved] (KAFKA-3926) Transient failures in org.apache.kafka.streams.integration.RegexSourceIntegrationTest

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3926?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3926. -- Resolution: Fixed Fix Version/s: 0.10.1.0 Issue resolved by pull request 1590

Re: [DISCUSS] KIP-67: Queryable state for Kafka Stream

2016-07-06 Thread Damian Guy
Thanks - updated On Wed, 6 Jul 2016 at 20:08 Guozhang Wang wrote: > Thanks Damian, the KIP wiki looks good to me. One minor comment on the > "Compatibility, Deprecation, and Migration Plan" section: we probably also > want to mentions that since we need to handle concurrent

[jira] [Commented] (KAFKA-3926) Transient failures in org.apache.kafka.streams.integration.RegexSourceIntegrationTest

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

[GitHub] kafka pull request #1590: KAFKA-3926: Transient test failures - confirm all ...

2016-07-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1590 --- 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] [Resolved] (KAFKA-3836) KStreamReduce and KTableReduce should not pass nulls to Deserializers

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3836?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3836. -- Resolution: Fixed Fix Version/s: 0.10.0.1 Issue resolved by pull request 1591

[jira] [Updated] (KAFKA-3836) RocksDBStore.get() should not pass nulls to Deserializers

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3836?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3836: - Summary: RocksDBStore.get() should not pass nulls to Deserializers (was: KStreamReduce and

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

2016-07-06 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3794: added stream / table names as prefix to print / writeAsText -- [...truncated 5680 lines...] kafka.log.LogCleanerIntegrationTest > cleanerTest[0] PASSED

[GitHub] kafka pull request #1591: KAFKA-3836: KStreamReduce and KTableReduce should ...

2016-07-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1591 --- 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] [Commented] (KAFKA-3836) KStreamReduce and KTableReduce should not pass nulls to Deserializers

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

[jira] [Commented] (KAFKA-3101) Optimize Aggregation Outputs

2016-07-06 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15364919#comment-15364919 ] Bill Bejeck commented on KAFKA-3101: Is this available now? If so I'd like to pick this up if

Re: [DISCUSS] KIP-67: Queryable state for Kafka Stream

2016-07-06 Thread Guozhang Wang
Thanks Damian, the KIP wiki looks good to me. One minor comment on the "Compatibility, Deprecation, and Migration Plan" section: we probably also want to mentions that since we need to handle concurrent access with the queryable state support, this may incur slight overhead on the streams

[jira] [Created] (KAFKA-3932) Consumer fails to consume in a round robin fashion

2016-07-06 Thread Elias Levy (JIRA)
Elias Levy created KAFKA-3932: - Summary: Consumer fails to consume in a round robin fashion Key: KAFKA-3932 URL: https://issues.apache.org/jira/browse/KAFKA-3932 Project: Kafka Issue Type: Bug

Build failed in Jenkins: kafka-trunk-jdk8 #737

2016-07-06 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3794: added stream / table names as prefix to print / writeAsText -- [...truncated 3356 lines...] kafka.message.ByteBufferMessageSetTest > testValidBytes PASSED

[jira] [Commented] (KAFKA-3857) Additional log cleaner metrics

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

[GitHub] kafka pull request #1593: KAFKA-3857 Additional log cleaner metrics

2016-07-06 Thread kiranptivo
GitHub user kiranptivo opened a pull request: https://github.com/apache/kafka/pull/1593 KAFKA-3857 Additional log cleaner metrics Fixes KAFKA-3857 Changes proposed in this pull request: The following additional log cleaner metrics have been added. 1. num-runs:

[jira] [Commented] (KAFKA-3929) Add prefix for underlying clients configs in StreamConfig

2016-07-06 Thread Ishita Mandhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3929?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15364802#comment-15364802 ] Ishita Mandhan commented on KAFKA-3929: --- So have a function say appConfigsWithPrefix that is called

[jira] [Updated] (KAFKA-3452) Support session windows besides time interval windows

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3452?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3452: - Issue Type: New Feature (was: Sub-task) Parent: (was: KAFKA-2590) > Support session

[jira] [Updated] (KAFKA-3262) Make KafkaStreams debugging friendly

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3262?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3262: - Issue Type: Bug (was: Sub-task) Parent: (was: KAFKA-2590) > Make KafkaStreams

[jira] [Updated] (KAFKA-3185) Allow users to cleanup internal data

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3185?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3185: - Issue Type: Bug (was: Sub-task) Parent: (was: KAFKA-2590) > Allow users to cleanup

[jira] [Updated] (KAFKA-3184) Add Checkpoint for In-memory State Store

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3184?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3184: - Issue Type: Improvement (was: Sub-task) Parent: (was: KAFKA-2590) > Add Checkpoint

[jira] [Updated] (KAFKA-3183) Add metrics for persistent store caching layer

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3183: - Issue Type: Bug (was: Sub-task) Parent: (was: KAFKA-2590) > Add metrics for

[jira] [Updated] (KAFKA-3101) Optimize Aggregation Outputs

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3101?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3101: - Issue Type: Improvement (was: Sub-task) Parent: (was: KAFKA-2590) > Optimize

[jira] [Commented] (KAFKA-3794) Add Stream / Table prefix in print functions

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

[GitHub] kafka pull request #1577: KAFKA-3794: added stream / table names as prefix t...

2016-07-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1577 --- 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] [Resolved] (KAFKA-3794) Add Stream / Table prefix in print functions

2016-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3794?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3794. -- Resolution: Fixed Fix Version/s: 0.10.1.0 Issue resolved by pull request 1577

[jira] [Commented] (KAFKA-3809) Auto-generate documentation for topic-level configuration

2016-07-06 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15364686#comment-15364686 ] James Cheng commented on KAFKA-3809: [~ijuma], [~ewencp], would either of you be able to help me find

Re: Parallelisation factor in kafka streams

2016-07-06 Thread Matthias J. Sax
Hi Jeyhun, the number of partitions determine the number of tasks within a Kafka Streams application and thus, the maximum number of parallelism for your application. For more details see http://docs.confluent.io/3.0.0/streams/architecture.html#parallelism-model You can set the number of

[jira] [Work started] (KAFKA-3931) kafka.api.PlaintextConsumerTest.testPatternUnsubscription transient failure

2016-07-06 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3931?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3931 started by Vahid Hashemian. -- > kafka.api.PlaintextConsumerTest.testPatternUnsubscription transient

Build failed in Jenkins: kafka-trunk-jdk8 #736

2016-07-06 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-3802; log mtimes reset on broker restart / shutdown -- [...truncated 4384 lines...] kafka.utils.ByteBoundedBlockingQueueTest > testByteBoundedBlockingQueue STARTED

[jira] [Created] (KAFKA-3931) kafka.api.PlaintextConsumerTest.testPatternUnsubscription transient failure

2016-07-06 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-3931: -- Summary: kafka.api.PlaintextConsumerTest.testPatternUnsubscription transient failure Key: KAFKA-3931 URL: https://issues.apache.org/jira/browse/KAFKA-3931

[jira] [Updated] (KAFKA-3931) kafka.api.PlaintextConsumerTest.testPatternUnsubscription transient failure

2016-07-06 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3931?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-3931: --- Issue Type: Sub-task (was: Bug) Parent: KAFKA-2054 >

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

2016-07-06 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-3802; log mtimes reset on broker restart / shutdown -- [...truncated 3435 lines...] kafka.log.LogCleanerIntegrationTest > cleanerTest[0] PASSED

Parallelisation factor in kafka streams

2016-07-06 Thread Jeyhun Karimov
Hi community, How can I set parallelisation factor in kafka streams? Is it related with the number of partitions within topics? Cheers Jeyhun -- -Cheers Jeyhun

Build failed in Jenkins: kafka-0.10.0-jdk7 #143

2016-07-06 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-3802; log mtimes reset on broker restart / shutdown -- [...truncated 4025 lines...] kafka.log.BrokerCompressionTest > testBrokerSideCompression[8] PASSED

[GitHub] kafka-site issue #16: Update design.html

2016-07-06 Thread ijuma
Github user ijuma commented on the issue: https://github.com/apache/kafka-site/pull/16 Thank you, can you close this PR then? --- 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

[GitHub] kafka-site issue #16: Update design.html

2016-07-06 Thread nihed
Github user nihed commented on the issue: https://github.com/apache/kafka-site/pull/16 Done, https://github.com/apache/kafka/pull/1592 --- 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

[GitHub] kafka pull request #1592: Update design.html

2016-07-06 Thread nihed
GitHub user nihed opened a pull request: https://github.com/apache/kafka/pull/1592 Update design.html Since 0.9.0.1 Configuration parameter log.cleaner.enable is now true by default. You can merge this pull request into a Git repository by running: $ git pull

Re: [VOTE] KIP-60: Make Java client class loading more flexible

2016-07-06 Thread Rajini Sivaram
Hi all, The vote has passed with 3 binding +1s and 1 non-binding +1. Many thanks to those who voted. If you do have any more comments or suggestions, please do add them. I will update the KIP page and rebase the PR. On Tue, Jul 5, 2016 at 12:03 AM, Harsha Chintalapani wrote:

[jira] [Updated] (KAFKA-3802) log mtimes reset on broker restart

2016-07-06 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3802?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3802: --- Assignee: Moritz Siuts > log mtimes reset on broker restart > -- > >

[jira] [Updated] (KAFKA-3802) log mtimes reset on broker restart

2016-07-06 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3802?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3802: --- Resolution: Fixed Status: Resolved (was: Patch Available) > log mtimes reset on broker

[jira] [Commented] (KAFKA-3802) log mtimes reset on broker restart

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

[GitHub] kafka pull request #1497: KAFKA-3802 log mtimes reset on broker restart / sh...

2016-07-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1497 --- 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-site issue #16: Update design.html

2016-07-06 Thread ijuma
Github user ijuma commented on the issue: https://github.com/apache/kafka-site/pull/16 Thanks for the PR. The file you edited is just a copy from the following file: https://github.com/apache/kafka/blob/trunk/docs/design.html Can you please file a PR against the

Kafka - offset preservation

2016-07-06 Thread Pawel Huszcza
Hello, I tried every different property I can think of - I have set ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG = true; ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG = 1000, but I have also tried with many different values And still every time I start my consumer - I am receiving all the messages

[GitHub] kafka-site pull request #16: Update design.html

2016-07-06 Thread nihed
GitHub user nihed opened a pull request: https://github.com/apache/kafka-site/pull/16 Update design.html Hi, Since 0.9.0.1 Configuration parameter log.cleaner.enable is true by default. so I updated the documentation. Regards, You can merge this pull request

Re: NPE using GenericAvroSerde to deserialize

2016-07-06 Thread Philippe Derome
This item is moved to confluent google group. On Wed, Jul 6, 2016 at 6:25 AM, Philippe Derome wrote: > Please ignore until I become quite more specific about my code usage (will > try to recover the same code I used). In the mean time, I managed to > serialize into output

[jira] [Commented] (KAFKA-3910) Cyclic schema support in ConnectSchema and SchemaBuilder

2016-07-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15364188#comment-15364188 ] ASF GitHub Bot commented on KAFKA-3910: --- GitHub user johnhofman reopened a pull request:

[jira] [Commented] (KAFKA-3910) Cyclic schema support in ConnectSchema and SchemaBuilder

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

[GitHub] kafka pull request #1582: KAFKA-3910: Cyclic schema support in ConnectSchema...

2016-07-06 Thread johnhofman
GitHub user johnhofman reopened a pull request: https://github.com/apache/kafka/pull/1582 KAFKA-3910: Cyclic schema support in ConnectSchema and SchemaBuilder This feature uses a FutureSchema as a placeholder to be resolved later. Resolution is attempted whenever a ConnectSchema is

[GitHub] kafka pull request #1582: KAFKA-3910: Cyclic schema support in ConnectSchema...

2016-07-06 Thread johnhofman
Github user johnhofman closed the pull request at: https://github.com/apache/kafka/pull/1582 --- 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] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-06 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15364141#comment-15364141 ] Andy Coates commented on KAFKA-3919: [~junrao] My understanding was that the offset index looks at the

Re: KStream: KTable-KTable leftJoin with key only on RHS of join generates null in joined table

2016-07-06 Thread Philippe Derome
thanks, I understand that it's not modelled the same way as database joins. Let's take an example of inner join of very small population set (NBA rookies or US senators) with larger table (data on zip codes). Let's assume we want to identify the crime rate of zip codes where current senators live

Re: NPE using GenericAvroSerde to deserialize

2016-07-06 Thread Philippe Derome
Please ignore until I become quite more specific about my code usage (will try to recover the same code I used). In the mean time, I managed to serialize into output topic using a JsonSerde, which was straightforward. On Wed, Jul 6, 2016 at 5:26 AM, Michael Noll wrote: >

[jira] [Commented] (KAFKA-3758) KStream job fails to recover after Kafka broker stopped

2016-07-06 Thread Strong Liu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3758?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15364085#comment-15364085 ] Strong Liu commented on KAFKA-3758: --- I run into same problem with the

Re: NPE using GenericAvroSerde to deserialize

2016-07-06 Thread Michael Noll
Phil, > I then specify a Serde (new GenericAvroSerde) as value > deserializer when outputting to topic via table.to method. I suppose that was a typo, and you actually meant "as a value *serializer*", right? On Tue, Jul 5, 2016 at 11:55 PM, Philippe Derome wrote: > This

Re: Kafka Streams - production use

2016-07-06 Thread Michael Noll
Dirk, we included the note "be careful when using Kafka Streams in production" because Kafka Streams as shipped in Kafka 0.10.0.0 is the first-ever release of Kafka Streams. In practice, users are running Streams applications in a variety of stages -- some are doing pilots or evaluations, some

[jira] [Assigned] (KAFKA-3856) Move inner classes accessible only functions in TopologyBuilder out of public APIs

2016-07-06 Thread Jeyhun Karimov (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3856?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeyhun Karimov reassigned KAFKA-3856: - Assignee: Jeyhun Karimov > Move inner classes accessible only functions in

[jira] [Commented] (KAFKA-3836) KStreamReduce and KTableReduce should not pass nulls to Deserializers

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

[GitHub] kafka pull request #1591: KAFKA-3836: KStreamReduce and KTableReduce should ...

2016-07-06 Thread jeyhunkarimov
GitHub user jeyhunkarimov opened a pull request: https://github.com/apache/kafka/pull/1591 KAFKA-3836: KStreamReduce and KTableReduce should not pass nulls to Deserializers Minor changes to check null changes. You can merge this pull request into a Git repository by running:

Re: [jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-06 Thread Andrew Coates
[~junrao] i thought that the indexer took the offset of the first record in a compressed set. Looking at `LogSegment.recover` in the 0.9.0.1 code base that does indeed seen to be the case. I haven't dumped the offsets again, but I can of you still need it? On Wed, 6 Jul 2016, 09:04 Andrew

[jira] [Updated] (KAFKA-3930) IPv6 address can't used as ObjectName

2016-07-06 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3930?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3930: --- Fix Version/s: 0.10.1.0 > IPv6 address can't used as ObjectName >

[jira] [Created] (KAFKA-3930) IPv6 address can't used as ObjectName

2016-07-06 Thread wateray (JIRA)
wateray created KAFKA-3930: -- Summary: IPv6 address can't used as ObjectName Key: KAFKA-3930 URL: https://issues.apache.org/jira/browse/KAFKA-3930 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-3836) KStreamReduce and KTableReduce should not pass nulls to Deserializers

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

Re: [jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-06 Thread Andrew Coates
[~junrao] will double check On Tue, 5 Jul 2016, 18:53 Jun Rao (JIRA), wrote: > > [ > https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15362865#comment-15362865 > ] > > Jun Rao commented on KAFKA-3919: >

[GitHub] kafka pull request #1569: Kafka 3836: KStreamReduce and KTableReduce should ...

2016-07-06 Thread jeyhunkarimov
Github user jeyhunkarimov closed the pull request at: https://github.com/apache/kafka/pull/1569 --- 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

[GitHub] kafka pull request #1585: KAFKA-3836: KStreamReduce and KTableReduce should ...

2016-07-06 Thread jeyhunkarimov
Github user jeyhunkarimov closed the pull request at: https://github.com/apache/kafka/pull/1585 --- 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

[jira] [Commented] (KAFKA-3480) Autogenerate metrics documentation

2016-07-06 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3480?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15363943#comment-15363943 ] James Cheng commented on KAFKA-3480: PR is here: https://github.com/apache/kafka/pull/1202 >

[GitHub] kafka pull request #1586: Kafka-3836: KStreamReduce and KTableReduce should ...

2016-07-06 Thread jeyhunkarimov
Github user jeyhunkarimov closed the pull request at: https://github.com/apache/kafka/pull/1586 --- 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

[jira] [Updated] (KAFKA-3480) Autogenerate metrics documentation

2016-07-06 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3480?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] James Cheng updated KAFKA-3480: --- Reviewer: Jason Gustafson Status: Patch Available (was: In Progress) [~hachikuji], this is