Re: [VOTE] Add REST Server to Apache Kafka

2016-10-26 Thread Jaikiran Pai
-1. I would personally like Kafka core to be limited to the core aspects. I consider REST server like this more a tooling thing than a core aspect of Kafka and would like it to be outside of Kafka. -Jaikiran On Wednesday 26 October 2016 02:46 AM, Harsha Chintalapani wrote: Hi All,

Re: [DISCUSS] KIP-81: Max in-flight fetches

2016-10-26 Thread Jason Gustafson
Hey Mickael, Thanks for picking this up and sorry for the late comment. In the proposed changes section, you have the following: Update Fetcher.java to check the number of existing in-flight fetches (this > is already tracked by numInFlightFetches) before initiating new fetch > requests in

[jira] [Commented] (KAFKA-4340) Change the default value of log.message.timestamp.difference.max.ms to the same as log.retention.ms

2016-10-26 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4340?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15610518#comment-15610518 ] ASF GitHub Bot commented on KAFKA-4340: --- GitHub user becketqin opened a pull request:

[GitHub] kafka pull request #2071: KAFKA-4340: Change default message.timestamp.diffe...

2016-10-26 Thread becketqin
GitHub user becketqin opened a pull request: https://github.com/apache/kafka/pull/2071 KAFKA-4340: Change default message.timestamp.difference.max.ms to the same as log.retention.ms You can merge this pull request into a Git repository by running: $ git pull

Re: handleFetchRequest throw exception

2016-10-26 Thread Json Tu
it make the cluster can not provide normal service,which leades some producer or fetch fail for a long time before I restart current broker. this error may be come from some formerly fetch operation which contain this partition,which leads many fetch response error. The delayFetch's

[jira] [Commented] (KAFKA-3536) ReplicaFetcherThread should not log errors when leadership changes

2016-10-26 Thread Json Tu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3536?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15608708#comment-15608708 ] Json Tu commented on KAFKA-3536: Thank you,But I think I can’t ignore it,because it make the cluster can

[VOTE] KIP-85: Dynamic JAAS configuration for Kafka clients

2016-10-26 Thread Rajini Sivaram
I would like to initiate the voting process for KIP-85: Dynamic JAAS configuration for Kafka Clients: https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients This KIP enables Java clients to connect to Kafka using SASL without a physical jaas.conf

[GitHub] kafka pull request #2067: Add missing zookeeper parameter

2016-10-26 Thread thbar
GitHub user thbar opened a pull request: https://github.com/apache/kafka/pull/2067 Add missing zookeeper parameter You can merge this pull request into a Git repository by running: $ git pull https://github.com/thbar/kafka patch-1 Alternatively you can review and apply these

[jira] [Commented] (KAFKA-4161) Decouple flush and offset commits

