Re: [VOTE] KIP-331 Add default implementation to close() and configure() for Serializer, Deserializer and Serde

2018-07-06 Thread Chia-Ping Tsai
hi Ismael > I think we should be thinking about > https://issues.apache.org/jira/browse/KAFKA-6923 at the same time. you are right. KAFKA-6923 and KAFKA-6161 are the keys to complete the Serializer and Deserializer. Let us add the default implementations first (KAFKA-6161). And then integrate

Re: [kafka-clients] [VOTE] 1.1.1 RC2

2018-07-06 Thread Dong Lin
Hi all, We found a deadlock that affects metrics reporters that use synchronization (https://issues.apache.org/jira/browse/KAFKA-7136). Since it may introduce new issue in the bug fix release, I will create another RC that includes the fix. Thank you all for testing and voting for this release!

[jira] [Resolved] (KAFKA-7136) PushHttpMetricsReporter may deadlock when processing metrics changes

2018-07-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7136?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-7136. -- Resolution: Fixed Fix Version/s: 1.1.1 > PushHttpMetricsReporter may deadlock when

Jenkins build is back to normal : kafka-1.1-jdk7 #162

2018-07-06 Thread Apache Jenkins Server
See

Jenkins build is back to normal : kafka-1.0-jdk7 #217

2018-07-06 Thread Apache Jenkins Server
See

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

2018-07-06 Thread Apache Jenkins Server
See Changes: [github] MINOR: Use FetchRequest v8 and ListOffsetRequest v3 in -- [...truncated 2.48 MB...] org.apache.kafka.streams.StreamsConfigTest >

[jira] [Resolved] (KAFKA-7130) EOFException after rolling log segment

2018-07-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-7130. Resolution: Duplicate > EOFException after rolling log segment >

Re: [DISCUSS] KIP-328: Ability to suppress updates for KTables

2018-07-06 Thread Guozhang Wang
I think I agree with Matthias for having dedicated APIs for windowed operation final output scenario, PLUS separating the window close which the "final output" would rely on, from the window retention time itself (admittedly it would make this KIP effort larger, but if we believe we need to do

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-07-06 Thread Jun Rao
Hi, Jason, Thanks for the KIP. Looks good overall. Just a few minor comments below. 1. "As the consumer is fetching from a partition, it will keep a small cache of the recent epochs that were fetched for each partition. " Do we need to cache more than one leader epoch? Also, during consumer

Re: [DISCUSS] KIP-307: Allow to define custom processor names with KStreams DSL

2018-07-06 Thread Florian Hussonnois
Hi, The option #3 seems to be a good alternative and I find the API more elegant (thanks John). But, we still have the need to overload some methods either because they do not accept an action instance or because they are translated to multiple processors. For example, this is the case for

Re: [DISCUSS] KIP-326: Schedulable KTable as Graph source

2018-07-06 Thread Guozhang Wang
That's a lot of email exchanges for me to catch up :) My original proposed alternative solution is indeed relying on pre-aggregate before sending to the single-partition topic, so that the traffic on that single-partition topic would not be huge (I called it partial-aggregate but the intent was

Re: [DISCUSS] KIP-307: Allow to define custom processor names with KStreams DSL

2018-07-06 Thread Guozhang Wang
Hi folks, just to summarize the options we have so far: 1) Add a new "as" for KTable / KStream, plus adding new fields for operators-returns-void control objects (the current wiki's proposal). Pros: no more overloads. Cons: a bit departing with the current high-level API design of the DSL, plus,

Re: [VOTE] KIP-280: Enhanced log compaction

2018-07-06 Thread Jun Rao
Hi, Guozhang, For #4, what you suggested could make sense for timestamp based de-dup, but not sure how general it is since the KIP also supports de-dup based on header. Thanks, Jun On Fri, Jul 6, 2018 at 1:12 PM, Guozhang Wang wrote: > Hello Jun, > Thanks for your feedbacks. I'd agree on #3

Build failed in Jenkins: kafka-1.1-jdk7 #161

2018-07-06 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-7136: Avoid deadlocks in synchronized metrics reporters (#5341) -- [...truncated 422.58 KB...] kafka.log.LogCleanerIntegrationTest >

Re: [VOTE] KIP-280: Enhanced log compaction

2018-07-06 Thread Ismael Juma
Thanks for the KIP, Luis. A brief comment below. On Wed, Jul 4, 2018 at 11:11 AM Luís Cabral wrote: > As a reader, I tend to prefer brief documentation on new features (they > tend to be too many for me to find the willpower to read a 200-page essay > about each one), so that influences me to

Re: [VOTE] KIP-280: Enhanced log compaction

2018-07-06 Thread Guozhang Wang
Hello Jun, Thanks for your feedbacks. I'd agree on #3 that it's worth adding a special check to not delete the last message, since although unlikely, it is still possible that a new active segment gets rolled out but contains no data yet, and hence the actual last message in this case would be in

Re: [VOTE] 2.0.0 RC1

2018-07-06 Thread Rajini Sivaram
Hi all, We found a deadlock that affects metrics reporters that use synchronisation (https://issues.apache.org/jira/browse/KAFKA-7136). A fix has been merged. I will create another RC that includes the fix for this as well as the PR to update request versions in ReplicaFetcherThread (

Re: [VOTE] KIP-280: Enhanced log compaction

2018-07-06 Thread Jun Rao
Hi, Luis, 1. The cleaning policy is configurable at both global and topic level. The global one has the name log.cleanup.policy and the topic level has the name cleanup.policy by just stripping the log prefix. We can probably do the same for the new configs. 2. Since this KIP may require an

Re: [VOTE] KIP-231: Improve the Required ACL of ListGroups API

2018-07-06 Thread Harsha
+1. Thanks, Harsha On Fri, Jun 1st, 2018 at 10:21 AM, "Vahid S Hashemian" wrote: > > > > I'm bumping this vote thread up as the KIP requires only one binding +1 to > > pass. > The KIP is very similar in nature to the recently approved KIP-277 ( >

Re: [VOTE] KIP-231: Improve the Required ACL of ListGroups API

2018-07-06 Thread Vahid S Hashemian
Bump! From: "Vahid S Hashemian" To: dev Date: 06/01/2018 10:21 AM Subject:Re: [VOTE] KIP-231: Improve the Required ACL of ListGroups API I'm bumping this vote thread up as the KIP requires only one binding +1 to pass. The KIP is very similar in nature to the recently

Re: [VOTE] KIP-331 Add default implementation to close() and configure() for Serializer, Deserializer and Serde

2018-07-06 Thread Ismael Juma
Thanks for the KIP. I think we should be thinking about https://issues.apache.org/jira/browse/KAFKA-6923 at the same time. Ismael On Thu, 5 Jul 2018, 07:45 Chia-Ping Tsai, wrote: > hi all, > > I would like to start voting on "KIP-331 Add default implementation to > close() and configure() for

Re: [VOTE] KIP-331 Add default implementation to close() and configure() for Serializer, Deserializer and Serde

2018-07-06 Thread John Roesler
Thank you! I'm +1 (non-binding) On Thu, Jul 5, 2018 at 10:24 PM Richard Yu wrote: > Nice KIP! > +1 (non-binding) > -Richard > > On Friday, July 6, 2018, 9:10:43 AM GMT+8, Matthias J. Sax < > matth...@confluent.io> wrote: > > Thanks for the KIP! > > +1 (binding) > > > -Matthias > > On

Re: [DISCUSS] KIP-307: Allow to define custom processor names with KStreams DSL

2018-07-06 Thread John Roesler
Hi Florian, Sorry I'm late to the party, but I missed the message originally. Regarding the names, it's probably a good idea to stick to the same character set we're currently using: letters, numbers, and hyphens. The names are used in Kafka topics, files and folders, and RocksDB databases, and

Re: [VOTE] 2.0.0 RC1

2018-07-06 Thread Matthias Wessendorf
+1 used RC1 in little app, that I have - worked fine On Wed, Jul 4, 2018 at 1:58 PM Mickael Maison wrote: > +1 (non-binding) > Ran tests and quickstart using kafka_2.12-2.0.0.tgz with Java 8 > > Thanks > > On Wed, Jul 4, 2018 at 10:24 AM, Manikumar > wrote: > > +1 (non-binding) Verified the

[jira] [Created] (KAFKA-7138) Kafka Connect - Make errors.deadletterqueue.topic.replication.factor default consistent

2018-07-06 Thread Robin Moffatt (JIRA)
Robin Moffatt created KAFKA-7138: Summary: Kafka Connect - Make errors.deadletterqueue.topic.replication.factor default consistent Key: KAFKA-7138 URL: https://issues.apache.org/jira/browse/KAFKA-7138

[jira] [Created] (KAFKA-7137) ability to trigger compaction for tombstoning and GDPR

2018-07-06 Thread Brett Rann (JIRA)
Brett Rann created KAFKA-7137: - Summary: ability to trigger compaction for tombstoning and GDPR Key: KAFKA-7137 URL: https://issues.apache.org/jira/browse/KAFKA-7137 Project: Kafka Issue Type: