Re: [VOTE] KIP-659: Improve TimeWindowedDeserializer and TimeWindowedSerde to handle window size

2020-09-29 Thread Sophie Blee-Goldman
But I'm not sure how this would help > > with 2.b) since the proposal is on adding to StreamsConfig. If it is the > > case, then I agree that probably we can just not add an extra config but > > just deprecating the constructs. > > > > > > Guozhang > > > >

Re: [DISCUSS] KIP-598: Augment TopologyDescription with store and source / sink serde information

2020-09-29 Thread Sophie Blee-Goldman
Hey Guozhang, what's the status of this KIP? I was recently digging through a particularly opaque Streams application and it occurred to me that it might also be useful to print the kind of store attached to each node (eg RocksDBWindowStore, InMemoryKeyValueStore, custom, etc). That made me think

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread Sophie Blee-Goldman
onal would naturally raise awareness of this subtlety On Tue, Sep 29, 2020 at 7:40 PM Sophie Blee-Goldman wrote: > Does my reply address your concerns? > > > Yes; also, I definitely misread part of the proposal earlier and thought > you had put > the timestamp field in RecordMe

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread Sophie Blee-Goldman
tadata split, we will have the freedom to > move fields into the Record class (or even add new fields) > if we want them to become "data" as opposed to "metadata" in > the future. > > Thanks for your reply; I was similarly floored when I > realized the true nature of the cu

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread Sophie Blee-Goldman
a richer > > > > > > > > > way of querying its data (like perhaps getting all values > > > > > > > > > associated with a secondary key), while still ultimately > > > > > > > > > writing to the changelog topic for

Re: [VOTE] KIP-666: Add Instant-based methods to ReadOnlySessionStore

2020-09-21 Thread Sophie Blee-Goldman
Thanks for pointing out the vote in the discussion thread, this email somehow skipped my inbox ¯\_(ツ)_/¯ I'm +1 (non-binding) -Sophie On Mon, Sep 7, 2020 at 4:18 AM Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Hi everyone, > > I'd like to start a thread to vote for

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-18 Thread Sophie Blee-Goldman
Makes sense to me :) On Thu, Sep 17, 2020 at 9:34 AM Bruno Cadonna wrote: > Hi Sophie, > > Thank you for the feedback! I replied inline. > > Best, > Bruno > > On 16.09.20 19:19, Sophie Blee-Goldman wrote: > >> > >> We guarantee that the metadata of the

[jira] [Resolved] (KAFKA-6127) Streams should never block infinitely

2020-09-18 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-6127. Resolution: Fixed > Streams should never block infinit

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-16 Thread Sophie Blee-Goldman
t; existing close() method to accomplish the shutdown from inside an > >>> uncaught exception handler. But I completely agree with you that we > need > >>> to ensure that we do not run into a deadlock in this case. > >>> > >>> > >>&

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2020-09-14 Thread Sophie Blee-Goldman
Hey all, I'm not convinced either epoch-aligned or data-aligned will fit all possible use cases. Both seem totally reasonable to me: data-aligned is useful for example when you know that a large number of updates to a single key will occur in short bursts, and epoch- aligned when you specifically

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-11 Thread Sophie Blee-Goldman
Hey Walker, The proposal makes sense to me, but while reading up on those old tickets I started wondering if we should give users two options: one that would shut down the entire application, as described in the current KIP, and another that would only shut down an individual instance. I think

[jira] [Resolved] (KAFKA-7970) Missing topic causes service shutdown without exception

2020-09-11 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7970?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-7970. Fix Version/s: 2.7.0 Assignee: Bruno Cadonna Resolution: Fixed

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-10 Thread Sophie Blee-Goldman
they have had in their > stores all along, as well as the new recommended pattern > (just pass everything you need in the value). If that's > unsatisfying, _then_ we should consider amending the API. > > Thanks, > -John > > On Thu, 2020-09-10 at 15:21 -0700, Sophie Blee-Goldm

Re: KIP-669: Preserve Source Partition in Kafka Streams from context

