Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-27 Thread Jukka Karvanen
Hi, >>(4) Should we switch from `long` for timestamps to `Instant` and `Duration` ? >This version startTimestamp is Instant and autoAdvance Duration in Initialization and with manual configured collection pipe methods. >Now timestamp of TestRecord is still Long and similarly single record pipeInpu

[jira] [Created] (KAFKA-8606) Provide a method to fetch committed offsets for a collection of TopicPartition

2019-06-27 Thread ov7a (JIRA)
ov7a created KAFKA-8606: --- Summary: Provide a method to fetch committed offsets for a collection of TopicPartition Key: KAFKA-8606 URL: https://issues.apache.org/jira/browse/KAFKA-8606 Project: Kafka I

Re: [DISCUSS] KIP-480 : Sticky Partitioner

2019-06-27 Thread Ismael Juma
Thanks for the KIP Justine. It looks pretty good. A few comments: 1. Should we favor partitions that are not under replicated? This is something that Netflix did too. 2. If there's no measurable performance difference, I agree with Stanislav that Optional would be better than Integer. 3. We shou

[jira] [Created] (KAFKA-8607) Reduce AdminClient Metadata request rate when invalid node id is given

2019-06-27 Thread Stanislav Kozlovski (JIRA)
Stanislav Kozlovski created KAFKA-8607: -- Summary: Reduce AdminClient Metadata request rate when invalid node id is given Key: KAFKA-8607 URL: https://issues.apache.org/jira/browse/KAFKA-8607 Proj

Re: [DISCUSS] KIP-373: Allow users to create delegation tokens for other users

2019-06-27 Thread Viktor Somogyi-Vass
Hi Folks, I took over this issue from Manikumar. Recently another motivation have been raised in Spark for this (SPARK-28173) and I think it'd be great to continue this task. I updated the KIP and will wait for a few days to get some feedback then proceed for the vote. Thanks, Viktor On Tue, Dec

Re: [VOTE] KIP-412: Extend Admin API to support dynamic application log levels

2019-06-27 Thread Stanislav Kozlovski
Hey there everybody, We're moving along. I wanted to come back with some updates we came up with while iterating on the PR: - Removed the ALL log level - all the rest log levels are quite standard but this is not that expected. Simpler is always better :) - We have changed the DescribeConfigs API

[jira] [Created] (KAFKA-8608) Broker shows WARN on reassignment partitions on new brokers: Replica LEO & Cache truncation

2019-06-27 Thread Di Campo (JIRA)
Di Campo created KAFKA-8608: --- Summary: Broker shows WARN on reassignment partitions on new brokers: Replica LEO & Cache truncation Key: KAFKA-8608 URL: https://issues.apache.org/jira/browse/KAFKA-8608 Proje

Re: [DISCUSS] KIP-435: Incremental Partition Reassignment

2019-06-27 Thread Viktor Somogyi-Vass
Hi Colin, Certainly there will be some interaction and good idea with that you said, I've added it to my KIP. Will start a new discussion thread and link this one. Viktor On Wed, Jun 26, 2019 at 11:39 PM Colin McCabe wrote: > Hi Viktor, > > Good point. Sorry, I should have read the KIP more c

[DISCUSS] KIP-435: Internal Partition Reassignment Batching

2019-06-27 Thread Viktor Somogyi-Vass
Hi All, I've renamed my KIP as its name was a bit confusing so we'll continue it in this thread. The previous thread for record: https://lists.apache.org/thread.html/0e97e30271f80540d4da1947bba94832639767e511a87bb2ba530fe7@%3Cdev.kafka.apache.org%3E A short summary of the KIP: In case of a vast p

Re: [DISCUSS] KIP-385: Provide configuration allowing consumer to no throw away prefetched data

2019-06-27 Thread Sean Glover
Hi everyone, I want to revive a solution to this issue. I created a new PR that accomodates Jason Gustafson's suggestion in the original PR to re-add paused completed fetches back to the completed fetches queue for less bookeeping. If someone could jump in and do a review it would be appreciated

Re: [DISCUSS] KIP-455: Create an Administrative API for Replica Reassignment

2019-06-27 Thread Viktor Somogyi-Vass
Hey Colin, So in my understanding this is how the controller handles a reassignment in a simple scenario: 1. When an AlterPartitionReassignmentRequest arrives it updates the partitions' ZK data in /brokers/topics/[topic]/partitions/[partitionId]/state with targetReplicas 2. Sends out LeaderAndIsr