2016-10-26 Thread Konrad Dowgird (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15608703#comment-15608703 ] Konrad Dowgird commented on KAFKA-4161: --- We have tested the above patch and it worked for us quite

Re: [VOTE] KIP-85: Dynamic JAAS configuration for Kafka clients

2016-10-26 Thread Harsha Chintalapani
Thanks for the KIP Rajini. This looks great. +1. -Harsha On Wed, Oct 26, 2016 at 8:26 AM Rajini Sivaram wrote: > I would like to initiate the voting process for KIP-85: Dynamic JAAS > configuration for Kafka Clients: > > >

[jira] [Comment Edited] (KAFKA-4348) On Mac OS, KafkaConsumer.poll returns 0 when there are still messages on Kafka server

2016-10-26 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4348?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15610383#comment-15610383 ] huxi edited comment on KAFKA-4348 at 10/27/16 2:20 AM: --- Seems that you do not set

Re: handleFetchRequest throw exception

2016-10-26 Thread Guozhang Wang
Json, As you mentioned yourself the "NotLeaderForPartitionException" thrown from getLeaderReplicaIfLocal should be caught in the end, and hence I'm not sure why the reported stack trace "ERROR: ..." throwing the NotLeaderForPartitionException should be seen from "tryComplete". Also I have checked

[GitHub] kafka-site issue #26: add trademark symbol on all pages plus longer footer m...

2016-10-26 Thread junrao
Github user junrao commented on the issue: https://github.com/apache/kafka-site/pull/26 Merged into kafka-site. --- 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

[GitHub] kafka-site issue #26: add trademark symbol on all pages plus longer footer m...

2016-10-26 Thread junrao
Github user junrao commented on the issue: https://github.com/apache/kafka-site/pull/26 Thanks for the patch. LGTM. Will remove the duplicated "are" in includes/_footer.htm. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as

[jira] [Commented] (KAFKA-4348) On Mac OS, KafkaConsumer.poll returns 0 when there are still messages on Kafka server

2016-10-26 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4348?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15610383#comment-15610383 ] huxi commented on KAFKA-4348: - Seems that you do not set 'auto.offset.reset' whose default value is 'latest'.

Re: handleFetchRequest throw exception

2016-10-26 Thread Json Tu
Thanks to guozhang. According to your suggestions,I found my new patch to kafka 0.9.0.0 may casue the problem, In delayedfetch.scala, I include import org.apache.kafka.common.errors.NotLeaderForPartitionException but not import kafka.common.NotLeaderForPartitionException for intelij auto

[jira] [Created] (KAFKA-4350) Can't mirror from Kafka 0.9 to Kafka 0.10.1

2016-10-26 Thread Emanuele Cesena (JIRA)
Emanuele Cesena created KAFKA-4350: -- Summary: Can't mirror from Kafka 0.9 to Kafka 0.10.1 Key: KAFKA-4350 URL: https://issues.apache.org/jira/browse/KAFKA-4350 Project: Kafka Issue Type:

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

2016-10-26 Thread stevejs
Github user stevejs closed the pull request at: https://github.com/apache/kafka/pull/1057 --- 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 #2065: KAFKA-4302: Simplify KTableSource

2016-10-26 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2065 KAFKA-4302: Simplify KTableSource KTableSource is always materialized since IQ: - removed flag KTableSource#materialized - removed MaterializedKTableSourceProcessor You can merge this

[jira] [Commented] (KAFKA-4302) Simplify KTableSource

2016-10-26 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4302?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15607559#comment-15607559 ] ASF GitHub Bot commented on KAFKA-4302: --- GitHub user mjsax opened a pull request:

Re: [DISCUSS] KIP-80: Kafka REST Server

2016-10-26 Thread Guozhang Wang
I find it a bit hard to just discuss "whether or not we should add a REST proxy into Apache Kafka repo" without discussing about "what should be included in the Apache Kafka repo", which, as people mentioned, was a grey-area and deserves ongoing discussions. So I would like to first throw my

[jira] [Updated] (KAFKA-4302) Simplify KTableSource

2016-10-26 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4302?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4302: --- Status: Patch Available (was: In Progress) > Simplify KTableSource > - >

[GitHub] kafka pull request #2068: MINOR: improve JavaDoc for Streams window retentio...

2016-10-26 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2068 MINOR: improve JavaDoc for Streams window retention time You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka

Re: [VOTE] KIP-85: Dynamic JAAS configuration for Kafka clients

2016-10-26 Thread Mickael Maison
+1 Great work Rajini. On Wed, Oct 26, 2016 at 5:29 PM, Harsha Chintalapani wrote: > Thanks for the KIP Rajini. This looks great. +1. > -Harsha > > On Wed, Oct 26, 2016 at 8:26 AM Rajini Sivaram > wrote: > >> I would like to initiate the voting

[GitHub] kafka pull request #2067: Add missing zookeeper parameter

2016-10-26 Thread thbar
Github user thbar closed the pull request at: https://github.com/apache/kafka/pull/2067 --- 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: [VOTE] KIP-85: Dynamic JAAS configuration for Kafka clients

2016-10-26 Thread Jim Jagielski
+1! > On Oct 26, 2016, at 11:26 AM, Rajini Sivaram > wrote: > > I would like to initiate the voting process for KIP-85: Dynamic JAAS > configuration for Kafka Clients: > >

Re: [VOTE] Add REST Server to Apache Kafka

2016-10-26 Thread Andrew Otto
-1 for http kafka client in core Although a read only management interface, perhaps via http, sounds kinda useful for things like health checks as mentioned. On Wed, Oct 26, 2016 at 2:00 PM, Zakee wrote: > -1 > > Thanks. > > On Oct 25, 2016, at 2:16 PM, Harsha

Re: [VOTE] Add REST Server to Apache Kafka

2016-10-26 Thread Guruditta Golani
+1 supporting this natively will help immensely in building operational tools on top of it and augment ease of use cases in large deployments. -Guru From: Shekar Tippur Sent: Wednesday, October 26, 2016 9:23 AM To: users Cc:

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-10-26 Thread Magnus Edenhill
Hi Renu, that is not completely true, the LZ4 compression codec was added without a MagicByte bump. (LZ4 might be a bad example though since this feature was added without updating the protocol docs..) Unless the broker needs the MagicByte internally (for translating old logs on disk or

[jira] [Assigned] (KAFKA-4272) Kafka Connect batch scripts are missing under `bin/windows/`

2016-10-26 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4272?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian reassigned KAFKA-4272: -- Assignee: Vahid Hashemian > Kafka Connect batch scripts are missing under

Re: [DISCUSS] KIP-84: Support SASL/SCRAM mechanisms

2016-10-26 Thread Jun Rao
Hi, Rajini, Thanks for the reply. 4. Implementation wise, it seems to me that it's simpler to read from the cache than reading directly from ZK since the config manager already propagates all config changes through ZK. Also, it's probably a good idea to limit the places in the code base that

Re: [VOTE] Add REST Server to Apache Kafka

2016-10-26 Thread Dana Powers
-1 On Wed, Oct 26, 2016 at 9:23 AM, Shekar Tippur wrote: > +1 > > Thanks

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-10-26 Thread Mayuresh Gharat
I see the reasoning Magnus described. If you check the docs https://kafka.apache.org/documentation#messageformat, it says : 1 byte "magic" identifier to allow format changes, value is 0 or 1 Moreover as per comments in the code : /** * The "magic" value * When magic value is 0, the message

[GitHub] kafka pull request #2068: MINOR: improve JavaDoc for Streams window retentio...

2016-10-26 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2068 --- 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-4349) Consumer group describe exception when group state is PreparingRebalance

2016-10-26 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-4349: -- Summary: Consumer group describe exception when group state is PreparingRebalance Key: KAFKA-4349 URL: https://issues.apache.org/jira/browse/KAFKA-4349 Project:

[jira] [Commented] (KAFKA-4349) Consumer group describe exception when group state is PreparingRebalance

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

[GitHub] kafka pull request #2070: KAFKA-4349: Handle 'PreparingRebalance' state in c...

2016-10-26 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2070 KAFKA-4349: Handle 'PreparingRebalance' state in consumer group describe The edge case where consumer group state is `PreparingRebalance` will be explicitly handled as the group assignment

[jira] [Commented] (KAFKA-2066) Replace FetchRequest / FetchResponse with their org.apache.kafka.common.requests equivalents

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

[GitHub] kafka pull request #2069: KAFKA-2066: Use client-side FetchRequest/FetchResp...

2016-10-26 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2069 KAFKA-2066: Use client-side FetchRequest/FetchResponse on server You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-2066

Re: [DISCUSS] KIP-68 Add a consumed log retention before log retention

2016-10-26 Thread Guozhang Wang
Hello Becket, I am not 100 percent sure I get your points, reading the first half of the paragraph I thought we were on the same page that "the committed offsets and the offsets the applications ( most likely the consumers) would like to tell the brokers to trim to, could be totally different",

[jira] [Commented] (KAFKA-4161) Decouple flush and offset commits

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

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-10-26 Thread Magnus Edenhill
2016-10-25 21:36 GMT+02:00 Nacho Solis : > I think you probably require a MagicByte bump if you expect correct > behavior of the system as a whole. > > From a client perspective you want to make sure that when you deliver a > message that the broker supports the

Re: [VOTE] KIP-47 - Add timestamp-based log deletion policy

2016-10-26 Thread Dong Lin
Hey Bill, I have some follow up questions after Jun's questions: - It seems that the consumer will need to write log.retention.min.timestamp periodically to zookeeper as dynamic configuration of the topic, so that broker can pick up log.retention.min.timestamp. However, this introduces

[jira] [Created] (KAFKA-4347) outputStreamId in Kafka spout

2016-10-26 Thread Rahim Dastar (JIRA)
Rahim Dastar created KAFKA-4347: --- Summary: outputStreamId in Kafka spout Key: KAFKA-4347 URL: https://issues.apache.org/jira/browse/KAFKA-4347 Project: Kafka Issue Type: Bug

Re: [VOTE] Add REST Server to Apache Kafka

2016-10-26 Thread Ofir Manor
-1 First, I think there was no technical discussion of the KIP details in the DISCUSS until now, just a binary yes/no. Real discussion just started with the excellent ten technical points from Ewen a few hours ago. Generally,I don't see how a standard REST makes sense for format-agnostic realtime

[GitHub] kafka pull request #2040: KAFKA-4161: prototype for exploring API change

2016-10-26 Thread shikhar
Github user shikhar closed the pull request at: https://github.com/apache/kafka/pull/2040 --- 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 #2066: MINOR: Increase zk connection timeout in tests for...

2016-10-26 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/2066 MINOR: Increase zk connection timeout in tests for client created in `KafkaServer` We had already made this change to the client created in `ZooKeeperTestHarness`. I last saw this failure when

Re: [DISCUSS] KIP-84: Support SASL/SCRAM mechanisms

2016-10-26 Thread Rajini Sivaram
Hi Jun, Thank you for reviewing the KIP. Answers below: 1. Yes, agree, Updated KIP. 2. User specifies a password and iteration count. kaka-configs.sh generates a random salt and then generates StoredKey and ServerKey for that password using the same message formatter implementation

[GitHub] kafka pull request #2054: kafka-4295: ConsoleConsumer does not delete the te...

2016-10-26 Thread amethystic
GitHub user amethystic reopened a pull request: https://github.com/apache/kafka/pull/2054 kafka-4295: ConsoleConsumer does not delete the temporary group in zookeeper Since consumer stop logic and zk node removal code are in separate threads, so when two threads execute in an

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-10-26 Thread Xavier Léauté
Does this mean that starting with V4 requests we would allow storing null messages in compacted topics? The KIP should probably clarify the behavior for null messages where the tombstone flag is not net. On Wed, Oct 26, 2016 at 1:32 AM Magnus Edenhill wrote: > 2016-10-25

[jira] [Commented] (KAFKA-4344) Exception when accessing partition, offset and timestamp in processor class

2016-10-26 Thread saiprasad mishra (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15609709#comment-15609709 ] saiprasad mishra commented on KAFKA-4344: - Sorry i was just writing the reply while you were

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

2016-10-26 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: improve JavaDoc for Streams window retention time -- [...truncated 14318 lines...] org.apache.kafka.streams.kstream.internals.KTableSourceTest >

Re: [DISCUSS] KIP-86: Configurable SASL callback handlers

2016-10-26 Thread Rajini Sivaram
Hi Jun, The main processing method in the callback handler is "handle(Callback[] callbacks)". This method is invoked by SaslClient either when the SaslClient is constructed or when Kafka's SASL authentication code invokes saslClient.evaluateChallenge(). These are always done under Subject.doAs()

[jira] [Commented] (KAFKA-4344) Exception when accessing partition, offset and timestamp in processor class

2016-10-26 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15609744#comment-15609744 ] Guozhang Wang commented on KAFKA-4344: -- Thanks [~saimishra], you also mentioned another issue on the

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-10-26 Thread Joel Koshy
A magic byte bump would be required for the addition of new fields; or removal of existing fields. Changing the interpretation of an existing field (e.g., switching from absolute to relative offsets) almost always needs a magic byte bump as well. One concern Nacho alluded to (I think) is if a

Re: [DISCUSS] KIP-86: Configurable SASL callback handlers

2016-10-26 Thread Jun Rao
Hi, Rajini, The javadoc of Subject.getSubjectAccessControlContext acc) says the following. So, are we depending on the correct ordering to get the right subject? Is there any issue if two Subject.doAs() are called

[jira] [Commented] (KAFKA-4349) Consumer group describe exception when group state is PreparingRebalance

2016-10-26 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4349?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15609777#comment-15609777 ] Jason Gustafson commented on KAFKA-4349: Good catch. I think this was also causing a streams

[jira] [Updated] (KAFKA-4180) Shared authentication with multiple active Kafka producers/consumers

2016-10-26 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4180?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Olson updated KAFKA-4180: Summary: Shared authentication with multiple active Kafka producers/consumers (was: Shared

[jira] [Commented] (KAFKA-4344) Exception when accessing partition, offset and timestamp in processor class

2016-10-26 Thread saiprasad mishra (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15609837#comment-15609837 ] saiprasad mishra commented on KAFKA-4344: - Yes it is fixed now. Will write to the mailing list

[jira] [Created] (KAFKA-4348) On Mac OS, KafkaConsumer.poll returns 0 when there are still messages on Kafka server

2016-10-26 Thread Yiquan Zhou (JIRA)
Yiquan Zhou created KAFKA-4348: -- Summary: On Mac OS, KafkaConsumer.poll returns 0 when there are still messages on Kafka server Key: KAFKA-4348 URL: https://issues.apache.org/jira/browse/KAFKA-4348

[jira] [Updated] (KAFKA-4348) On Mac OS, KafkaConsumer.poll returns 0 when there are still messages on Kafka server

2016-10-26 Thread Yiquan Zhou (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4348?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yiquan Zhou updated KAFKA-4348: --- Environment: Mac OS X EI Capitan, Java 1.8.0_111 (was: Max OS X EI Capitan, Java 1.8.0_111) > On

[jira] [Updated] (KAFKA-4348) On Mac OS, KafkaConsumer.poll returns 0 when there are still messages on Kafka server

2016-10-26 Thread Yiquan Zhou (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4348?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yiquan Zhou updated KAFKA-4348: --- Environment: Max OS X EI Capitan, Java 1.8.0_111 (was: Max OS X EI Capitan, Java 1.8.0_77) > On Mac

[jira] [Updated] (KAFKA-4348) On Mac OS, KafkaConsumer.poll returns 0 when there are still messages on Kafka server

2016-10-26 Thread Yiquan Zhou (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4348?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yiquan Zhou updated KAFKA-4348: --- Description: Steps to reproduce: 1. start the zookeeper and kafka server using the default properties

[jira] [Resolved] (KAFKA-4344) Exception when accessing partition, offset and timestamp in processor class

2016-10-26 Thread saiprasad mishra (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4344?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] saiprasad mishra resolved KAFKA-4344. - Resolution: Not A Bug This was all about kafka streams app not being started correctly

Re: [VOTE] KIP-85: Dynamic JAAS configuration for Kafka clients

2016-10-26 Thread Gwen Shapira
Woohoo! +1 (binding) On Wed, Oct 26, 2016 at 5:26 PM, Rajini Sivaram < rajinisiva...@googlemail.com> wrote: > I would like to initiate the voting process for KIP-85: Dynamic JAAS > configuration for Kafka Clients: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+ >

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-10-26 Thread Joel Koshy
I'm not sure why it would be useful, but it should be theoretically possible if the attribute bit alone is enough to mark a tombstone. OTOH, we could consider that as invalid if we wish. These are relevant details that I think should be added to the KIP. Also, in the few odd scenarios that I

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-10-26 Thread Mayuresh Gharat
+1 @Joel. I think a clear migration plan of upgrading and downgrading of server and clients along with handling of issues that Joel mentioned, on the KIP would be really great. Thanks, Mayuresh On Wed, Oct 26, 2016 at 3:31 PM, Joel Koshy wrote: > I'm not sure why it would

[jira] [Updated] (KAFKA-4349) Consumer group describe exception when group state is PreparingRebalance

2016-10-26 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4349?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-4349: --- Status: Patch Available (was: Open) > Consumer group describe exception when group state is

[jira] [Commented] (KAFKA-4344) Exception when accessing partition, offset and timestamp in processor class

2016-10-26 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15609681#comment-15609681 ] Guozhang Wang commented on KAFKA-4344: -- I wanted to verify how did you get the {{ProcessorContext}}

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-10-26 Thread Mayuresh Gharat
I agree with Magnus about deciding rules on when to update. If we can handle such changes by request versioning, it would be good to understand the use cases for bumping up magic byte or vice versa. Unless I am not understanding it correctly, having two versioning schemes seems little weird.

[jira] [Comment Edited] (KAFKA-4344) Exception when accessing partition, offset and timestamp in processor class

2016-10-26 Thread saiprasad mishra (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15609709#comment-15609709 ] saiprasad mishra edited comment on KAFKA-4344 at 10/26/16 9:21 PM: ---

[jira] [Commented] (KAFKA-4344) Exception when accessing partition, offset and timestamp in processor class

2016-10-26 Thread saiprasad mishra (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15609687#comment-15609687 ] saiprasad mishra commented on KAFKA-4344: - Thanks [~guozhang] for your time on testing this and

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

2016-10-26 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: improve JavaDoc for Streams window retention time -- [...truncated 14597 lines...] org.apache.kafka.streams.processor.internals.AbstractTaskTest >

Re: [VOTE] Add REST Server to Apache Kafka

2016-10-26 Thread Hugo Picado
-1 >From my experience, nothing good comes from merges similar to this one, as it has huge implications in terms of codebase maintenance, CD pipeline duration and stability as well as isolated testability. I don't see any particular worth benefit on having the REST server in the core repository

Re: [VOTE] Add REST Server to Apache Kafka

2016-10-26 Thread Shekar Tippur
+1 Thanks

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-10-26 Thread Renu Tewari
+1 @Magnus. It is also in line with traditional use of the magic field to indicate a change in the format of the message. Thus a change in the magic field indicates a different "schema" which in this case would reflect adding a new field, removing a field or changing the type of fields etc. The

Re: [VOTE] Add REST Server to Apache Kafka

2016-10-26 Thread Zakee
-1 Thanks. > On Oct 25, 2016, at 2:16 PM, Harsha Chintalapani wrote: > > Hi All, > We are proposing to have a REST Server as part of Apache Kafka > to provide producer/consumer/admin APIs. We Strongly believe having > REST server functionality with Apache Kafka will