2020-09-10 Thread Sophie Blee-Goldman
Hey Balan, thanks for the KIP! The motivation here makes sense to me, but I have a few questions about the proposed API I guess the main thing to point out is that if we just add new addSink() overloads to Topology, then only the lower level Processor API will benefit and users of the DSL won't

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-10 Thread Sophie Blee-Goldman
It sounds like you're basically in favor of the conservative > approach, and you just wanted to understand the blockers > that I implied. Does my clarification make sense? > > Thanks, > -John > > On Thu, 2020-09-10 at 10:54 -0700, Sophie Blee-Goldman > wro

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-10 Thread Sophie Blee-Goldman
s. Just that it > seems a little that we can be conservative and avoid them for now. If it > turns out we really need to solve them, we can always do it later. > > Thanks, > John > > On Wed, Sep 9, 2020, at 22:46, Sophie Blee-Goldman wrote: > > > > > > If yo

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-09 Thread Sophie Blee-Goldman
> > If you were to call "put" from a punctuator, or do a > `range()` query and then update one of those records with > `put()`, you'd have a very subtle bug on your hands. Can you elaborate on this a bit? I agree that the punctuator case is an obvious exemption to the assumption that store

Re: [VOTE] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

2020-09-09 Thread Sophie Blee-Goldman
+1 from me as well (non-binding) Gokul, it looks you've now received enough binding votes and the vote has been open for sufficiently long. You can conclude the vote and open the PR for review when it's ready Cheers, Sophie On Wed, Sep 9, 2020 at 2:47 PM Guozhang Wang wrote: > +1. Thanks for

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-09 Thread Sophie Blee-Goldman
; threads of a Kafka Streams client. > > > > > > -Matthias > > > > On 9/8/20 2:37 PM, Sophie Blee-Goldman wrote: > >> Ah, I forgot about localThreadsMetadata(). In that. case I agree, > there's > >> no reason > >> to introduce a new method

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-08 Thread Sophie Blee-Goldman
hread's name (as `Optional`) for both add() and > remove() is fine with me. > > > -Matthias > > On 9/8/20 12:58 PM, Sophie Blee-Goldman wrote: > > Sorry Bruno, I think I missed the end of your message with the > > numberOfAliveStreamThreads() > > proposal. I agr

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-08 Thread Sophie Blee-Goldman
hould report. If others also think that > the proposed name in the KIP is too clumsy, I am open to rename it, though. > > Best, > Bruno > > > On 08.09.20 20:12, Sophie Blee-Goldman wrote: > >> it's never a good sign when the discussion moves into the vote thread > &g

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-08 Thread Sophie Blee-Goldman
tional. A caller can catch an > > exception or react to a `null` return value just the same > > either way. Having both add/remove methods behave similarly > > is probably more valuable. > > > > Thanks, > > -John > > > > > > On Thu, 2020-09-03 at 12:1

Re: [VOTE] KIP-659: Improve TimeWindowedDeserializer and TimeWindowedSerde to handle window size

2020-09-08 Thread Sophie Blee-Goldman
hias J. Sax > wrote: > > > > > > > +1 (binding) > > > > > > > > On 8/26/20 8:02 AM, John Roesler wrote: > > > > > Hi all, > > > > > > > > > > I've just sent a new message to the DISCUSS thread. We > &g

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-09-03 Thread Sophie Blee-Goldman
ing the processing threads" I only considered them in > > `RUNNING` and thought we figure out the detail on the PR... Excellent > catch! > > > > Changing state transitions is to some extend backward incompatible, but > > I think (IIRC) we did it in the past and I personally te

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

2020-09-03 Thread Sophie Blee-Goldman
ever contribution - is the next step to initiate the > voting on this KIP? > > -Gokul > > On 04-09-2020 00:34, Sophie Blee-Goldman wrote: > > I think the current proposal looks good to me. One minor suggestion I > have > > is to consider keeping the same error

Re: [VOTE] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-03 Thread Sophie Blee-Goldman
Hey, sorry for the late reply, I just have one minor suggestion. Since we don't make any guarantees about which thread gets removed or allow the user to specify, I think we should return either the index or full name of the thread that does get removed by removeThread(). I know you just updated

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

