Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Damian Guy
Eno, re: GlobalKTable - yeah that seems fine. On Tue, 11 Apr 2017 at 14:18 Eno Thereska wrote: > About GlobalKTables, I suppose there is no reason why they cannot also use > this KIP for consistency, e.g., today you have: > > public GlobalKTable

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Matthias J. Sax
Not sure, if we are on the same page already? > "A __store__ can be queryable whether is't materialized or not" This does not make sense -- there is nothing like a non-materialized store -- only non-materialized KTables. > "Yes, there is nothing that will prevent users from querying internally

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Matthias J. Sax
+1 on including GlobalKTable But I am not sure about the materialization / queryable question. For full consistency, all KTables should be queryable nevertheless if they are materialized or not. -- Maybe this is a second step though (even if I would like to get this done right away) If we don't

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Eno Thereska
Hi Matthias, See my note: "A store can be queryable whether it's materialized or not". I think we're on the same page. Stores with an internal name are also queryable. I'm just pointing out that. although that is the case today and with this KIP, I don't think we have an obligation to make

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Eno Thereska
Hi Matthias, > On 11 Apr 2017, at 09:41, Matthias J. Sax wrote: > > Not sure, if we are on the same page already? > >> "A __store__ can be queryable whether is't materialized or not" > > This does not make sense -- there is nothing like a non-materialized > store --

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Matthias J. Sax
+1 On 4/11/17 10:34 AM, Eno Thereska wrote: > Hi Matthias, > > >> On 11 Apr 2017, at 09:41, Matthias J. Sax wrote: >> >> Not sure, if we are on the same page already? >> >>> "A __store__ can be queryable whether is't materialized or not" >> >> This does not make sense --

[jira] [Comment Edited] (KAFKA-4988) JVM crash when running on Alpine Linux

