Re: Subject: [VOTE] 2.4.1 RC0

2020-03-04 Thread Eno Thereska
Hi Bill, I built from source and ran unit and integration tests. They passed. There was a large number of skipped tests, but I'm assuming that is intentional. Cheers Eno On Tue, Mar 3, 2020 at 8:42 PM Eric Lalonde wrote: > > Hi, > > I ran: > $ https://github.com/elalonde/kafka/blob/master/bin/

Re: [VOTE] KIP-559: Make the Kafka Protocol Friendlier with L7 Proxies

2020-01-22 Thread Eno Thereska
This is awesome! +1 (non binding) Eno On Tue, Jan 21, 2020 at 10:00 PM Gwen Shapira wrote: > > Thank you for the KIP. Awesomely cloud-native improvement :) > > +1 (binding) > > > On Tue, Jan 21, 2020, 9:35 AM David Jacot wrote: > > > Hi all, > > > > I would like to start a vote on KIP-559: Make

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-10-24 Thread Eno Thereska
s do all this, what if we just expose an API that > lets external tooling register a URI for a given segment? If I've copied a > segment file to S3, say with a daily cron job, why not just tell Kafka > where to find it? Assuming I've got a plugin to _read_ from S3, that's all

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-10-24 Thread Eno Thereska
remote storage files and let the follower's RLM picking that up > > > > makes > > > > > > it > > > > > > > easy to have the current replication protocol without any changes. > > > > So we > > > > > > > don't determine

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-10-23 Thread Eno Thereska
; > > > are > > > > > > > > > > older than a configurable time and delegates to the > > > configured > > > > RSM to > > > > > > > > > ship > > > > > > > > > > them to remote storage" (or something like that -- just make > > > > it cl

Re: [VOTE] KIP-416: Notify SourceTask of ACK'd offsets, metadata

2019-10-06 Thread Eno Thereska
+1 (Non binding) Thanks Eno > On 4 Oct 2019, at 15:09, Harsha Chintalapani wrote: > > +1 (binding). > > Thanks, > Harsha > > >> On Fri, Oct 04, 2019 at 6:53 AM, Manikumar >> wrote: >> >> Hi All, >> >> Please vote here for the formal approval of this KIP. >> https://github.com/apache/kaf

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-08-21 Thread Eno Thereska
Hi Colin, Nice KIP! For such a big change it would be good to add a pointer or two to related work that provides some sort of soft proof that the approach taken makes sense. Also such work often builds on other work and it might be useful to trace its roots. May I recommend adding a pointer to "Ta

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-03-28 Thread Eno Thereska
Thanks Harsha, A couple of comments: Performance & durability -- - would be good to have more discussion on performance implications of tiering. Copying the data from the local storage to the remote storage is going to be expensive in terms of network bandwidth and

Re: [VOTE] KIP-392: Allow consumers to fetch from the closest replica

2019-03-25 Thread Eno Thereska
+1 (non-binding) Thanks for updating the KIP and addressing my previous comments. Eno On Mon, Mar 25, 2019 at 4:35 PM Ryanne Dolan wrote: > +1 (non-binding) > > Great stuff, thanks. > > Ryanne > > On Mon, Mar 25, 2019, 11:08 AM Jason Gustafson wrote: > > > Hi All, discussion on the KIP seems t

Re: [DISCUSS] KIP-426: Persist Broker Id to Zookeeper

2019-03-02 Thread Eno Thereska
Hi Harsha, Li Kan, What Colin mentioned is what I see in practice as well (at AWS and our clusters). A control plane management tool decides the mapping hostname-broker ID and can change it as it sees fit as brokers fail and new ones are brought in. That control plane usually already has a databas

Re: [VOTE] 2.1.1 RC2

2019-02-09 Thread Eno Thereska
+1 passes unit + integration tests. Eno On Fri, Feb 8, 2019 at 11:10 PM Magnus Edenhill wrote: > +1 > > Passes librdkafka test suite. > > Den fre 8 feb. 2019 kl 21:02 skrev Colin McCabe : > > > Hi all, > > > > This is the third candidate for release of Apache Kafka 2.1.1. This > > release inclu

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-02-05 Thread Eno Thereska
Thanks Harsha for the KIP. A couple of comments: - the motivation needs more work. I think some of the questions about ETL and other tools would benefit from a clearer motivation. In particular right now the motivation is mostly about reducing recovery time. However I'd argue that this KIP is abou