2020-09-03 Thread Sophie Blee-Goldman
;> Extending from `RetriableException` seems reasonable as well. I guess > >> the > >> only question is whether it's safe to catch it as a `RetriableException` > >> and apply common retry logic. For a transactional producer, my > >> expectation > >&g

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-09-02 Thread Sophie Blee-Goldman
If we're going to add a new GLOBAL_RESTORING state to the KafkaStreams FSM, maybe it would make sense to add a new plain RESTORING state that we transition to when restoring non-global state stores following a rebalance. Right now all restoration occurs within the REBALANCING state, which is

[jira] [Created] (KAFKA-10455) Probing rebalances are not guaranteed to be triggered by non-leader members

2020-09-01 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10455: --- Summary: Probing rebalances are not guaranteed to be triggered by non-leader members Key: KAFKA-10455 URL: https://issues.apache.org/jira/browse/KAFKA-10455

Re: [DISCUSS] KIP-667: Remove deprecated methods from ReadOnlyWindowStore

2020-08-31 Thread Sophie Blee-Goldman
Thanks for this KIP as well! Seems like the methods were deprecated in 2.1. What's our rule for how something has to stay deprecated before we can go ahead and remove it? Assuming 3.0 comes after 2.8, it certainly seems like enough time/releases have passed for us to do so in 3.0. But I'm pretty

Re: [DISCUSS] KIP-666: Add Instant-based methods to ReadOnlySessionStore

2020-08-31 Thread Sophie Blee-Goldman
Thanks for bringing the IQ API into alignment -- the proposal looks good, although one nit: you missed updating the startTime long to Instant in both appearances of the fetchSession(key, startTime, sessionEndTime) method. Also, I think by "startTime" you actually meant "earliestSessionEndTime".

Re: There is an error on the main page

2020-08-31 Thread Sophie Blee-Goldman
Thanks for the heads up. Would you be interested in submitting a PR to fix this? The typo seems to appear in two files, quickstart-docker.html and quickstart-zookeeper.html, of the kafka-site repo . On Mon, Aug 31, 2020 at 8:43 AM Liu Lycos wrote: > Hello,

Re: [DISCUSS] KIP-659: Improve TimeWindowedDeserializer and TimeWindowedSerde to handle window size

2020-08-27 Thread Sophie Blee-Goldman
ur collective > > > > instinct is that Leah's proposal doesn't need to change to > > > > account for whatever we might decide to do in KIP-645. > > > > > > > > I have no further concerns about KIP-645, and I think it's a > > > > good

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

2020-08-27 Thread Sophie Blee-Goldman
Hey Gokul, thanks for taking up this KIP! I agree with Matthias that directly extending KafkaException may not be ideal, and we should instead extend APIException or RetriableException. Of the two, I think APIException would be more appropriate. My understanding is that RetriableException is

Re: [VOTE] KIP-659: Improve TimeWindowedDeserializer and TimeWindowedSerde to handle window size

2020-08-24 Thread Sophie Blee-Goldman
Thanks for the KIP! +1 (non-binding) Sophie On Mon, Aug 24, 2020 at 5:06 PM John Roesler wrote: > Thanks Leah, > I’m +1 (binding) > > -John > > On Mon, Aug 24, 2020, at 16:54, Leah Thomas wrote: > > Hi everyone, > > > > I'd like to kick-off the vote for KIP-659: Improve > >

Re: [DISCUSS] KIP-662: Throw Exception when Source Topics of a Streams App are Deleted

2020-08-21 Thread Sophie Blee-Goldman
Thanks for the KIP! I'm totally in favor of this approach and to be honest, have always wondered why we just silently shut down instead of throwing an exception. This has definitely been a source of confusion for users in my personal experience. I was originally hesitant to extend

Re: [DISCUSS] KIP-659: Improve TimeWindowedDeserializer and TimeWindowedSerde to handle window size