Re: [DISCUSS] KIP-455: Create an Administrative API for Replica Reassignment

2019-06-27 Thread Jason Gustafson
> > We'd remove nodes from targetReplicas just as soon as they entered the > ISR. They would become regular replicas at that point. I think we can save a lot of back and forth by working through an example. Suppose we have the following initial state: replicas: [1, 2, 3] isr: [1, 2, 3] targetRe

Re: [DISCUSS] KIP-444: Augment Metrics for Kafka Streams

2019-06-27 Thread Guozhang Wang
Hello folks, As 2.3 is released now, I'd like to bump up this KIP discussion again for your reviews. Guozhang On Thu, May 23, 2019 at 4:44 PM Guozhang Wang wrote: > Hello Patrik, > > Since we are rolling out 2.3 and everyone is busy with the release now > this KIP does not have much discussi

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-27 Thread John Roesler
I think I agree with you, right joins (and therefore full outer joins) don't make sense here, because the result is a keyed table, where the key is the PK of the left-hand side. So, when you have a right-hand-side record with no incoming FK references, you would want to produce a join result like `

Re: [DISCUSS] KIP-479: Add Materialized to Join

2019-06-27 Thread Guozhang Wang
Hi John, I actually feels better about a new interface but I'm not sure if we would need the full configuration of store / log / cache, now or in the future ever for stream-stream join. Right now I feel that 1) we want to improve our implementation of stream-stream join, and potentially also allo

Re: [DISCUSS] KIP-480 : Sticky Partitioner

2019-06-27 Thread Justine Olshan
Ismael, Thanks for the feedback! For 1, currently the sticky partitioner favors "available partitions." From my understanding, these are partitions that are not under-replicated. If that is not the same, please let me know. As for 2, I've switched to Optional, and the few tests I've run so far su

Re: [DISCUSS] KIP-480 : Sticky Partitioner

2019-06-27 Thread Ismael Juma
Hey Justine. Available could mean that some replicas are down but the leader is available. The suggestion was to try a partition where no replica was down if it's available. Such partitions are safer in general. There could be some downsides too, so worth thinking about the trade-offs. Ismael On

Re: [DISCUSS] KIP-480 : Sticky Partitioner

2019-06-27 Thread Justine Olshan
I was going through fixing some of the overloaded methods and I realized I overloaded the RecordAccumulator constructor. I added a parameter to include the partitioner so I can call my method. However, the tests for the record accumulator do not have a partitioner. There is the potential for a NPE

Re: [DISCUSS] KIP-480 : Sticky Partitioner

2019-06-27 Thread Justine Olshan
Moving the previous comment to the PR discussion. :) On Thu, Jun 27, 2019 at 10:51 AM Justine Olshan wrote: > I was going through fixing some of the overloaded methods and I realized I > overloaded the RecordAccumulator constructor. I added a parameter to > include the partitioner so I can call

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-27 Thread Adam Bellemare
You're stretching my brain, John! I prefer STRATEGY 1 because it solves the problem in a simple way, and allows us to deprecate support for older message types as we go (ie, we only support the previous 3 versions, so V5,V4,V3, but not v2 or V1). STRATEGY 2 is akin to Avro schemas between two mic

Re: [DISCUSS] KIP-479: Add Materialized to Join

2019-06-27 Thread Matthias J. Sax
Thanks for the KIP Bill! Great discussion to far. About John's idea about querying upstream stores and don't materialize a store: I agree with Bill that this seems to be an orthogonal question, and it might be better to treat it as an independent optimization and exclude from this KIP. > What sh

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-27 Thread John Roesler
Hi Adam, Hah! Yeah, I felt a headache coming on myself when I realized this would be a concern. For what it's worth, I'd also lean toward versioning. It seems more explicit and more likely to keep us all sane in the long run. Since we don't _think_ our wire protocol will be subject to a lot of re

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-27 Thread Matthias J. Sax
Thanks for bringing this issue to our attention. Great find @Joe! Adding the instruction field to the `subscription` sounds like a good solution. What I don't understand atm: for which case would we need to send unnecessary tombstone? I thought that the `instruction` field helps to avoid any unnec

Re: [DISCUSS] KIP-455: Create an Administrative API for Replica Reassignment

2019-06-27 Thread Colin McCabe
On Thu, Jun 27, 2019, at 08:58, Jason Gustafson wrote: > > > > We'd remove nodes from targetReplicas just as soon as they entered the > > ISR. They would become regular replicas at that point. > > > I think we can save a lot of back and forth by working through an example. > Suppose we have the