Re: [VOTE] 2.1.1 RC1

2019-01-30 Thread Eno Thereska
n the Jenkins build we did here: > https://builds.apache.org/job/kafka-2.1-jdk8/118/ > > Perhaps there is an environment issue at play here? Do you get the same > failures running those tests on the 2.1 release? > > Best, > Colin > > On Wed, Jan 30, 2019, at 09:11, Eno Theresk

Re: [VOTE] 2.1.1 RC1

2019-01-30 Thread Eno Thereska
Hi Colin, I've been running the tests and so far I get the following failures. Are they known? kafka.server.ReplicaManagerQuotasTest > shouldGetBothMessagesIfQuotasAllow FAILED kafka.server.ReplicaManagerQuotasTest > testCompleteInDelayedFetchWithReplicaThrottling FAILED kafka.server.ReplicaManag

Re: [VOTE] KIP-396: Add Commit/List Offsets Operations to AdminClient

2019-01-21 Thread Eno Thereska
+1 (non binding). Thanks. On Mon, Jan 21, 2019 at 12:30 PM Mickael Maison wrote: > Bumping this thread. Considering this KIP is relatively straigh > forward, can we get some votes or feedback if you think it's not? > Thanks > > On Tue, Jan 8, 2019 at 5:40 PM Edoardo Comar wrote: > > > > +1 (non

Re: [VOTE] KIP-382 MirrorMaker 2.0

2018-12-21 Thread Eno Thereska
+1 (non-binding) Great stuff! Eno On Fri, Dec 21, 2018 at 10:04 AM Sönke Liebau wrote: > +1 (non-binding) > > Thanks for your effort Ryanne! > > On Fri, Dec 21, 2018 at 2:23 AM Srinivas Reddy > wrote: > > > > +1 (non binding) > > > > Thank you Ryan for the KIP, let me know if you need support

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-12-18 Thread Eno Thereska
Ryanne, thank you, this looks great and will be really appreciated by the community. My only comment at this point: is the REST api strictly necessary for this KIP? Perhaps consider moving that to a different KIP since the main contribution is more than sufficient. However that is just a suggestio

[jira] [Reopened] (KAFKA-4790) Kafka cannot recover after a disk full

2018-12-12 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4790?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reopened KAFKA-4790: - > Kafka cannot recover after a disk f

Re: [DISCUSS] KIP-392: Allow consumers to fetch from the closest replica

2018-12-11 Thread Eno Thereska
g to the ISR. > The main issue, as I've mentioned in the KIP, is the increased latency > before a committed offset is exposed on followers. > > Perhaps I have misunderstood your question? > > Thanks, > Jason > > On Mon, Dec 3, 2018 at 9:18 AM Eno Thereska > wro

Re: [DISCUSS] KIP-392: Allow consumers to fetch from the closest replica

2018-12-03 Thread Eno Thereska
Hi Jason, This is an interesting KIP. This will have massive implications for consistency and serialization, since currently the leader for a partition serializes requests. A few questions for now: - before we deal with the complexity, it'd be great to see a crisp example in the motivation as to

Re: [VOTE] 2.1.0 RC1

2018-11-13 Thread Eno Thereska
Built code and ran tests. Getting a single integration test failure: kafka.log.LogCleanerParameterizedIntegrationTest > testCleansCombinedCompactAndDeleteTopic[3] FAILED java.lang.AssertionError: Contents of the map shouldn't change expected: (340,340), 5 -> (345,345), 10 -> (350,350), 14 -> (

Re: [VOTE] 2.0.1 RC0

2018-11-07 Thread Eno Thereska
.1, if we are going with another RC. > > We need couple of more PMC votes to pass this vote thread. > > On Wed, Nov 7, 2018 at 4:43 PM Eno Thereska > wrote: > > > Two JIRAs are still marked as blockers, although it's not clear to me if > > they really are. Any upda