2020-08-21 Thread Sophie Blee-Goldman
Just want to make a quick comment on the question that John raised about whether we should introduce a separate config for "key" and "value" window sizes: My short answer is No, I don't think that's necessary. First of all, as you said, there is no first-class concept of a "Windowed value" in the

[jira] [Resolved] (KAFKA-10395) TopologyTestDriver does not work with dynamic topic routing

2020-08-18 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10395?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-10395. - Fix Version/s: 2.7.0 Resolution: Fixed > TopologyTestDriver d

[jira] [Created] (KAFKA-10395) TopologyTestDriver does not work with dynamic topic routing

2020-08-12 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10395: --- Summary: TopologyTestDriver does not work with dynamic topic routing Key: KAFKA-10395 URL: https://issues.apache.org/jira/browse/KAFKA-10395 Project

Re: [VOTE] KIP-614: Add Prefix Scan support for State Stores

2020-08-04 Thread Sophie Blee-Goldman
Thanks Sagar! +1 (non-binding) Sophie On Sun, Aug 2, 2020 at 11:37 PM Sagar wrote: > Hi All, > > Just thought of bumping this voting thread again to see if we can form any > consensus around this. > > Thanks! > Sagar. > > > On Mon, Jul 20, 2020 at 4:21 AM Adam Bellemare > wrote: > > > LGTM >

Re: [VOTE] KIP-450: Sliding Window Aggregations in the DSL

2020-07-28 Thread Sophie Blee-Goldman
Thanks for the KIP! It's been an enlightening discussion +1 (non-binding) Sophie On Tue, Jul 28, 2020 at 8:03 AM Leah Thomas wrote: > Hi all, > > I'd like to kick-off the vote for KIP-450 > < > https://cwiki.apache.org/confluence/display/KAFKA/KIP-450%3A+Sliding+Window+Aggregations+in+the+DSL

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-28 Thread Sophie Blee-Goldman
Different implementations can (and do) > enumerate > >>>> different windows to suit different use cases. > >>>> > >>>> On the other hand, I can’t think of any way to extend SessionWindows > to > >>>> do something diffe

Re: [DISCUSS] KIP-645: Replace abstract class Windows with a proper interface

2020-07-28 Thread Sophie Blee-Goldman
that you wouldn't be able to join a TimeWindow > > with a SessionWindow, for example. > > > > However, because of covariance, it's more painful to change Window > > than Windows, so it might not be worth it right now. If anything, it > > would be more feasible to

Re: [DISCUSS] KIP-645: Replace abstract class Windows with a proper interface

2020-07-27 Thread Sophie Blee-Goldman
e of using a > > base class vs the interface, is it making the code fragile, or requiring > > redundant implementation, etc. > > > > Boyang > > > > On Tue, Jul 21, 2020 at 2:19 PM Sophie Blee-Goldman > > > wrote: > > > > > Hey John, > >

Re: [VOTE] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2020-07-24 Thread Sophie Blee-Goldman
Thanks all, +1 (non-binding) Cheers, Sophie On Wed, Jul 8, 2020 at 4:02 AM Bruno Cadonna wrote: > Thanks Will and Piotr, > > +1 (non-binding) > > Best, > Bruno > > On Wed, Jul 8, 2020 at 8:12 AM Matthias J. Sax wrote: > > > > Thanks for the KIP. > > > > +1 (binding) > > > > > > -Matthias > >

Re: Re: [DISCUSS] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-07-23 Thread Sophie Blee-Goldman
s, I decided to make some major modifications to this > KIP > > >> and kick off discussion again. > > >> > > >> > > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-607%3A+Add+Metrics+to+Kafka+Streams+to+Report+Pr

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-22 Thread Sophie Blee-Goldman
> > > As a matter of fact, even if the user might not be aware of it, the > > algorithm we use does already leak into the API (if a user extends > > `Windows` is must work with our hopping/tumbling window algorithm and if > > a user extends `SessionWindows` it mus

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-22 Thread Sophie Blee-Goldman
Hey John, Just a few follow-up questions/comments about the whole Windows thing: That's a good way of looking at things; in particular the point about SessionWindows for example requiring a Merger while other "statically enumerable" windows require only an adder seems to touch on the heart of

Re: [DISCUSS] KIP-645: Replace abstract class Windows with a proper interface

2020-07-21 Thread Sophie Blee-Goldman
Hey John, Thanks for the KIP. I know this has been bugging you :) That said, I think the KIP is missing some elaboration in the Motivation section. You mention a number of problems we've had and lived with in the past -- could you give an example of one, and how it would be solved by your