2017-04-11 Thread Murad M (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4988?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15964809#comment-15964809 ] Murad M edited comment on KAFKA-4988 at 4/11/17 6:55 PM: - Just faced same problem

[jira] [Commented] (KAFKA-4988) JVM crash when running on Alpine Linux

2017-04-11 Thread Murad M (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4988?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15964809#comment-15964809 ] Murad M commented on KAFKA-4988: Just faced same problem here. Running application in fully blown ubuntu

[jira] [Updated] (KAFKA-4818) Implement transactional clients

2017-04-11 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4818?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta updated KAFKA-4818: Description: This covers the implementation of the producer and consumer to support transactions,

[jira] [Updated] (KAFKA-4818) Implement transactional clients

2017-04-11 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4818?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta updated KAFKA-4818: Description: This covers the implementation of the producer and consumer to support transactions.

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Eno Thereska
KIP updated, thank you. Eno > On 11 Apr 2017, at 08:23, Damian Guy wrote: > > Eno, re: GlobalKTable - yeah that seems fine. > > On Tue, 11 Apr 2017 at 14:18 Eno Thereska wrote: > >> About GlobalKTables, I suppose there is no reason why they

[jira] [Work started] (KAFKA-5038) running multiple kafka streams instances causes one or more instance to get into file contention

2017-04-11 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5038?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5038 started by Eno Thereska. --- > running multiple kafka streams instances causes one or more instance to get >

[jira] [Assigned] (KAFKA-5038) running multiple kafka streams instances causes one or more instance to get into file contention

2017-04-11 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5038?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reassigned KAFKA-5038: --- Assignee: Eno Thereska > running multiple kafka streams instances causes one or more

[jira] [Commented] (KAFKA-5038) running multiple kafka streams instances causes one or more instance to get into file contention

2017-04-11 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5038?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15964883#comment-15964883 ] Eno Thereska commented on KAFKA-5038: - [~btirumala] thank you for posting. Looks like it could be a

[jira] [Commented] (KAFKA-4930) Connect Rest API allows creating connectors with an empty name

2017-04-11 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15964886#comment-15964886 ] Sönke Liebau commented on KAFKA-4930: - [~gwenshap]: Did you have a chance yet to have a look at my PR?

[GitHub] kafka pull request #2841: KAFKA-5038: Catch exception

2017-04-11 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/2841 KAFKA-5038: Catch exception You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka KAFKA-5038 Alternatively you can review and

[jira] [Assigned] (KAFKA-4818) Implement transactional producer

2017-04-11 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4818?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta reassigned KAFKA-4818: --- Assignee: Apurva Mehta (was: Guozhang Wang) > Implement transactional producer >

[jira] [Commented] (KAFKA-4967) java.io.EOFException Error while committing offsets

2017-04-11 Thread Upendra Yadav (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4967?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15964757#comment-15964757 ] Upendra Yadav commented on KAFKA-4967: -- I solved this issue by calling 2nd time commitOffset.

[GitHub] kafka pull request #2840: KAFKA-4818: Exactly once transactional clients

2017-04-11 Thread apurvam
GitHub user apurvam opened a pull request: https://github.com/apache/kafka/pull/2840 KAFKA-4818: Exactly once transactional clients You can merge this pull request into a Git repository by running: $ git pull https://github.com/apurvam/kafka

[jira] [Commented] (KAFKA-4818) Implement transactional clients

2017-04-11 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4818?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15964765#comment-15964765 ] ASF GitHub Bot commented on KAFKA-4818: --- GitHub user apurvam opened a pull request:

[GitHub] kafka pull request #2842: MINOR: findbugs should generate XML reports

2017-04-11 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/2842 MINOR: findbugs should generate XML reports You can merge this pull request into a Git repository by running: $ git pull https://github.com/cmccabe/kafka findbugs-xml Alternatively you can

[GitHub] kafka pull request #2731: MINOR: Make LeaderAndIsr immutable case class.

2017-04-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2731 --- 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] [Updated] (KAFKA-5056) Shuffling of partitions in old consumer fetch requests removed

2017-04-11 Thread Todd Palino (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Todd Palino updated KAFKA-5056: --- Summary: Shuffling of partitions in old consumer fetch requests removed (was: Shuffling of

[jira] [Commented] (KAFKA-5056) Shuffling of partitions in old consume fetch requests removed

2017-04-11 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15965198#comment-15965198 ] ASF GitHub Bot commented on KAFKA-5056: --- GitHub user toddpalino opened a pull request:

[GitHub] kafka pull request #2843: KAFKA-5056: Add shuffling of FetchRequest requestI...

2017-04-11 Thread toddpalino
GitHub user toddpalino opened a pull request: https://github.com/apache/kafka/pull/2843 KAFKA-5056: Add shuffling of FetchRequest requestInfo back to old consumer KIP-74 removed the shuffle of requestInfo from the FetchRequest constructor, moving the logic to the replica fetcher

[jira] [Updated] (KAFKA-5056) Shuffling of partitions in old consume fetch requests removed

2017-04-11 Thread Todd Palino (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Todd Palino updated KAFKA-5056: --- Reviewer: Joel Koshy Status: Patch Available (was: In Progress) > Shuffling of partitions in

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

2017-04-11 Thread Apache Jenkins Server
See

Re: [VOTE] 0.10.2.1 RC0

2017-04-11 Thread Steven Schlansker
> On Apr 7, 2017, at 5:12 PM, Gwen Shapira wrote: > > Hello Kafka users, developers and client-developers, > > This is the first candidate for the release of Apache Kafka 0.10.2.1. This > is a bug fix release and it includes fixes and improvements from 24 JIRAs > (including

[jira] [Commented] (KAFKA-4997) Issue with running kafka-acls.sh when using SASL between Kafka and ZK

2017-04-11 Thread Shrikant (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15965186#comment-15965186 ] Shrikant commented on KAFKA-4997: - Rajini, Thanks for response. Figure out the issue, all the kafka node

[jira] [Created] (KAFKA-5056) Shuffling of partitions in old consume fetch requests removed

2017-04-11 Thread Todd Palino (JIRA)
Todd Palino created KAFKA-5056: -- Summary: Shuffling of partitions in old consume fetch requests removed Key: KAFKA-5056 URL: https://issues.apache.org/jira/browse/KAFKA-5056 Project: Kafka

[jira] [Resolved] (KAFKA-5013) Fail the build when findbugs fails

2017-04-11 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5013?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5013. Resolution: Fixed Fix Version/s: 0.11.0.0 Issue resolved by pull request 2805

[GitHub] kafka pull request #2805: KAFKA-5013. Fail the build when findbugs fails

2017-04-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2805 --- 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-5013) Fail the build when findbugs fails

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

Re: [VOTE] 0.10.2.1 RC0

2017-04-11 Thread Gwen Shapira
FYI: I just updated the upgrade notes with Streams changes: http://kafka.apache.org/documentation/#gettingStarted On Fri, Apr 7, 2017 at 5:12 PM, Gwen Shapira wrote: > Hello Kafka users, developers and client-developers, > > This is the first candidate for the release of

Re: [VOTE] 0.10.2.1 RC0

2017-04-11 Thread Gwen Shapira
Thanks for the feedback. I'm not super familiar with the inner workings of Apache's Maven repos, so I can't explain why we do things the way we do. I followed the same process on all Apache projects I was on (Kafka, Sqoop, Flume). Do you know projects that do things the way you suggested? Either

[jira] [Resolved] (KAFKA-4997) Issue with running kafka-acls.sh when using SASL between Kafka and ZK

2017-04-11 Thread Shrikant (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shrikant resolved KAFKA-4997. - Resolution: Not A Problem > Issue with running kafka-acls.sh when using SASL between Kafka and ZK >

[GitHub] kafka pull request #2824: MINOR: Added changes in 0.10.2.1

2017-04-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2824 --- 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] [Work started] (KAFKA-5056) Shuffling of partitions in old consume fetch requests removed

2017-04-11 Thread Todd Palino (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5056 started by Todd Palino. -- > Shuffling of partitions in old consume fetch requests removed >

Re: [VOTE] 0.10.2.1 RC0

2017-04-11 Thread Gwen Shapira
Wrong link :) http://kafka.apache.org/documentation/#upgrade and http://kafka.apache.org/documentation/streams#streams_api_changes_0102 On Tue, Apr 11, 2017 at 5:57 PM, Gwen Shapira wrote: > FYI: I just updated the upgrade notes with Streams changes: >

[GitHub] kafka pull request #2844: HOTFIX: HTML formatting error in upgrade docs from...

2017-04-11 Thread gwenshap
GitHub user gwenshap opened a pull request: https://github.com/apache/kafka/pull/2844 HOTFIX: HTML formatting error in upgrade docs from pr-2824 Already fixed in the website github You can merge this pull request into a Git repository by running: $ git pull

[jira] [Created] (KAFKA-5057) "Big Message Log"

2017-04-11 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-5057: --- Summary: "Big Message Log" Key: KAFKA-5057 URL: https://issues.apache.org/jira/browse/KAFKA-5057 Project: Kafka Issue Type: Bug Reporter: Gwen

Topic Creation programatically

2017-04-11 Thread Zishan Ali Saiyed
Hi Team, I am using kafka integrated with java client. I have a question " What is the best practice to create topic using programmatically or using CLI topic creation command?" Thanks, Zishan Ali

Can't start zookeeper and Kafka server

2017-04-11 Thread Amose Cd
Hi kafka, I placed my kafka source in "*C:\kafka_2.9.1-0.8.2.2*" ,Open command prompt as Administrator . moved to " *C:\kafka_2.9.1-0.8.2.2\bin\windows*" . now execute command "* zookeeper-server-start.bat ..\..\config\zookeeper.properties*" im getting following error. *The system cannot

[jira] [Commented] (KAFKA-5011) Replica fetchers may need to down-convert messages during a selective message format upgrade

2017-04-11 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5011?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15964994#comment-15964994 ] Joel Koshy commented on KAFKA-5011: --- Yes that is correct - it is rare. I think it's reasonable to close

[jira] [Assigned] (KAFKA-5054) ChangeLoggingKeyValueByteStore delete and putIfAbsent should be synchronized

2017-04-11 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5054?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy reassigned KAFKA-5054: - Assignee: Damian Guy > ChangeLoggingKeyValueByteStore delete and putIfAbsent should be

[GitHub] kafka-site pull request #54: Fix typo - Acls Examples, Adding or removing a ...

2017-04-11 Thread sunnykrGupta
GitHub user sunnykrGupta opened a pull request: https://github.com/apache/kafka-site/pull/54 Fix typo - Acls Examples, Adding or removing a principal as producer … …or consumer You can merge this pull request into a Git repository by running: $ git pull

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Damian Guy
Hi Eno, Thanks for the update. I agree with what Matthias said. I wonder if the KIP should talk less about materialization and more about querying? After all, that is what is being provided from an end-users perspective. I think if no store name is provided users would still be able to query the

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-04-11 Thread Damian Guy
Hi Onur, It was in my previous email. But here it is again. 1. Better rebalance timing. We will try to rebalance only when all the consumers in a group have joined. The challenge would be someone has to define what does ALL consumers

[jira] [Created] (KAFKA-5054) ChangeLoggingKeyValueByteStore delete and putIfAbsent should be synchronized

2017-04-11 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-5054: - Summary: ChangeLoggingKeyValueByteStore delete and putIfAbsent should be synchronized Key: KAFKA-5054 URL: https://issues.apache.org/jira/browse/KAFKA-5054 Project: Kafka

Re: [DISCUSS] KIP 130: Expose states of active tasks to KafkaStreams public API

2017-04-11 Thread Damian Guy
Hi Florian, Thanks for the updates. The KIP is looking good. Cheers, Damian On Fri, 7 Apr 2017 at 22:41 Matthias J. Sax wrote: > What about KafkaStreams#toString() method? > > I think, we want to deprecate it as with KIP-120 and the changes of this > KIP, is gets

Credentials for Confluence

2017-04-11 Thread Stephane Maarek
Hi, I’d like to create a KIP on Confluence but according to Matthias Sax I need credentials. Can you please provide me some? Looking forward to redacting my first KIP :) Regards, Stephane

[jira] [Commented] (KAFKA-3355) GetOffsetShell command doesn't work with SASL enabled Kafka

2017-04-11 Thread Mohammed amine GARMES (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15963998#comment-15963998 ] Mohammed amine GARMES commented on KAFKA-3355: -- Hello [~fredji], this fix is for kafka

[GitHub] kafka pull request #2825: KAFKA-5043 : Add FindCoordinatorRPC stub and updat...

2017-04-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2825 --- 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-5043) Add FindCoordinatorRequest RPC stub and update InitPidRequest for KIP-98

2017-04-11 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5043?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5043. Resolution: Fixed Fix Version/s: 0.11.0.0 Issue resolved by pull request 2825

[GitHub] kafka-site issue #54: Fix typo - Acls Examples, Adding or removing a princip...

2017-04-11 Thread omkreddy
Github user omkreddy commented on the issue: https://github.com/apache/kafka-site/pull/54 LGTM --- 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

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Eno Thereska
Hi Damian, Thanks. I agree, I'll adjust the tone so it's more about querying, while materialisation is an internal concept. If no store name is provided, the user would still be able to discover the store, however we are not making any strong guarantees in that case, since after all it i an

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

2017-04-11 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Fix typo in consumer ACL example -- [...truncated 116.86 KB...] kafka.api.ProducerFailureHandlingTest > testTooLargeRecordWithAckOne STARTED

[GitHub] kafka pull request #2661: kafka-4866: Kafka console consumer property is ign...

2017-04-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2661 --- 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-4866) Kafka console consumer property is ignored

2017-04-11 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4866?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4866. Resolution: Fixed Fix Version/s: 0.11.0.0 Issue resolved by pull request 2661

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

2017-04-11 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4866; Console consumer `print.value` property is ignored -- [...truncated 112.75 KB...] kafka.api.PlaintextProducerSendTest >

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Michael Noll
Thanks for the updates, Eno! In addition to what has already been said: We should also explicitly mention that this KIP is not touching GlobalKTable. I'm sure that some users will throw KTable and GlobalKTable into one conceptual "it's all tables!" bucket and then wonder how the KIP might

[GitHub] kafka pull request #2839: Fix typo - Acls Examples, Adding or removing a pri...

2017-04-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2839 --- 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

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

2017-04-11 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5043; Rename GroupCoordinator to FindCoordinator (KIP-98) [ismael] MINOR: Document ordering contract of iterator for window stores and --

[GitHub] kafka pull request #2839: Fix typo - Acls Examples, Adding or removing a pri...

2017-04-11 Thread sunnykrGupta
GitHub user sunnykrGupta opened a pull request: https://github.com/apache/kafka/pull/2839 Fix typo - Acls Examples, Adding or removing a principal as producer … …or consumer You can merge this pull request into a Git repository by running: $ git pull

[GitHub] kafka-site issue #54: Fix typo - Acls Examples, Adding or removing a princip...

2017-04-11 Thread sunnykrGupta
Github user sunnykrGupta commented on the issue: https://github.com/apache/kafka-site/pull/54 Sure. Did PR https://github.com/apache/kafka/pull/2839 . :) --- 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

[jira] [Commented] (KAFKA-5043) Add FindCoordinatorRequest RPC stub and update InitPidRequest for KIP-98

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

[GitHub] kafka-site issue #54: Fix typo - Acls Examples, Adding or removing a princip...

2017-04-11 Thread ijuma
Github user ijuma commented on the issue: https://github.com/apache/kafka-site/pull/54 Thanks for the PR, can you do a PR to the code repo https://github.com/apache/kafka/tree/trunk/docs? --- If your project is set up for it, you can reply to this email and have your reply appear on

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Eno Thereska
Hi Michael, comments inline: > On 11 Apr 2017, at 03:25, Michael Noll wrote: > > Thanks for the updates, Eno! > > In addition to what has already been said: We should also explicitly > mention that this KIP is not touching GlobalKTable. I'm sure that some > users will

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-11 Thread Eno Thereska
About GlobalKTables, I suppose there is no reason why they cannot also use this KIP for consistency, e.g., today you have: public GlobalKTable globalTable(final Serde keySerde, final Serde valSerde,