Re: [VOTE] 2.0.1 RC0

2018-11-07 Thread Eno Thereska
e not blockers for 2.0.1 release. We need more votes from > PMC/committers :) > > Thanks Stanislav! for the system test results. > > Thanks, > Manikumar > > On Thu, Nov 1, 2018 at 10:20 PM Eno Thereska > wrote: > > > Anything else holding this up? > > &g

Re: [VOTE] 2.0.1 RC0

2018-11-01 Thread Eno Thereska
Anything else holding this up? Thanks Eno On Thu, Nov 1, 2018 at 10:27 AM Jakub Scholz wrote: > +1 (non-binding) ... I used the staged binaries and run tests with > different clients. > > On Fri, Oct 26, 2018 at 4:29 AM Manikumar > wrote: > > > Hello Kafka users, developers and client-develope

Re: [VOTE] 2.1.0 RC0

2018-10-30 Thread Eno Thereska
2 tests failed for me and 4 were skipped when doing ./gradlew test: Failed tests: DeleteTopicTest. testAddPartitionDuringDeleteTopic SaslOAuthBearerSslEndToEndAuthorizationTest. testNoConsumeWithDescribeAclViaSubscribe Ignored tests: ConsumerBounceTest. testConsumptionWithBrokerFailures UncleanLe

Re: [VOTE] 2.0.1 RC0

2018-10-30 Thread Eno Thereska
0 src artifacts. > maybe related to gradle task ordering. > I will look into it and try to fix it on trunk. > > Similar issue reported here: > https://jira.apache.org/jira/browse/KAFKA-6500 > > Thanks, > > > On Mon, Oct 29, 2018 at 5:28 PM Eno Thereska > wrote: > &g

Re: [VOTE] 2.0.1 RC0

2018-10-29 Thread Eno Thereska
Thanks. Tested basic building and running of unit and integration tests. They work. Tested docs. The following fails. Is it a known issue? " ./gradlew aggregatedJavadoc with info: > Configure project : Building project 'core' with Scala version 2.11.12 Building project 'streams-scala' with Scala v

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-10-16 Thread Eno Thereska
This update is much needed, thank you! Could you comment on the approach of your method vs. using other open source tools like Uber's uReplicator or the recently open-sourced Mirus from Salesforce? ( https://engineering.salesforce.com/open-sourcing-mirus-3ec2c8a38537). I strongly believe Mirrormake

Re: [DISCUSS] KIP-375: TopicCommand to use AdminClient

2018-09-25 Thread Eno Thereska
This would be very useful. Could you clarify a bit the difference to https://issues.apache.org/jira/browse/KAFKA-5561 since I didn't get it from the JIRA notes. It's fine if you pick up that work but wanted to make sure we're not duplicating efforts. Thanks Eno On Mon, Sep 24, 2018 at 8:26 PM, Vi

Re: [VOTE] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-09-19 Thread Eno Thereska
+1 (non-binding). Thanks Eno On Wed, Sep 19, 2018 at 10:09 AM, Rajini Sivaram wrote: > Hi Edo, > > Thanks for the KIP! > > +1 (binding) > > On Tue, Sep 18, 2018 at 3:51 PM, Edoardo Comar wrote: > > > Hi All, > > > > I'd like to start the vote on KIP-302: > > > > https://cwiki.apache.org/conflu

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-09-18 Thread Eno Thereska
doardo Comar > > IBM Message Hub > > IBM UK Ltd, Hursley Park, SO21 2JN > > > > From: Eno Thereska > To: dev@kafka.apache.org > Date: 18/09/2018 10:24 > Subject:Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all > DNS resolved IP addresses >

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-09-18 Thread Eno Thereska
Hi folks, Could you expand the motivation a bit? When would it make sense to use an LB in front of Kafka brokers? A client needs to access each broker directly to consume the data in that broker and cannot be redirected to another broker. What exact scenario are you seeing that needs this KIP? Th

Re: [DISCUSS] KIP-371: Add a configuration to build custom SSL principal name