Re: [VOTE] KIP-617: Allow Kafka Streams State Stores to be iterated backwards

2020-07-16 Thread Sophie Blee-Goldman
Hey Jorge, Thanks for the reminder -- +1 (non-binding) Cheers, Sophie On Thu, Jul 16, 2020 at 4:06 PM Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Bumping this vote thread to check if there's any feedback. > > Cheers, > Jorge. > > On Sat, Jul 4, 2020 at 6:20 PM John

[jira] [Reopened] (KAFKA-9161) Close gaps in Streams configs documentation

2020-07-15 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9161?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman reopened KAFKA-9161: > Close gaps in Streams configs documentat

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-10 Thread Sophie Blee-Goldman
Thanks Leah! This kind of assumes an implicit answer to Matthias's question, but I was wondering if we should take this opportunity to choose a better default value for the grace period. Note that the default of -1 in the TimeWindows class, for example, ultimately gets translated into a default

[jira] [Created] (KAFKA-10264) Flaky Test TransactionsTest.testBumpTransactionalEpoch

2020-07-09 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10264: --- Summary: Flaky Test TransactionsTest.testBumpTransactionalEpoch Key: KAFKA-10264 URL: https://issues.apache.org/jira/browse/KAFKA-10264 Project: Kafka

[jira] [Created] (KAFKA-10262) StateDirectory is not thread-safe

2020-07-09 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10262: --- Summary: StateDirectory is not thread-safe Key: KAFKA-10262 URL: https://issues.apache.org/jira/browse/KAFKA-10262 Project: Kafka Issue Type

[jira] [Created] (KAFKA-10251) Flaky Test kafka.api.TransactionsBounceTest.testWithGroupMetadata

2020-07-08 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10251: --- Summary: Flaky Test kafka.api.TransactionsBounceTest.testWithGroupMetadata Key: KAFKA-10251 URL: https://issues.apache.org/jira/browse/KAFKA-10251

Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-07-08 Thread Sophie Blee-Goldman
or 2.6.0, please mark the issue's `fix version` as `2.6.0` > > > _and_ > > > > > respond to this thread with details, and I will work with you to > > > > determine > > > > > whether it is indeed a blocker. > > > > > > > > > > As always, let

[jira] [Created] (KAFKA-10249) In-memory stores are skipped when checkpointing but not skipped when reading the checkpoint

2020-07-08 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10249: --- Summary: In-memory stores are skipped when checkpointing but not skipped when reading the checkpoint Key: KAFKA-10249 URL: https://issues.apache.org/jira/browse

[jira] [Resolved] (KAFKA-10191) fix flaky StreamsOptimizedTest

2020-07-07 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-10191. - Resolution: Fixed > fix flaky StreamsOptimizedT

[jira] [Reopened] (KAFKA-10166) Excessive TaskCorruptedException seen in testing

2020-06-30 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10166?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman reopened KAFKA-10166: - Assignee: (was: Bruno Cadonna) Found two edge cases we missed earlier so

[jira] [Resolved] (KAFKA-6508) Look into optimizing StreamPartitionAssignor StandbyTask Assignment

2020-06-29 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6508?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-6508. Resolution: Fixed This should be addressed via KIP-441 > Look into optimiz

[jira] [Resolved] (KAFKA-10166) Excessive TaskCorruptedException seen in testing

2020-06-26 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10166?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-10166. - Resolution: Fixed > Excessive TaskCorruptedException seen in test

[jira] [Reopened] (KAFKA-8197) Flaky Test kafka.server.DynamicBrokerConfigTest > testPasswordConfigEncoderSecretChange