Re: [DISCUSS] KIP-480 : Sticky Partitioner

2019-06-27 Thread Colin McCabe
On Thu, Jun 27, 2019, at 01:31, Ismael Juma wrote: > Thanks for the KIP Justine. It looks pretty good. A few comments: > > 1. Should we favor partitions that are not under replicated? This is > something that Netflix did too. This seems like it could lead to cascading failures, right? If a parti

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-27 Thread Matthias J. Sax
Thanks Jukka! The idea to use `Instant/Duration` was a proposal. If we think it's not a good one, we could still stay with `long`. Because `ProducerRecord` and `ConsumerRecord` are both based on `long,` it might make sense to keep `long`? > The result of converting millis to Instant directly gene

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

2019-06-27 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix typos in upgrade guide (#7005) -- [...truncated 4.81 MB...] java.lang.NullPointerException at org.apache.kafka.streams.kstr

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-27 Thread Matthias J. Sax
@Andy: What about the factory methods of `AdminClient` class? Should they be deprecated? One nit about the KIP: can you maybe insert "code blocks" to highlight the API changes? Code blocks would simplify to read the KIP a lot. -Matthias On 6/26/19 6:56 AM, Ryanne Dolan wrote: > +1 (non-binding

[jira] [Created] (KAFKA-8609) Add consumer metrics for rebalances

2019-06-27 Thread Sophie Blee-Goldman (JIRA)
Sophie Blee-Goldman created KAFKA-8609: -- Summary: Add consumer metrics for rebalances Key: KAFKA-8609 URL: https://issues.apache.org/jira/browse/KAFKA-8609 Project: Kafka Issue Type: Sub

Build failed in Jenkins: kafka-trunk-jdk11 #662

2019-06-27 Thread Apache Jenkins Server
See Changes: [rajinisivaram] MINOR: Fix failing upgrade test by supporting both [rajinisivaram] MINOR: Support listener config overrides in system tests (#6981) -- [...

Re: [VOTE] KIP-429: Kafka Consumer Incremental Rebalance Protocol

2019-06-27 Thread Sophie Blee-Goldman
We would like to tack on some rebalance-related metrics as part of this KIP as well. The details can be found in the sub-task JIRA: https://issues.apache.org/jira/browse/KAFKA-8609 On Thu, May 30, 2019 at 5:09 PM Guozhang Wang wrote: > +1 (binding) from me as well. > > Thanks to everyone who hav

Request for Permission to Create KIP

2019-06-27 Thread Maulin Vasavada
Hi Can you please give me permission to Create KIP? My username: maulin.vasavada Thank you. Maulin

Re: KIP-457: Add DISCONNECTED state to Kafka Streams

2019-06-27 Thread Richard Yu
Hi Matthias and Hachikuji, Sorry for getting back to you so late. Currently on a trip, so I hadn't got the time to respond. Currently, I'm not sure which approach we should do ATM, considering that Guozhang posed multiple possibilities in the previous email.Do you have any preferences as to whi

Re: [VOTE] KIP-429: Kafka Consumer Incremental Rebalance Protocol

2019-06-27 Thread Boyang Chen
Thank you Sophie for the update. Is this also reflected on the KIP? On Thu, Jun 27, 2019 at 3:28 PM Sophie Blee-Goldman wrote: > We would like to tack on some rebalance-related metrics as part of this KIP > as well. The details can be found in the sub-task JIRA: > https://issues.apache.org/jira/

[jira] [Created] (KAFKA-8610) Don't use /bin/bash in scripts

2019-06-27 Thread Richard Lee (JIRA)
Richard Lee created KAFKA-8610: -- Summary: Don't use /bin/bash in scripts Key: KAFKA-8610 URL: https://issues.apache.org/jira/browse/KAFKA-8610 Project: Kafka Issue Type: Improvement

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

2019-06-27 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-8538) Add `group.instance.id` to DescribeGroup for better visibility

2019-06-27 Thread Boyang Chen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8538?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-8538. Resolution: Fixed > Add `group.instance.id` to DescribeGroup for better visibility > --

[jira] [Resolved] (KAFKA-8356) Add static membership to Round Robin assignor

2019-06-27 Thread Boyang Chen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8356?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-8356. Resolution: Fixed > Add static membership to Round Robin assignor > ---

[jira] [Resolved] (KAFKA-8569) Integrate the poll timeout warning with leave group call

2019-06-27 Thread Boyang Chen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8569?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-8569. Resolution: Fixed > Integrate the poll timeout warning with leave group call >