2018-09-14 Thread Eno Thereska
Manikumar, thanks. If I understand the KIP motivation right, this is currently already possible, but you want to have an easier way of doing it, right? The motivation would be stronger if we had 2-3 common cases (from experience) where the suggested pattern would solve them, and perhaps 1-2 cases w

Re: [DISCUSS] KIP-372: Naming Joins and Grouping

2018-09-13 Thread Eno Thereska
Hi folks, I know we don't normally have a "Related work" section in KIPs, but sometimes I find it useful to see what others have done in similar cases. Since this will be important for rolling re-deployments, I wonder what other frameworks like Flink (or Samza) have done in these cases. Perhaps th

Re: [DISCUSS] KIP-369: Alternative Partitioner to Support "Always Round-Robin" Selection

2018-08-30 Thread Eno Thereska
Hi there, I can't see this KIP on the KIP wiki, could you double check? Thanks Eno On Thu, Aug 30, 2018 at 9:56 AM, M. Manna wrote: > Hello, > > I opened a very simple KIP and there exists a JIRA for it. > > I have already sent it to users' list - but the mail thread doesn't seem to > get upd

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-22 Thread Eno Thereska
oned in the motivation part. > > Thanks, > > Jiangjie (Becket) Qin > > > On Aug 20, 2018, at 11:33 PM, Eno Thereska > wrote: > > > > Hi folks, > > > > I looked at the previous numbers that Lucas provided (thanks!) but it's > > still not clear to

Re: [ANNOUNCE] New Kafka PMC member: Dong Lin

2018-08-21 Thread Eno Thereska
Congrats Dong! Eno On Tue, Aug 21, 2018 at 7:05 AM, Ted Yu wrote: > Congratulation Dong! > > On Tue, Aug 21, 2018 at 1:59 AM Viktor Somogyi-Vass < > viktorsomo...@gmail.com> > wrote: > > > Congrats Dong! :) > > > > On Tue, Aug 21, 2018 at 10:09 AM James Cheng > wrote: > > > > > Congrats Dong!

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-20 Thread Eno Thereska
Hi folks, I looked at the previous numbers that Lucas provided (thanks!) but it's still not clear to me whether the performance benefits justify the added complexity. I'm looking for some intuition here (a graph would be great but not required): for a small/medium/large cluster, what are the expec

Re: [VOTE] KIP-356: Add withCachingDisabled() to StoreBuilder

2018-08-14 Thread Eno Thereska
+1 (non binding) Thanks Eno On Tue, Aug 14, 2018 at 10:53 AM, Bill Bejeck wrote: > Thanks for the KIP. > > +1 > > -Bill > > On Tue, Aug 14, 2018 at 1:42 PM Guozhang Wang wrote: > > > Hello folks, > > > > I'd like to start a voting thread on the following KIP: > > > > > > https://cwiki.apache.o

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-08-14 Thread Eno Thereska
her issues (such as orphan log segments under certain > > conditions) that lead to GDPR problem but they are more like something > we > > need to fix anyway regardless of GDPR. > > > > > > -- Xiongqi (Wesley) Wu > > > > On Mon, Aug 13, 2018 at 2:

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-08-13 Thread Eno Thereska
Hello, Thanks for the KIP. I'd like to see a more precise definition of what part of GDPR you are targeting as well as some sort of verification that this KIP actually addresses the problem. Right now I find this a bit vague: "Ability to delete a log message through compaction in a timely manner

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-08-13 Thread Eno Thereska
Hi Matthias, Good stuff. Could you comment a bit on how future-proof is this change? For example, if we want to store both event timestamp "and" processing time in RocksDB will we then need another interface (e.g. called KeyValueWithTwoTimestampsStore)? Thanks Eno On Thu, Aug 9, 2018 at 2:30 PM,

Re: [VOTE] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-07-04 Thread Eno Thereska
+1 (non binding) On Wed, Jul 4, 2018 at 1:19 PM, Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > +1 (non-binding) > > On Wed, Jul 4, 2018 at 5:22 PM Magnus Edenhill wrote: > > > +1 (non-binding) > > > > 2018-07-04 13:40 GMT+02:00 Satish Duggana : > > > > > +1 > > > > > > Thanks,

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-20 Thread Eno Thereska
Hi Lucas, Sorry for the delay, just had a look at this. A couple of questions: - did you notice any positive change after implementing this KIP? I'm wondering if you have any experimental results that show the benefit of the two queues. - priority is usually not sufficient in addressing the probl