2020-06-24 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8197?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman reopened KAFKA-8197: Saw this fail again:  kafka.server.DynamicBrokerConfigTest

Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-06-24 Thread Sophie Blee-Goldman
t; > > > already complete will need to push out to the next release > (either > > > 2.7 > > > > or > > > > > > 3.0). There are a few PRs for KIPs that are nearing completion, > and > > > > we're > > > > > > having some Jenkins build issues. I wi

[jira] [Created] (KAFKA-10198) Dirty tasks may be recycled instead of closed

2020-06-24 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10198: --- Summary: Dirty tasks may be recycled instead of closed Key: KAFKA-10198 URL: https://issues.apache.org/jira/browse/KAFKA-10198 Project: Kafka

[jira] [Created] (KAFKA-10186) Aborting transaction with pending data should throw non-fatal exception

2020-06-18 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10186: --- Summary: Aborting transaction with pending data should throw non-fatal exception Key: KAFKA-10186 URL: https://issues.apache.org/jira/browse/KAFKA-10186

[jira] [Reopened] (KAFKA-9509) Fix flaky test MirrorConnectorsIntegrationTest.testReplication

2020-06-17 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9509?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman reopened KAFKA-9509: Saw this fail again on a PR build: h3. Stacktrace java.lang.RuntimeException: Could

[jira] [Resolved] (KAFKA-10151) Flaky Test StoreUpgradeIntegrationTest.shouldMigratePersistentKeyValueStoreToTimestampedKeyValueStoreUsingPapi

2020-06-17 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10151?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-10151. - Fix Version/s: 2.6.0 Resolution: Fixed > Flaky T

[jira] [Reopened] (KAFKA-9974) Flaky Test OptimizedKTableIntegrationTest#shouldApplyUpdatesToStandbyStore

2020-06-17 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9974?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman reopened KAFKA-9974: Unfortunately this just failed again, at a different place: h3. Stacktrace

[jira] [Created] (KAFKA-10177) Replace/improve/remove Percentiles metric

2020-06-16 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10177: --- Summary: Replace/improve/remove Percentiles metric Key: KAFKA-10177 URL: https://issues.apache.org/jira/browse/KAFKA-10177 Project: Kafka

[jira] [Reopened] (KAFKA-9896) Flaky Test StandbyTaskEOSIntegrationTest#surviveWithOneTaskAsStandby[exactly_once_beta]

2020-06-16 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9896?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman reopened KAFKA-9896: Assignee: Sophie Blee-Goldman (was: Matthias J. Sax) Saw this fail again

[jira] [Created] (KAFKA-10169) KafkaException: Failing batch since transaction was aborted

2020-06-15 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10169: --- Summary: KafkaException: Failing batch since transaction was aborted Key: KAFKA-10169 URL: https://issues.apache.org/jira/browse/KAFKA-10169 Project

[jira] [Created] (KAFKA-10166) Excessive TaskCorruptedException seen in soak

2020-06-15 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10166: --- Summary: Excessive TaskCorruptedException seen in soak Key: KAFKA-10166 URL: https://issues.apache.org/jira/browse/KAFKA-10166 Project: Kafka

[jira] [Created] (KAFKA-10165) Percentiles metric leaking memory

2020-06-15 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10165: --- Summary: Percentiles metric leaking memory Key: KAFKA-10165 URL: https://issues.apache.org/jira/browse/KAFKA-10165 Project: Kafka Issue Type

[jira] [Created] (KAFKA-10156) Flaky Test SslSelectorTest.testCloseOldestConnection

2020-06-11 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10156: --- Summary: Flaky Test SslSelectorTest.testCloseOldestConnection Key: KAFKA-10156 URL: https://issues.apache.org/jira/browse/KAFKA-10156 Project: Kafka

[jira] [Created] (KAFKA-10155) Flaky Test ReassignPartitionsUnitTest.testModifyBrokerThrottles

