[jira] [Updated] (KAFKA-4743) Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Jorge Quilcate (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4743?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Quilcate updated KAFKA-4743: -- Description: Add an external tool to reset Consumer Group offsets, and achieve rewind over the

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Gwen Shapira
As long as the CLI is a bit consistent? Like, not just adding 3 arguments and a JSON parser to the existing tool, right? On Tue, Feb 7, 2017 at 10:29 PM, Onur Karaman wrote: > I think it makes sense to just add the feature to kafka-consumer-groups.sh > > On Tue, Feb 7, 2017 at 10:24 PM, Gwen Shap

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Onur Karaman
I think it makes sense to just add the feature to kafka-consumer-groups.sh On Tue, Feb 7, 2017 at 10:24 PM, Gwen Shapira wrote: > Thanks for the KIP. I'm super happy about adding the capability. > > I hate the interface, though. It looks exactly like the replica > assignment tool. A tool everyon

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Gwen Shapira
Thanks for the KIP. I'm super happy about adding the capability. I hate the interface, though. It looks exactly like the replica assignment tool. A tool everyone loves so much that there are multiple projects, open and closed, that try to fix it. Can we swap it with something that looks a bit mor

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Matthias J. Sax
Jorge, can you please add your KIP to this table: https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals#KafkaImprovementProposals-KIPsunderdiscussion Thanks! -Matthias On 2/7/17 9:29 PM, Matthias J. Sax wrote: > Jorge, > > thanks for you KIP. I like it a lot and thin

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Onur Karaman
I've been meaning to suggest something very similar to this KIP. Something lacking in the KIP is under what scenarios the offset reset tool will run. Are all members of the group expected to be offline or can we override offsets while members of the group are live? This matters when factoring in t

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-07 Thread Matthias J. Sax
Damian, I am not strict about it either. However, if there is no advantage in disabling it, we might not want to allow it. This would have the advantage to guard users to accidentally switch it off. -Matthias On 2/3/17 2:03 AM, Damian Guy wrote: > Hi Matthias, > > It possibly doesn't make sens

[jira] [Updated] (KAFKA-4743) Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4743?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4743: --- Labels: kip (was: ) > Add a tool to Reset Consumer Group Offsets > --

[jira] [Assigned] (KAFKA-4743) Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4743?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-4743: -- Assignee: (was: Matthias J. Sax) > Add a tool to Reset Consumer Group Offsets > ---

[jira] [Assigned] (KAFKA-4743) Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4743?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-4743: -- Assignee: Matthias J. Sax > Add a tool to Reset Consumer Group Offsets > --

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Matthias J. Sax
Jorge, thanks for you KIP. I like it a lot and think it will be a nice addition! -Matthias On 2/7/17 7:04 PM, Dong Lin wrote: > Hey Jorge, > > Thanks for the KIP. I have some quick comments: > > - Should we allow user to use wildcard to reset offset of all groups for a > given topic as well?

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Jay Kreps
Totally agree on CompletableFuture. Also agree with some of the rough edges on the Consumer. I don't have much of a leg to stand on with the splitting vs not splitting thing, really hard to argue one or the other is better. I guess the one observation in watching us try to make good public apis ov

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread William Gray
+1 for placing the AdminClient in a separate package. I have several dozen applications that use kafka-clients.jar to produce/consume, and those applications don't have a need for the AdminClient. And, it's probably better not to have the AdminClient methods available in those applications from a

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-07 Thread Roger Hoover
Hi Jun, How does it allow impersonation at the connection level? Looking at the KIP, the DelegationTokenRequest does not have an "Owner" field that can be set. The owner field of the DelegationTokenResponse says it's the "Kakfa Principal which requested the delegation token". For impersonation

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Dong Lin
Hey Jorge, Thanks for the KIP. I have some quick comments: - Should we allow user to use wildcard to reset offset of all groups for a given topic as well? - Should we allow user to specify timestamp per topic partition in the json file as well? - Should the script take some credential file to mak

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-07 Thread Jun Rao
Hi, Roger, Just to clarify. This KIP already allows you to do impersonation at the connection level. Are you talking about impersonation at the request level? Thanks, Jun On Tue, Feb 7, 2017 at 5:53 PM, Roger Hoover wrote: > Just wondering...how difficult would be it be to later add impersona

KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Jorge Esteban Quilcate Otoya
Hi all, I would like to propose a KIP to Add a tool to Reset Consumer Group Offsets. https://cwiki.apache.org/confluence/display/KAFKA/KIP-122%3A+Add+a+tool+to+Reset+Consumer+Group+Offsets Please, take a look at the proposal and share your feedback. Thanks, Jorge.

[jira] [Updated] (KAFKA-4743) Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Jorge Quilcate (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4743?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jorge Quilcate updated KAFKA-4743: -- Summary: Add a tool to Reset Consumer Group Offsets (was: Add a tool to Reset Consumers Offsets

[jira] [Created] (KAFKA-4743) Add a tool to Reset Consumers Offsets

2017-02-07 Thread Jorge Quilcate (JIRA)
Jorge Quilcate created KAFKA-4743: - Summary: Add a tool to Reset Consumers Offsets Key: KAFKA-4743 URL: https://issues.apache.org/jira/browse/KAFKA-4743 Project: Kafka Issue Type: New Feature

[GitHub] kafka pull request #2514: MINOR: Add logging when commitSync fails in Stream...

2017-02-07 Thread jmoney8080
GitHub user jmoney8080 opened a pull request: https://github.com/apache/kafka/pull/2514 MINOR: Add logging when commitSync fails in StreamTask When `consumer.commitSync` fails in `StreamTask`, the `CommitFailedException` bubbles up to [here](https://github.com/apache/kafka/blob/tru

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-07 Thread Roger Hoover
Just wondering...how difficult would be it be to later add impersonation ( https://issues.apache.org/jira/browse/KAFKA-3712)? One use case would be a Kafka admin UI that would take action on the cluster on behalf different users.I suppose we could later add an "effectiveUserId" (in Unix termin

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

2017-02-07 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-4725; Stop leaking messages in produce request body when requests -- [...truncated 8409 lines...] kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache STARTED

[jira] [Commented] (KAFKA-4564) When the destination brokers are down or misconfigured in config, Streams should fail fast

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15857198#comment-15857198 ] Matthias J. Sax commented on KAFKA-4564: [~guozhang] Can we close this ? > When t

[jira] [Commented] (KAFKA-4564) When the destination brokers are down or misconfigured in config, Streams should fail fast

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15857193#comment-15857193 ] Matthias J. Sax commented on KAFKA-4564: [~gwenshap] You can configure it via `req

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Gwen Shapira
Ismael raised the concern (offline) that even if we mark the API as unstable, people will use it right away because it is so valuable. So we'll have trouble changing later if we prefer a different API. I agree, but since I still prefer to experiment a bit "in the wild", I suggested placing the Adm

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-07 Thread Grant Henke
+1 from me as well. On Tue, Feb 7, 2017 at 7:10 PM, Jason Gustafson wrote: > Looks like a great proposal! I noticed that key rotation is not included. > That may be reasonable for the initial work, but it might be nice to share > some thoughts on how that might work in the future. For example, I

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-07 Thread Dong Lin
Hey Eno, Thanks much for the comment! I still think the complexity added to Kafka is justified by its benefit. Let me provide my reasons below. 1) The additional logic is easy to understand and thus its complexity should be reasonable. On the broker side, it needs to catch exception when access

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-07 Thread Jason Gustafson
Looks like a great proposal! I noticed that key rotation is not included. That may be reasonable for the initial work, but it might be nice to share some thoughts on how that might work in the future. For example, I could imagine delegation.token.master.key could be a list, which would allow users

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

2017-02-07 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-4725; Stop leaking messages in produce request body when requests -- [...truncated 15912 lines...] org.apache.kafka.clients.producer.internals.RecordAccumulatorTest >

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-07 Thread Dong Lin
Hey Jun, Thanks for the all the comments. I should have written the summary earlier but got delayed. I think Grant has summarized pretty much every major issues we discussed in the KIP meeting. I have provided answer to each issue. Let me try to address your questions here. I will update the KIP

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-07 Thread Gwen Shapira
Read the KIP again and I think it looks good. +1 from me. On Tue, Feb 7, 2017 at 3:05 PM, Jun Rao wrote: > Hi, Mani, > > If a token expires, then every broker will potentially try to delete it > around the same time, but only one will succeed. So, we will have to deal > with failures in that cas

Reg: Kafka Kerberos

2017-02-07 Thread BigData dev
Hi, I am using Kafka 0.10.1.0 and kerberozied cluster. Kafka_jaas.conf file: Client { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true keyTab="/etc/security/keytabs/kafka.service.keytab" storeKey=true useTicketCache=false serviceName="zookeeper" principal=

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Gwen Shapira
>>- Personally I don't think splitting the admin methods up actually makes >>things more usable. It just makes you have to dig through our >> hierarchy. I >>think a flat class with a bunch of operations (like the consumer api) is >>probably the easiest for people to grok and find th

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-07 Thread Dong Lin
Hey Grant, Thanks much for the detailed summary! Yes, this is pretty much my understanding of the KIP meeting. I also think everyone agreed on the point you outlined in the email. Here is my reply to the five issues you mentioned. 1) Automatic vs Manual Recovery In the case where a disk is repl

[subscribe Request]

2017-02-07 Thread Ashish Singh
Hi Team, I would like to subscribe to Kafka dev group Thanks, Ashish Singh

Re: [VOTE] 0.10.2.0 RC0

2017-02-07 Thread Jun Rao
Hi, Ewen, KAFKA-4725 has been committed to 0.10.2. Thanks, Jun On Mon, Feb 6, 2017 at 11:18 PM, Ewen Cheslack-Postava wrote: > All, > > It sounds like https://issues.apache.org/jira/browse/KAFKA-4725 is a > regression in an earlier release and worth fixing. I've marked it blocker > for 0.10.2

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Ismael Juma
Hi Jay, Thanks for the feedback. Comments inline. On Tue, Feb 7, 2017 at 8:18 PM, Jay Kreps wrote: > >- I think it would be good to not use "get" as the prefix for things >making remote calls. We've tried to avoid the java getter convention >entirely (see code style guide), but for r

[jira] [Updated] (KAFKA-4725) Kafka broker fails due to OOM when producer exceeds throttling quota for extended periods of time

2017-02-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4725?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-4725: --- Resolution: Fixed Fix Version/s: (was: 0.10.3.0) Status: Resolved (was: Patch Available)

[jira] [Commented] (KAFKA-4725) Kafka broker fails due to OOM when producer exceeds throttling quota for extended periods of time

2017-02-07 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4725?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856990#comment-15856990 ] ASF GitHub Bot commented on KAFKA-4725: --- Github user asfgit closed the pull request

[GitHub] kafka pull request #2496: KAFKA-4725: Stop leaking messages in produce reque...

2017-02-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2496 --- 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 enab

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

2017-02-07 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-4727; A Production server configuration needs to be updated [junrao] KAFKA-4734; Trim the time index on old segments -- [...truncated 37436 lines...] org.apache.kaf

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-07 Thread Jason Gustafson
A couple follow-ups from Ismael's previous comments. 1. We have removed the individual message CRC field from the message format. Because this field can already change on the broker in some situations, we feel it is probably not wise to let clients depend on it, and removing it saves some space an

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-07 Thread Jun Rao
Hi, Mani, If a token expires, then every broker will potentially try to delete it around the same time, but only one will succeed. So, we will have to deal with failures in that case? Another way is to let just one broker (say, the controller) deletes expired tokens. It would also be helpful for

[jira] [Commented] (KAFKA-4725) Kafka broker fails due to OOM when producer exceeds throttling quota for extended periods of time

2017-02-07 Thread Tim Carey-Smith (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4725?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856965#comment-15856965 ] Tim Carey-Smith commented on KAFKA-4725: We have run stress tests on builds which

[jira] [Commented] (KAFKA-4564) When the destination brokers are down or misconfigured in config, Streams should fail fast

2017-02-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856964#comment-15856964 ] Gwen Shapira commented on KAFKA-4564: - OMG, this is great. I got this: [2017-02-

[jira] [Comment Edited] (KAFKA-4564) When the destination brokers are down or misconfigured in config, Streams should fail fast

2017-02-07 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856964#comment-15856964 ] Gwen Shapira edited comment on KAFKA-4564 at 2/7/17 10:51 PM: --

[jira] [Updated] (KAFKA-4697) Simplify Streams Reset Tool

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4697: --- Affects Version/s: 0.10.2.0 > Simplify Streams Reset Tool > --- > >

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-07 Thread Jun Rao
Hi, Dong, Thanks for the discussion in the KIP meeting today. A few comments inlined below. On Mon, Feb 6, 2017 at 7:22 PM, Dong Lin wrote: > Hey Jun, > > Thanks for the review! Please see reply inline. > > On Mon, Feb 6, 2017 at 6:21 PM, Jun Rao wrote: > > > Hi, Dong, > > > > Thanks for the p

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-07 Thread Grant Henke
Hi Dong, Thanks for proposing the KIP and all the hard work on it! In order to help summarize the discussion from the KIP call today I wanted to list the things I heard as the main discussion points that people would like to be considered or discussed. However, this is strictly from memory so ple

[jira] [Created] (KAFKA-4742) ResetTool does not commit offsets correclty

2017-02-07 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4742: -- Summary: ResetTool does not commit offsets correclty Key: KAFKA-4742 URL: https://issues.apache.org/jira/browse/KAFKA-4742 Project: Kafka Issue Type: Bug

Re: Kafka KIP meeting Feb 7 at 11:00am PST

2017-02-07 Thread Jun Rao
The following are the notes from today's KIP discussion. - KIP-112 - Handle disk failure for JBOD: We discussed whether we need to support JBOD directly in Kafka or just rely on the 1 disk per broker model. The general consensus is that direct JBOD support in Kafka is needed. There is

[jira] [Updated] (KAFKA-4721) KafkaStreams (and possibly others) should inherit Closeable

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4721?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4721: --- Labels: needs-kip (was: ) > KafkaStreams (and possibly others) should inherit Closeable > ---

[jira] [Commented] (KAFKA-1194) The kafka broker cannot delete the old log files after the configured time

2017-02-07 Thread Brady Vidovic (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856748#comment-15856748 ] Brady Vidovic commented on KAFKA-1194: -- I'm trying to determine if/when this will be

Re: [VOTE] KIP-111 Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-07 Thread Mayuresh Gharat
Bumping up this thread. Thanks, Mayuresh On Fri, Feb 3, 2017 at 5:09 PM, radai wrote: > +1 > > On Fri, Feb 3, 2017 at 11:25 AM, Mayuresh Gharat < > gharatmayures...@gmail.com > > wrote: > > > Hi All, > > > > It seems that there is no further concern with the KIP-111. At this point > > we would

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Jay Kreps
Hey Colin, This is great, thought I'd throw out a couple of opinions to the mix, feel free to ignore: - I think it would be good to not use "get" as the prefix for things making remote calls. We've tried to avoid the java getter convention entirely (see code style guide), but for remote

[jira] [Commented] (KAFKA-4738) Remove generic type of class ClientState

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856683#comment-15856683 ] Matthias J. Sax commented on KAFKA-4738: Maybe. But we should have a new JIRA for

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856602#comment-15856602 ] Rodrigo Queiroz Saramago commented on KAFKA-4686: - {noformat:title=broker0

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856592#comment-15856592 ] Jason Gustafson commented on KAFKA-4686: [~rodrigo.saramago] Thanks for the update

Re: KIP-121 [Discuss]: Add KStream peek method

2017-02-07 Thread Gwen Shapira
Far better! Thank you! On Tue, Feb 7, 2017 at 10:19 AM, Steven Schlansker wrote: > Thanks for the feedback. I improved the javadoc a bit, do you like it better? > > /** > * Perform an action on each record of {@code KStream}. > * This is a stateless record-by-record operation (cf.

[jira] [Comment Edited] (KAFKA-4738) Remove generic type of class ClientState

2017-02-07 Thread Sharad (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856547#comment-15856547 ] Sharad edited comment on KAFKA-4738 at 2/7/17 6:58 PM: --- [~mjsax] Tas

[jira] [Commented] (KAFKA-4738) Remove generic type of class ClientState

2017-02-07 Thread Sharad (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856547#comment-15856547 ] Sharad commented on KAFKA-4738: --- [~mjsax] TaskAssignor> takes two generic arguments. Generi

[jira] [Comment Edited] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856534#comment-15856534 ] Rodrigo Queiroz Saramago edited comment on KAFKA-4686 at 2/7/17 6:54 PM: ---

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856534#comment-15856534 ] Rodrigo Queiroz Saramago commented on KAFKA-4686: - Sure, the results are b

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856522#comment-15856522 ] Jason Gustafson commented on KAFKA-4686: Can you try using DumpLogSegments on the

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856518#comment-15856518 ] Rodrigo Queiroz Saramago commented on KAFKA-4686: - Running the Dumplog wit

[jira] [Comment Edited] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856500#comment-15856500 ] Rodrigo Queiroz Saramago edited comment on KAFKA-4686 at 2/7/17 6:42 PM: ---

[GitHub] kafka pull request #2497: [MINOR] fixed some JavaDoc typos

2017-02-07 Thread mjsax
Github user mjsax closed the pull request at: https://github.com/apache/kafka/pull/2497 --- 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 enabl

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856500#comment-15856500 ] Rodrigo Queiroz Saramago commented on KAFKA-4686: - No, there are no more f

Re: KIP-121 [Discuss]: Add KStream peek method

2017-02-07 Thread Steven Schlansker
Thanks for the feedback. I improved the javadoc a bit, do you like it better? /** * Perform an action on each record of {@code KStream}. * This is a stateless record-by-record operation (cf. {@link #process(ProcessorSupplier, String...)}). * * Peek is a non-terminal opera

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Colin McCabe
On Tue, Feb 7, 2017, at 08:37, Ismael Juma wrote: > Hi all, > > I think it's good that we have discussed a number of API that would make > sense in the AdminClient. Having done that, I think we should now narrow > the focus of this KIP to a small set of methods to get us started. Once > we > have

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Colin McCabe
Hi Hu Xi, On Mon, Feb 6, 2017, at 19:37, Hu Xi wrote: > Two things I want to confirm. Please advise. > > > 1. Seems the KIP only cares about topic management things. Is there any > plan for this KIP to merge the feature of what `GetOffsetShell` script > offers? Since a lot of people really want

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Colin McCabe
On Mon, Feb 6, 2017, at 14:05, Jianbin Wei wrote: > In the specify group information, can we also return information like > partition assignment for each member, the lag/offset of each > member/partition? It would be useful for Ops/Admin regarding the health > of the consumer group. This informat

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Colin McCabe
On Mon, Feb 6, 2017, at 13:54, Guozhang Wang wrote: > Some follow-up on 2) / 3) below. > > On Mon, Feb 6, 2017 at 11:21 AM, Colin McCabe wrote: > > > On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote: > > > Thanks for the proposal Colin. A few comments below: > > > > Thanks for taking a look, G

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Sriram Subramanian
+1 > On Feb 7, 2017, at 9:17 AM, radai wrote: > > +1. > > under ismael's "facet" approach we could always start with > AdminClient.topics() and then pile on more of them later. > >> On Tue, Feb 7, 2017 at 8:57 AM, Grant Henke wrote: >> >> +1 I think its important to focus this KIP discussio

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856412#comment-15856412 ] Jason Gustafson commented on KAFKA-4686: One additional question: which clients an

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856379#comment-15856379 ] Jason Gustafson commented on KAFKA-4686: [~rodrigo.saramago] Are there no files wi

[jira] [Comment Edited] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856272#comment-15856272 ] Rodrigo Queiroz Saramago edited comment on KAFKA-4686 at 2/7/17 5:31 PM: ---

[GitHub] kafka pull request #2513: WIP: Remove Struct from Request/Response classes

2017-02-07 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/2513 WIP: Remove Struct from Request/Response classes You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka separate-struct Alternatively you ca

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread radai
+1. under ismael's "facet" approach we could always start with AdminClient.topics() and then pile on more of them later. On Tue, Feb 7, 2017 at 8:57 AM, Grant Henke wrote: > +1 I think its important to focus this KIP discussion on the "patterns" we > would like to see in the client and a few ke

[jira] [Commented] (KAFKA-2122) Remove controller.message.queue.size Config

2017-02-07 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856324#comment-15856324 ] ASF GitHub Bot commented on KAFKA-2122: --- Github user soenkeliebau closed the pull re

[GitHub] kafka pull request #2512: Removed obsolete parameter form example config in ...

2017-02-07 Thread soenkeliebau
Github user soenkeliebau closed the pull request at: https://github.com/apache/kafka/pull/2512 --- 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 i

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Grant Henke
+1 I think its important to focus this KIP discussion on the "patterns" we would like to see in the client and a few key methods in order to make progress and then iterate from there. I think we should let Colin drive the APIs he thinks are important since he is volunteering to do the work. And th

[jira] [Commented] (KAFKA-4734) timeindex on old segments not trimmed to actual size

2017-02-07 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4734?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856309#comment-15856309 ] ASF GitHub Bot commented on KAFKA-4734: --- Github user asfgit closed the pull request

[jira] [Updated] (KAFKA-4734) timeindex on old segments not trimmed to actual size

2017-02-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4734?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-4734: --- Resolution: Fixed Fix Version/s: 0.10.2.0 Status: Resolved (was: Patch Available) Issue res

[GitHub] kafka pull request #2501: KAFKA-4734; Trim the time index on old segments

2017-02-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2501 --- 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 enab

[jira] [Resolved] (KAFKA-4727) A Production server configuration needs to be updated

2017-02-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4727?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-4727. Resolution: Fixed Fix Version/s: 0.10.2.0 Issue resolved by pull request 2490 [https://github.com/apa

[GitHub] kafka pull request #2490: kafka-4727: A Production server configuration need...

2017-02-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2490 --- 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 enab

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-07 Thread Ismael Juma
Hi all, I think it's good that we have discussed a number of API that would make sense in the AdminClient. Having done that, I think we should now narrow the focus of this KIP to a small set of methods to get us started. Once we have an AdminClient in the codebase, we can propose subsequent KIPs t

[jira] [Comment Edited] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856272#comment-15856272 ] Rodrigo Queiroz Saramago edited comment on KAFKA-4686 at 2/7/17 4:29 PM: ---

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856272#comment-15856272 ] Rodrigo Queiroz Saramago commented on KAFKA-4686: - Hi [~hachikuji], no, we

[jira] [Commented] (KAFKA-2122) Remove controller.message.queue.size Config

2017-02-07 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856134#comment-15856134 ] ASF GitHub Bot commented on KAFKA-2122: --- GitHub user soenkeliebau opened a pull requ

[GitHub] kafka pull request #2512: Removed obsolete parameter form example config in ...

2017-02-07 Thread soenkeliebau
GitHub user soenkeliebau opened a pull request: https://github.com/apache/kafka/pull/2512 Removed obsolete parameter form example config in docs. Parameter controller.message.queue.size was removed in 0.9 (KAFKA-2122) but is still listed in an example broker configuration in the doc

[jira] [Commented] (KAFKA-2122) Remove controller.message.queue.size Config

2017-02-07 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15856115#comment-15856115 ] ASF GitHub Bot commented on KAFKA-2122: --- Github user soenkeliebau closed the pull re

[GitHub] kafka pull request #2183: Removed obsolete parameter form example config in ...

2017-02-07 Thread soenkeliebau
Github user soenkeliebau closed the pull request at: https://github.com/apache/kafka/pull/2183 --- 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 i

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-07 Thread Mathieu Fenniak
On Mon, Feb 6, 2017 at 2:35 PM, Matthias J. Sax wrote: > - adding KStreamBuilder#topologyBuilder() seems like be a good idea to > address any concern with limited access to TopologyBuilder and DSL/PAPI > mix-and-match approach. However, we should try to cover as much as > possible with #process()

[jira] [Commented] (KAFKA-4567) Connect Producer and Consumer ignore ssl parameters configured for worker

2017-02-07 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4567?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855932#comment-15855932 ] Sönke Liebau commented on KAFKA-4567: - Alright. I have added a small paragraph to the

[jira] [Commented] (KAFKA-4567) Connect Producer and Consumer ignore ssl parameters configured for worker

2017-02-07 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4567?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855931#comment-15855931 ] ASF GitHub Bot commented on KAFKA-4567: --- GitHub user soenkeliebau opened a pull requ

[GitHub] kafka pull request #2511: KAFKA-4567 - Connect Producer and Consumer ignore ...

2017-02-07 Thread soenkeliebau
GitHub user soenkeliebau opened a pull request: https://github.com/apache/kafka/pull/2511 KAFKA-4567 - Connect Producer and Consumer ignore ssl parameters… KAFKA-4567 - Connect Producer and Consumer ignore ssl parameters configured for worker Added brief explanation to th

Correct prefetching of data to KTable-like structure on application startup

2017-02-07 Thread Jan Lukavský
Hi all, I have a question how to do a correct caching in KTable-like structure on application startup. I'm not sure if this belongs to user or dev maillist, so sorry if I've chosen the bad one. What is my observation so far: - if I don't send any data to a kafka partition for a period longe

[jira] [Commented] (KAFKA-4461) When using ProcessorTopologyTestDriver, the combination of map and .groupByKey does not produce any result

2017-02-07 Thread Adrian McCague (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855792#comment-15855792 ] Adrian McCague commented on KAFKA-4461: --- [~guozhang] Thanks! > When using Processor

  1   2   >