Re: [ANNOUNCE] New committer: Matthias J. Sax

2018-01-14 Thread Eno Thereska
Congrats Matthias! Eno On Sun, Jan 14, 2018 at 9:52 AM, Rajini Sivaram wrote: > Congratulations Matthias! > > On Sat, Jan 13, 2018 at 11:34 AM, Mickael Maison > > wrote: > > > Congratulations Matthias ! > > > > On Sat, Jan 13, 2018 at 7:01 AM, Paolo Patierno > > wrote: > > > Congratulations M

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Eno Thereska
ams > to > > > output streams. > > > > > > ** The Connector API allows building and running reusable producers or > > > consumers > > > that connect Kafka topics to existing applications or data systems. For > > > example, a connector to a

[jira] [Resolved] (KAFKA-5571) Possible deadlock during shutdown in setState in kafka streams 10.2

2017-08-20 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5571?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska resolved KAFKA-5571. - Resolution: Fixed > Possible deadlock during shutdown in setState in kafka streams 1

[jira] [Created] (KAFKA-5733) System tests get exception RocksDBException: db has more levels than options.num_levels

2017-08-14 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5733: --- Summary: System tests get exception RocksDBException: db has more levels than options.num_levels Key: KAFKA-5733 URL: https://issues.apache.org/jira/browse/KAFKA-5733

[jira] [Created] (KAFKA-5725) Additional failure testing for streams with bouncing brokers

2017-08-11 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5725: --- Summary: Additional failure testing for streams with bouncing brokers Key: KAFKA-5725 URL: https://issues.apache.org/jira/browse/KAFKA-5725 Project: Kafka

[jira] [Created] (KAFKA-5701) Unit test shouldTogglePrepareForBulkLoadDuringRestoreCalls fails

2017-08-04 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5701: --- Summary: Unit test shouldTogglePrepareForBulkLoadDuringRestoreCalls fails Key: KAFKA-5701 URL: https://issues.apache.org/jira/browse/KAFKA-5701 Project: Kafka

[jira] [Resolved] (KAFKA-5301) Improve exception handling on consumer path

2017-07-18 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5301?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska resolved KAFKA-5301. - Resolution: Won't Fix > Improve exception handling on consu

[jira] [Resolved] (KAFKA-5300) Improve exception handling on producer path

2017-07-18 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5300?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska resolved KAFKA-5300. - Resolution: Duplicate Duplicate of KAFKA-5006 > Improve exception handling on producer p

[jira] [Resolved] (KAFKA-5302) Improve exception handling on streams client (communication with brokers)

2017-07-18 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5302?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska resolved KAFKA-5302. - Resolution: Won't Fix > Improve exception handling on streams client (communication with

[jira] [Resolved] (KAFKA-5217) Improve Streams internal exception handling

2017-07-18 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5217?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska resolved KAFKA-5217. - Resolution: Duplicate > Improve Streams internal exception handl

Re: [VOTE] KIP-173: Add prefix to StreamsConfig to enable setting default internal topic configs

2017-07-17 Thread Eno Thereska
+1 (non-binding) Thanks Eno > On Jul 17, 2017, at 12:43 PM, Damian Guy wrote: > > Hi, > > I'd like to kick off the vote for KIP-173: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-173%3A+Add+prefix+to+StreamsConfig+to+enable+setting+default+internal+topic+configs > > A PR for this can

Re: [VOTE] KIP-167: Add interface for the state store restoration process

2017-07-13 Thread Eno Thereska
+1 (non-binding). Thanks Bill. Eno > On 12 Jul 2017, at 09:12, Bill Bejeck wrote: > > All, > > Now that we've concluded a second round of discussion on KIP-167, I'd like > to start a vote. > > > Thanks, > Bill > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-167%3A+Add+interface+fo