2020-06-11 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10155: --- Summary: Flaky Test ReassignPartitionsUnitTest.testModifyBrokerThrottles Key: KAFKA-10155 URL: https://issues.apache.org/jira/browse/KAFKA-10155

[jira] [Created] (KAFKA-10152) Attempt to write checkpoint without first committing during recycle

2020-06-11 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10152: --- Summary: Attempt to write checkpoint without first committing during recycle Key: KAFKA-10152 URL: https://issues.apache.org/jira/browse/KAFKA-10152

[jira] [Created] (KAFKA-10151) Flaky Test StoreUpgradeIntegrationTest.shouldMigratePersistentKeyValueStoreToTimestampedKeyValueStoreUsingPapiStoreUpgradeIntegrationTest.shouldMigratePersistentKeyValue

2020-06-11 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10151: --- Summary: Flaky Test

[jira] [Created] (KAFKA-10150) IllegalStateException when revoking task in CREATED state

2020-06-11 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10150: --- Summary: IllegalStateException when revoking task in CREATED state Key: KAFKA-10150 URL: https://issues.apache.org/jira/browse/KAFKA-10150 Project

[jira] [Created] (KAFKA-10144) Corrupted standby tasks are not always cleaned up

2020-06-10 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10144: --- Summary: Corrupted standby tasks are not always cleaned up Key: KAFKA-10144 URL: https://issues.apache.org/jira/browse/KAFKA-10144 Project: Kafka

[jira] [Resolved] (KAFKA-10119) StreamsResetter fails with TimeoutException for older Brokers

2020-06-10 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10119?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-10119. - Resolution: Duplicate Closing this as a duplicate, the issue is tracked

[jira] [Resolved] (KAFKA-9923) Join window store duplicates can be compacted in changelog

2020-06-09 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9923?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-9923. Resolution: Not A Problem [~cadonna] pointed out that this actually ins't

[jira] [Created] (KAFKA-10102) Source node references not updated after rebuilding topology

2020-06-04 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10102: --- Summary: Source node references not updated after rebuilding topology Key: KAFKA-10102 URL: https://issues.apache.org/jira/browse/KAFKA-10102 Project

[jira] [Created] (KAFKA-10085) Compute lag correctly for optimized source changelogs

2020-06-02 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10085: --- Summary: Compute lag correctly for optimized source changelogs Key: KAFKA-10085 URL: https://issues.apache.org/jira/browse/KAFKA-10085 Project: Kafka

[jira] [Resolved] (KAFKA-9987) Improve sticky partition assignor algorithm

2020-06-02 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9987?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-9987. Fix Version/s: 2.5.1 2.4.2 2.6.0

[jira] [Created] (KAFKA-10079) Support high availability for in-memory stores

2020-06-01 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10079: --- Summary: Support high availability for in-memory stores Key: KAFKA-10079 URL: https://issues.apache.org/jira/browse/KAFKA-10079 Project: Kafka

[jira] [Resolved] (KAFKA-9821) Stream task may skip assignment with static members and incremental rebalances

2020-06-01 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9821?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-9821. Resolution: Fixed > Stream task may skip assignment with static memb

[jira] [Created] (KAFKA-10078) Partition may skip assignment with static members and incremental rebalances

2020-06-01 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10078: --- Summary: Partition may skip assignment with static members and incremental rebalances Key: KAFKA-10078 URL: https://issues.apache.org/jira/browse/KAFKA-10078

[jira] [Resolved] (KAFKA-6145) Warm up new KS instances before migrating tasks - potentially a two phase rebalance

2020-05-27 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6145?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-6145. Fix Version/s: 2.6.0 Resolution: Fixed > Warm up new KS instances bef

[jira] [Created] (KAFKA-10054) Add TRACE-level end-to-end latency metrics to Streams

2020-05-27 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10054: --- Summary: Add TRACE-level end-to-end latency metrics to Streams Key: KAFKA-10054 URL: https://issues.apache.org/jira/browse/KAFKA-10054 Project: Kafka

[jira] [Created] (KAFKA-10052) Flaky Test InternalTopicsIntegrationTest.testCreateInternalTopicsWithFewerReplicasThanBrokers