[jira] [Reopened] (KAFKA-5566) Instable test QueryableStateIntegrationTest.shouldAllowToQueryAfterThreadDied

2017-07-07 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reopened KAFKA-5566: - > Instable test QueryableStateIntegrationTest.shouldAllowToQueryAfterThreadD

Re: [ANNOUNCE] New Kafka PMC member Ismael Juma

2017-07-07 Thread Eno Thereska
Congrats! Eno > On 7 Jul 2017, at 16:13, Kamal C wrote: > > Congratulations Ismael ! > > On 06-Jul-2017 14:11, "Ismael Juma" wrote: > >> Thanks everyone! >> >> Ismael >> >> On Wed, Jul 5, 2017 at 9:55 PM, Jun Rao wrote: >> >>> Hi, Everyone, >>> >>> Ismael Juma has been active in the Kafk

Re: [VOTE]: KIP-149: Enabling key access in ValueTransformer, ValueMapper, and ValueJoiner

2017-07-07 Thread Eno Thereska
+1 (non-binding) Thanks. Eno > On 6 Jul 2017, at 21:49, Gwen Shapira wrote: > > +1 > > On Wed, Jul 5, 2017 at 9:25 AM Matthias J. Sax > wrote: > >> +1 >> >> On 6/27/17 1:41 PM, Jeyhun Karimov wrote: >>> Dear all, >>> >>> I would like to start the vote on KIP-149 [1]. >>> >>> >>> Cheers, >

[jira] [Created] (KAFKA-5569) Document any changes from this task

2017-07-07 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5569: --- Summary: Document any changes from this task Key: KAFKA-5569 URL: https://issues.apache.org/jira/browse/KAFKA-5569 Project: Kafka Issue Type: Sub-task

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-07-04 Thread Eno Thereska
As part of the PR review we decided to add a metric to keep track of the number of skipped records due to deserialization. I updated the KIP to reflect that. Thanks Eno > On Jun 23, 2017, at 10:59 AM, Eno Thereska wrote: > > Done, thanks. I'll open a vote thread now. > &

[jira] [Created] (KAFKA-5552) testTransactionalProducerTopicAuthorizationExceptionInCommit fails

2017-07-03 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5552: --- Summary: testTransactionalProducerTopicAuthorizationExceptionInCommit fails Key: KAFKA-5552 URL: https://issues.apache.org/jira/browse/KAFKA-5552 Project: Kafka

Re: [DISCUSS] KIP-167: Add interface for the state store restoration process

2017-06-30 Thread Eno Thereska
Thanks Bill, My only remaining question is whether we expect that calling `onBatchRestore` after each `poll` could slow down the restoration? Thanks Eno > On Jun 30, 2017, at 4:27 PM, Bill Bejeck wrote: > > Damian, > > Thanks for comments. I've updated the KIP with the abstract classes. >

Re: [VOTE] KIP-161: streams deserialization exception handlers

2017-06-30 Thread Eno Thereska
>>>> Thanks for the KIP! >>>> >>>> +1 >>>> >>>> -Bill >>>> >>>> On Fri, Jun 23, 2017 at 7:15 AM, Damian Guy >> wrote: >>>> >>>>> Thanks for the KIP Eno. >>>&g

[jira] [Created] (KAFKA-5533) Timestamp-based log compaction

2017-06-28 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5533: --- Summary: Timestamp-based log compaction Key: KAFKA-5533 URL: https://issues.apache.org/jira/browse/KAFKA-5533 Project: Kafka Issue Type: New Feature

[jira] [Created] (KAFKA-5531) Check exception handling in all streams tests

2017-06-28 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5531: --- Summary: Check exception handling in all streams tests Key: KAFKA-5531 URL: https://issues.apache.org/jira/browse/KAFKA-5531 Project: Kafka Issue Type: Sub

[jira] [Reopened] (KAFKA-5006) KeyValueStore.put may throw exception unrelated to the current put attempt

2017-06-27 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5006?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reopened KAFKA-5006: - > KeyValueStore.put may throw exception unrelated to the current put atte

[jira] [Created] (KAFKA-5524) Streams systems tests should be with EoS

2017-06-27 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5524: --- Summary: Streams systems tests should be with EoS Key: KAFKA-5524 URL: https://issues.apache.org/jira/browse/KAFKA-5524 Project: Kafka Issue Type: Improvement

Re: Minimum Replication Factor

2017-06-27 Thread Eno Thereska
Good discussion. Stephane, we did briefly think of your options 1 and 2 but didn’t get time to make a KIP and discuss broadly. A dynamic config is appealing, however it has the drawback that you might end up with an unpredictable replication factor. A minimum replication factor also needs some

[jira] [Resolved] (KAFKA-4563) State transitions error PARTITIONS_REVOKED to NOT_RUNNING

2017-06-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4563?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska resolved KAFKA-4563. - Resolution: Duplicate This will be fixed as part of KAFKA-5372 > State transitions er

[jira] [Created] (KAFKA-5509) add gradle option for building tarball with test jars

2017-06-24 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5509: --- Summary: add gradle option for building tarball with test jars Key: KAFKA-5509 URL: https://issues.apache.org/jira/browse/KAFKA-5509 Project: Kafka Issue Type

Re: [VOTE] KIP-160: Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-06-23 Thread Eno Thereska
+1 thanks! Eno > On 23 Jun 2017, at 05:29, James Chain wrote: > > Hi all, > > I apply original idea on KStream#writeAsText() and also update my pull > request. > Please re-review and re-cast the vote. > > James Chien

[VOTE] KIP-161: streams deserialization exception handlers

2017-06-23 Thread Eno Thereska
Starting voting thread for: https://cwiki.apache.org/confluence/display/KAFKA/KIP-161%3A+streams+deserialization+exception+handlers Thanks Eno

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-23 Thread Eno Thereska
imestamp > extractor as "silent" data loss is no good default behavior IMHO. > > > -Matthias > > On 6/22/17 11:00 AM, Eno Thereska wrote: >> Answers inline: >> >>> On 22 Jun 2017, at 03:26, Guozhang Wang wrote: >>>

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-22 Thread Eno Thereska
is no need for the response config option. Thanks Eno > > Guozhang > > > > > > > > > On Wed, Jun 21, 2017 at 1:39 AM, Eno Thereska <mailto:eno.there...@gmail.com>> > wrote: > >> Thanks Guozhang, >> >> I’ve updated the K

[jira] [Created] (KAFKA-5499) Double check how we handle exceptions when commits fail

2017-06-22 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5499: --- Summary: Double check how we handle exceptions when commits fail Key: KAFKA-5499 URL: https://issues.apache.org/jira/browse/KAFKA-5499 Project: Kafka Issue

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-22 Thread Eno Thereska
roposed > APIs? > > Meanwhile Damian could you provide a formal set of APIs for people to > exercise on them? Also could you briefly describe how custom storage > engines could be swapped in with the above APIs? > > > > Guozhang > > > On Wed, Jun 21, 2017 at 9:0

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Eno Thereska
. >> >> Yes i agree. We could apply this same approach to most of the operations >> where we have multiple overloads, i.e., we have a single method for each >> operation that takes the required parameters and everything else is >> specified as you have done above. &

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Eno Thereska
(cc’ing user-list too) Given that we already have StateStoreSuppliers that are configurable using the fluent-like API, probably it’s worth discussing the other examples with joins and serdes first since those have many overloads and are in need of some TLC. So following your example, I guess yo

[jira] [Created] (KAFKA-5487) Rolling upgrade test for streams

2017-06-21 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5487: --- Summary: Rolling upgrade test for streams Key: KAFKA-5487 URL: https://issues.apache.org/jira/browse/KAFKA-5487 Project: Kafka Issue Type: Bug

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-21 Thread Eno Thereska
probably would like to not rename the > config again. Also from the experience of `default partitioner` and > `default timestamp extractor` we may also make sure that the passed in > object can be either a string "class name" or a class object? > > > Guozhang >

Re: [VOTE] 0.11.0.0 RC1