2020-05-27 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10052: --- Summary: Flaky Test InternalTopicsIntegrationTest.testCreateInternalTopicsWithFewerReplicasThanBrokers Key: KAFKA-10052 URL: https://issues.apache.org/jira/browse

Re: [VOTE] KIP-617: Allow Kafka Streams State Stores to be iterated backwards

2020-05-27 Thread Sophie Blee-Goldman
Looks good to me. Thanks Jorge! +1 (non-binding) -Sophie On Tue, May 26, 2020 at 3:43 PM Jorge Quilcate wrote: > Hi everyone, > > I'd like to start a vote on KIP-617: Allow Kafka Streams State Stores to > be iterated backwards. > > KIP wiki page: > >

Re: [DISCUSS] KIP-613: Add end-to-end latency metrics to Streams

2020-05-22 Thread Sophie Blee-Goldman
On Mon, May 18, 2020 at 9:32 AM Guozhang Wang wrote: > Thanks Sophie, I'm +1 on the updated KIP. > > On Mon, May 18, 2020 at 8:32 AM Sophie Blee-Goldman > wrote: > > > @Bruno > > No, this would be after inserting in the cache/state store/etc -- > basically > > on

Re: [DISCUSS] KIP-617: Allow Kafka Streams State Stores to be iterated backwards

2020-05-21 Thread Sophie Blee-Goldman
> want to point this out. > > > > > > If moving to this approach, I'd like to check if I got this right > before > > > updating the KIP: > > > > > > - only `StateStore` will change by having a new method: > > > `backwardIteration()`, `fal

Re: [DISCUSS] KIP-617: Allow Kafka Streams State Stores to be iterated backwards

2020-05-20 Thread Sophie Blee-Goldman
ct change. > > To echo your clarification, I'm also not advocating strongly in favor of my > proposal. I just wanted to present it for consideration alongside Jorge's > original one. > > Thanks for raising these very good points, > -John > > On Tue, May 19, 2020, at 20:49, Sop

Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-05-20 Thread Sophie Blee-Goldman
Hey Randall, Can you also add KIP-613 which was accepted yesterday? Thanks! Sophie On Wed, May 20, 2020 at 6:47 AM Randall Hauch wrote: > Hi, Tom. I saw last night that the KIP had enough votes before today’s > deadline and I will add it to the roadmap today. Thanks for driving this! > > On

Re: [VOTE] KIP-613: Add end-to-end latency metrics to Streams

2020-05-19 Thread Sophie Blee-Goldman
d. I still > have > > > > some comment about the processor-node level metrics and left a reply > on > > > > that thread. > > > > > > > > > > > > Guozhang > > > > > > > > On Fri, May 15, 2020 at 2:34 PM Sophie B

Re: [DISCUSS] KIP-617: Allow Kafka Streams State Stores to be iterated backwards

2020-05-19 Thread Sophie Blee-Goldman
he Jira. > > It doesn’t seem like it should be this KIP’s concern that some serdes > might be incorrectly written. > > Were there other practical concerns that you had in mind? > > Thanks, > John > > On Tue, May 19, 2020, at 19:10, Sophie Blee-Goldman wrote: > > I l

Re: [DISCUSS] KIP-617: Allow Kafka Streams State Stores to be iterated backwards

2020-05-19 Thread Sophie Blee-Goldman
I like this "fancy idea" to just flip the to/from bytes but I think there are some practical limitations to implementing this. In particular I'm thinking about this issue with the built-in signed number serdes. This trick would actually fix the

[jira] [Created] (KAFKA-10025) Segfault in RocksDB Statistics

2020-05-19 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10025: --- Summary: Segfault in RocksDB Statistics Key: KAFKA-10025 URL: https://issues.apache.org/jira/browse/KAFKA-10025 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-10017) Flaky Test EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]

2020-05-18 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10017: --- Summary: Flaky Test EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true] Key: KAFKA-10017 URL: https://issues.apache.org/jira/browse/KAFKA-10017

<    1   2   3   4   5   6   7   8   9   >