2017-06-19 Thread Eno Thereska
+1 (non-binding) passes Kafka Streams tests. Thanks, Eno > On 19 Jun 2017, at 06:49, Magnus Edenhill wrote: > > +1 (non-binding) > > Passes librdkafka integration tests (v0.9.5 and master) > > > 2017-06-19 0:32 GMT+02:00 Ismael Juma : > >> Hello Kafka users, developers and client-developers,

Re: [DISCUSS] KIP-167: Add a restoreAll method to StateRestoreCallback

2017-06-14 Thread Eno Thereska
Thanks Bill, A couple of questions: 1. why do we need both restore and restoreAll, why can't we just have one, that takes a collection (i.e., restore all)? Are there cases when people want to restore one at a time? In that case, they could still use restoreAll with just 1 record in the collec

Re: [ANNOUNCE] New committer: Damian Guy

2017-06-09 Thread Eno Thereska
Congrats Damian! Eno > On 9 Jun 2017, at 22:04, Ismael Juma wrote: > > Congratulations Damian! :) > > Ismael > > On Fri, Jun 9, 2017 at 9:34 PM, Guozhang Wang wrote: > >> Hello all, >> >> >> The PMC of Apache Kafka is pleased to announce that we have invited Damian >> Guy as a committer t

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-07 Thread Eno Thereska
Comments inline: > On 5 Jun 2017, at 18:19, Jan Filipiak wrote: > > Hi > > just my few thoughts > > On 05.06.2017 11:44, Eno Thereska wrote: >> Hi there, >> >> Sorry for the late reply, I was out this past week. Looks like good progress >> was

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-05 Thread Eno Thereska
Hi there, Sorry for the late reply, I was out this past week. Looks like good progress was made with the discussions either way. Let me recap a couple of points I saw into one big reply: 1. Jan mentioned CRC errors. I think this is a good point. As these happen in Kafka, before Kafka Streams g

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-05-26 Thread Eno Thereska
izer to a method that creates a state store. In which case it would > be a user error and probably should just abort? > > Thanks, > Damian > > On Fri, 26 May 2017 at 16:32 Eno Thereska wrote: > >> See latest reply to Jan's note. I think I unnecessarily broadened

[jira] [Updated] (KAFKA-3514) Stream timestamp computation needs some further thoughts

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3514?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-3514: Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Stream timestamp computation ne

[jira] [Updated] (KAFKA-4641) Improve test coverage of StreamsThread

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4641?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4641: Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Improve test coverage

[jira] [Updated] (KAFKA-3779) Add the LRU cache for KTable.to() operator

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3779?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-3779: Fix Version/s: (was: 0.11.0.0) > Add the LRU cache for KTable.to() opera

[jira] [Updated] (KAFKA-4640) Improve Streams unit test coverage

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4640?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4640: Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Improve Streams unit test cover

[jira] [Updated] (KAFKA-4653) Improve test coverage of RocksDBStore

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4653?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4653: Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Improve test coverage

[jira] [Updated] (KAFKA-4643) Improve test coverage of StreamsKafkaClient

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4643?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4643: Fix Version/s: (was: 0.11.0.0) > Improve test coverage of StreamsKafkaCli

[jira] [Updated] (KAFKA-4651) Improve test coverage of Stores

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4651?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4651: Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Improve test coverage of Sto

[jira] [Updated] (KAFKA-4650) Improve test coverage org.apache.kafka.streams.kstream.internals

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4650?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4650: Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Improve test cover

[jira] [Updated] (KAFKA-4655) Improve test coverage of CompositeReadOnlySessionStore

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4655?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4655: Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Improve test coverage

[jira] [Updated] (KAFKA-4658) Improve test coverage InMemoryKeyValueLoggedStore

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4658?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4658: Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Improve test cover

[jira] [Updated] (KAFKA-4659) Improve test coverage of CachingKeyValueStore

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4659?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4659: Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Improve test coverage

[jira] [Updated] (KAFKA-4656) Improve test coverage of CompositeReadOnlyKeyValueStore

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4656?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4656: Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Improve test coverage

[jira] [Updated] (KAFKA-4730) Streams does not have an in-memory windowed store

2017-05-26 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4730?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4730: Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Streams does not have an in-mem

  1   2   3   4   5   6   7   8   >