Hi,

I do understand that in a general case it's not possible to guarantee
that newValue and oldValue parts of a Change message arrive to the
same partitions, and I guess that's not really in the plans, but if I
correctly understand how it works, it should be possible to detect if
both newValue and oldValue go to the same partition and keep them
together, thus improving kafka-streams consistency guarantees. Right?

For example right now I have such a usecase that when I perform
groupBy on a table, my new keys are computed purely from old keys, and
not from the value. And handling of such cases (not a general case)
can be improved.
On Tue, Jul 17, 2018 at 1:48 AM Matthias J. Sax <matth...@confluent.io> wrote:
>
> It is not possible to use a single message, because both messages may go
> to different partitions and may be processed by different applications
> instances.
>
> Note, that the overall KTable state is sharded. Updating a single
> upstream shard, might required to update two different downstream shards.
>
>
> -Matthias
>
> On 7/16/18 2:50 PM, Vasily Sulatskov wrote:
> > Hi,
> >
> > It seems that it wouldn't be that difficult to address: just don't
> > break Change(newVal, oldVal) into Change(newVal, null) /
> > Change(oldVal, null) and update aggregator value in one .process()
> > call.
> >
> > Would this change make sense?
> > On Mon, Jul 16, 2018 at 10:34 PM Matthias J. Sax <matth...@confluent.io> 
> > wrote:
> >>
> >> Vasily,
> >>
> >> yes, it can happen. As you noticed, both messages might be processed on
> >> different machines. Thus, Kafka Streams provides 'eventual consistency'
> >> guarantees.
> >>
> >>
> >> -Matthias
> >>
> >> On 7/16/18 6:51 AM, Vasily Sulatskov wrote:
> >>> Hi John,
> >>>
> >>> Thanks a lot for you explanation. It does make much more sense now.
> >>>
> >>> The Jira issue I think is pretty well explained (with a reference to
> >>> this thread). And I've lest my 2 cents in the pull request.
> >>>
> >>> You are right I didn't notice that repartition topic contains the same
> >>> message effectively twice, and 0/1 bytes are non-visible, so when I
> >>> used kafka-console-consumer I didn't notice that. So I have a quick
> >>> suggestion here, wouldn't it make sense to change 0 and 1 bytes to
> >>> something that has visible corresponding ascii characters, say + and
> >>> -, as these messages are effectively commands to reducer to execute
> >>> either an addition or subtraction?
> >>>
> >>> On a more serious, side, can you please explain temporal aspects of
> >>> how change messages are handled? More specifically, is it guaranteed
> >>> that both Change(newValue, null) and Change(null, oldValue) are
> >>> handled before a new aggregated value is comitted to an output topic?
> >>> Change(newValue, null) and Change(null, oldValue) are delivered as two
> >>> separate messages via a kafka topic, and when they are read from a
> >>> topic (possibly on a different machine where a commit interval is
> >>> asynchronous to a machine that's put these changes into a topic) can
> >>> it happen so a Change(newValue, null) is processed by a
> >>> KTableReduceProcessor, the value of the aggregator is updated, and
> >>> committed to the changelog topic, and a Change(null, oldValue) is
> >>> processed only in the next commit interval? If I am understand this
> >>> correctly that would mean that in an aggregated table an incorrect
> >>> aggregated value will be observed briefly, before being eventually
> >>> corrected.
> >>>
> >>> Can that happen? Or I can't see something that would make it impossible?
> >>> On Fri, Jul 13, 2018 at 8:05 PM John Roesler <j...@confluent.io> wrote:
> >>>>
> >>>> Hi Vasily,
> >>>>
> >>>> I'm glad you're making me look at this; it's good homework for me!
> >>>>
> >>>> This is very non-obvious, but here's what happens:
> >>>>
> >>>> KStreamsReduce is a Processor of (K, V) => (K, Change<V>) . I.e., it 
> >>>> emits
> >>>> new/old Change pairs as the value.
> >>>>
> >>>> Next is the Select (aka GroupBy). In the DSL code, this is the
> >>>> KTableRepartitionMap (we call it a repartition when you select a new key,
> >>>> since the new keys may belong to different partitions).
> >>>> KTableRepartitionMap is a processor that does two things:
> >>>> 1. it maps K => K1 (new keys) and V => V1 (new values)
> >>>> 2. it "explodes" Change(new, old) into [ Change(null, old), Change(new,
> >>>> null)]
> >>>> In other words, it turns each Change event into two events: a retraction
> >>>> and an update
> >>>>
> >>>> Next comes the reduce operation. In building the processor node for this
> >>>> operation, we create the sink, repartition topic, and source, followed by
> >>>> the actual Reduce node. So if you want to look at how the changes get
> >>>> serialized and desesrialized, it's in KGroupedTableImpl#buildAggregate.
> >>>> You'll see that sink and source a ChangedSerializer and 
> >>>> ChangedDeserializer.
> >>>>
> >>>> By looking into those implementations, I found that they depend on each
> >>>> Change containing just one of new OR old. They serialize the underlying
> >>>> value using the serde you provide, along with a single byte that 
> >>>> signifies
> >>>> if the serialized value is the new or old value, which the deserializer
> >>>> uses on the receiving end to turn it back into a Change(new, null) or
> >>>> Change(null, old) as appropriate. This is why the repartition topic looks
> >>>> like it's just the raw data. It basically is, except for the magic byte.
> >>>>
> >>>> Does that make sense?
> >>>>
> >>>> Also, I've created https://issues.apache.org/jira/browse/KAFKA-7161 and
> >>>> https://github.com/apache/kafka/pull/5366 . Do you mind taking a look and
> >>>> leaving any feedback you have?
> >>>>
> >>>> Thanks,
> >>>> -John
> >>>>
> >>>> On Fri, Jul 13, 2018 at 12:00 PM Vasily Sulatskov <vas...@sulatskov.net>
> >>>> wrote:
> >>>>
> >>>>> Hi John,
> >>>>>
> >>>>> Thanks for your explanation.
> >>>>>
> >>>>> I have an answer to the practical question, i.e. a null aggregator
> >>>>> value should be interpreted as a fatal application error.
> >>>>>
> >>>>> On the other hand, looking at the app topology, I see that a message
> >>>>> from KSTREAM-REDUCE-0000000002 / "table" goes goes to
> >>>>> KTABLE-SELECT-0000000006 which in turn forwards data to
> >>>>> KSTREAM-SINK-0000000007 (topic: aggregated-table-repartition), and at
> >>>>> this point I assume that data goes back to kafka into a *-repartition
> >>>>> topic, after that the message is read from kafka by
> >>>>> KSTREAM-SOURCE-0000000008 (topics: [aggregated-table-repartition]),
> >>>>> and finally gets to Processor: KTABLE-REDUCE-0000000009 (stores:
> >>>>> [aggregated-table]), where the actual aggregation takes place. What I
> >>>>> don't get is where this Change value comes from, I mean if it's been
> >>>>> produced by KSTREAM-REDUCE-0000000002, but it shouldn't matter as the
> >>>>> message goes through kafka where it gets serialized, and looking at
> >>>>> kafka "repartition" topic, it contains regular values, not a pair of
> >>>>> old/new.
> >>>>>
> >>>>> As far as I understand, Change is a purely in-memory representation of
> >>>>> the state for a particular key, and at no point it's serialized back
> >>>>> to kafka, yet somehow this Change values makes it to reducer. I feel
> >>>>> like I am missing something here. Could you please clarify this?
> >>>>>
> >>>>> Can you please point me to a place in kafka-streams sources where a
> >>>>> Change of newValue/oldValue is produced, so I could take a look? I
> >>>>> found KTableReduce implementation, but can't find who makes these
> >>>>> Change value.
> >>>>> On Fri, Jul 13, 2018 at 6:17 PM John Roesler <j...@confluent.io> wrote:
> >>>>>>
> >>>>>> Hi again Vasily,
> >>>>>>
> >>>>>> Ok, it looks to me like this behavior is the result of the un-clean
> >>>>>> topology change.
> >>>>>>
> >>>>>> Just in case you're interested, here's what I think happened.
> >>>>>>
> >>>>>> 1. Your reduce node in subtopology1 (KSTREAM-REDUCE-0000000002 / 
> >>>>>> "table"
> >>>>> )
> >>>>>> internally emits pairs of "oldValue"/"newValue" . (side-note: It's by
> >>>>>> forwarding both the old and new value that we are able to maintain
> >>>>>> aggregates using the subtractor/adder pairs)
> >>>>>>
> >>>>>> 2. In the full topology, these old/new pairs go through some
> >>>>>> transformations, but still in some form eventually make their way down 
> >>>>>> to
> >>>>>> the reduce node (KTABLE-REDUCE-0000000009/"aggregated-table").
> >>>>>>
> >>>>>> 3. The reduce processor logic looks like this:
> >>>>>> final V oldAgg = store.get(key);
> >>>>>> V newAgg = oldAgg;
> >>>>>>
> >>>>>> // first try to add the new value
> >>>>>> if (value.newValue != null) {
> >>>>>>     if (newAgg == null) {
> >>>>>>         newAgg = value.newValue;
> >>>>>>     } else {
> >>>>>>         newAgg = addReducer.apply(newAgg, value.newValue);
> >>>>>>     }
> >>>>>> }
> >>>>>>
> >>>>>> // then try to remove the old value
> >>>>>> if (value.oldValue != null) {
> >>>>>>     // Here's where the assumption breaks down...
> >>>>>>     newAgg = removeReducer.apply(newAgg, value.oldValue);
> >>>>>> }
> >>>>>>
> >>>>>> 4. Here's what I think happened. This processor saw an event like
> >>>>>> {new=null, old=(key2, 732, 10:50:40)}. This would skip the first block,
> >>>>> and
> >>>>>> (since "oldValue != null") would go into the second block. I think that
> >>>>> in
> >>>>>> the normal case we can rely on the invariant that any value we get as 
> >>>>>> an
> >>>>>> "oldValue" is one that we've previously seen ( as "newValue" ).
> >>>>>> Consequently, we should be able to assume that if we get a non-null
> >>>>>> "oldValue", "newAgg" will also not be null (because we would have 
> >>>>>> written
> >>>>>> it to the store back when we saw it as "newValue" and then retrieved it
> >>>>>> just now as "newAgg = oldAgg").
> >>>>>>
> >>>>>> However, if subtopology2, along with KTABLE-SELECT-0000000006
> >>>>>> and KSTREAM-SINK-0000000013 get added after (KSTREAM-REDUCE-0000000002 
> >>>>>> /
> >>>>>> "table") has already emitted some values, then we might in fact receive
> >>>>> an
> >>>>>> event with some "oldValue" that we have in fact never seen before
> >>>>> (because (
> >>>>>> KTABLE-REDUCE-0000000009/"aggregated-table") wasn't in the topology 
> >>>>>> when
> >>>>> it
> >>>>>> was first emitted as a "newValue").
> >>>>>>
> >>>>>> This would violate our assumption, and we would unintentionally send a
> >>>>>> "null" as the "newAgg" parameter to the "removeReducer" (aka 
> >>>>>> subtractor).
> >>>>>> If you want to double-check my reasoning, you should be able to do so 
> >>>>>> in
> >>>>>> the debugger with a breakpoint in KTableReduce.
> >>>>>>
> >>>>>>
> >>>>>> tl;dr: Supposing you reset the app when the topology changes, I think
> >>>>> that
> >>>>>> you should be able to rely on non-null aggregates being passed in to
> >>>>> *both*
> >>>>>> the adder and subtractor in a reduce.
> >>>>>>
> >>>>>> I would be in favor, as you suggested, of adding an explicit check and
> >>>>>> throwing an exception if the aggregate is ever null at those points. 
> >>>>>> This
> >>>>>> would actually help us detect if the topology has changed unexpectedly
> >>>>> and
> >>>>>> shut down, hopefully before any damage is done. I'll send a PR and see
> >>>>> what
> >>>>>> everyone thinks.
> >>>>>>
> >>>>>> Does this all seem like it adds up to you?
> >>>>>> -John
> >>>>>>
> >>>>>>
> >>>>>> On Fri, Jul 13, 2018 at 4:06 AM Vasily Sulatskov <vas...@sulatskov.net>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Hi John,
> >>>>>>>
> >>>>>>> Thanks for your reply. I am not sure if this behavior I've observed is
> >>>>>>> a bug or not, as I've not been resetting my application properly. On
> >>>>>>> the other hand if the subtractor or adder in the reduce operation are
> >>>>>>> never supposed to be called with null aggregator value, perhaps it
> >>>>>>> would make sense to put a null check in the table reduce
> >>>>>>> implementation to detect an application entering an invalid state. A
> >>>>>>> bit like a check for topics having the same number of partitions when
> >>>>>>> doing a join.
> >>>>>>>
> >>>>>>> Here's some information about my tests. Hope that can be useful:
> >>>>>>>
> >>>>>>> Topology at start:
> >>>>>>>
> >>>>>>> 2018-07-13 10:29:48 [main] INFO  TableAggregationTest - Topologies:
> >>>>>>>    Sub-topology: 0
> >>>>>>>     Source: KSTREAM-SOURCE-0000000000 (topics: [slope])
> >>>>>>>       --> KSTREAM-MAP-0000000001
> >>>>>>>     Processor: KSTREAM-MAP-0000000001 (stores: [])
> >>>>>>>       --> KSTREAM-FILTER-0000000004
> >>>>>>>       <-- KSTREAM-SOURCE-0000000000
> >>>>>>>     Processor: KSTREAM-FILTER-0000000004 (stores: [])
> >>>>>>>       --> KSTREAM-SINK-0000000003
> >>>>>>>       <-- KSTREAM-MAP-0000000001
> >>>>>>>     Sink: KSTREAM-SINK-0000000003 (topic: table-repartition)
> >>>>>>>       <-- KSTREAM-FILTER-0000000004
> >>>>>>>
> >>>>>>>   Sub-topology: 1
> >>>>>>>     Source: KSTREAM-SOURCE-0000000005 (topics: [table-repartition])
> >>>>>>>       --> KSTREAM-REDUCE-0000000002
> >>>>>>>     Processor: KSTREAM-REDUCE-0000000002 (stores: [table])
> >>>>>>>       --> KTABLE-TOSTREAM-0000000006
> >>>>>>>       <-- KSTREAM-SOURCE-0000000005
> >>>>>>>     Processor: KTABLE-TOSTREAM-0000000006 (stores: [])
> >>>>>>>       --> KSTREAM-SINK-0000000007
> >>>>>>>       <-- KSTREAM-REDUCE-0000000002
> >>>>>>>     Sink: KSTREAM-SINK-0000000007 (topic: slope-table)
> >>>>>>>       <-- KTABLE-TOSTREAM-0000000006
> >>>>>>>
> >>>>>>> This topology just takes data from the source topic "slope" which
> >>>>>>> produces messages like this:
> >>>>>>>
> >>>>>>> key1
> >>>>>>> {"value":187,"timestamp":"2018-07-13T10:28:14.188+02:00[Europe/Paris]"}
> >>>>>>> key3
> >>>>>>> {"value":187,"timestamp":"2018-07-13T10:28:14.188+02:00[Europe/Paris]"}
> >>>>>>> key2
> >>>>>>> {"value":187,"timestamp":"2018-07-13T10:28:14.188+02:00[Europe/Paris]"}
> >>>>>>> key3
> >>>>>>> {"value":188,"timestamp":"2018-07-13T10:28:15.187+02:00[Europe/Paris]"}
> >>>>>>> key1
> >>>>>>> {"value":188,"timestamp":"2018-07-13T10:28:15.187+02:00[Europe/Paris]"}
> >>>>>>> key2
> >>>>>>> {"value":188,"timestamp":"2018-07-13T10:28:15.187+02:00[Europe/Paris]"}
> >>>>>>>
> >>>>>>> Every second, there are 3 new messages arrive from "slope" topic for
> >>>>>>> keys key1, key2 and key3, with constantly increasing value.
> >>>>>>> Data is transformed so that the original key is also tracked in the
> >>>>>>> message value, grouped by key, and windowed with a custom window, and
> >>>>>>> reduced with a dummy reduce operation to make a KTable.
> >>>>>>> KTable is converted back to a stream, and sent to a topic (just for
> >>>>>>> debugging purposes).
> >>>>>>>
> >>>>>>> Here's the source (it's kafka-streams-scala for the most part). Also
> >>>>>>> please ignore silly classes, it's obviously a test:
> >>>>>>>
> >>>>>>>     val slopeTable = builder
> >>>>>>>       .stream[String, TimedValue]("slope")
> >>>>>>>       .map(
> >>>>>>>         (key, value) =>
> >>>>>>>           (
> >>>>>>>             StringWrapper(key),
> >>>>>>>             TimedValueWithKey(value = value.value, timestamp =
> >>>>>>> value.timestamp, key = key)
> >>>>>>>         )
> >>>>>>>       )
> >>>>>>>       .groupByKey
> >>>>>>>       .windowedBy(new RoundedWindows(ChronoUnit.MINUTES, 1))
> >>>>>>>       .reduceMat((aggValue, newValue) => newValue, "table")
> >>>>>>>
> >>>>>>>     slopeTable.toStream
> >>>>>>>       .to("slope-table")
> >>>>>>>
> >>>>>>> Topology after change without a proper reset:
> >>>>>>>
> >>>>>>> 2018-07-13 10:38:32 [main] INFO  TableAggregationTest - Topologies:
> >>>>>>>    Sub-topology: 0
> >>>>>>>     Source: KSTREAM-SOURCE-0000000000 (topics: [slope])
> >>>>>>>       --> KSTREAM-MAP-0000000001
> >>>>>>>     Processor: KSTREAM-MAP-0000000001 (stores: [])
> >>>>>>>       --> KSTREAM-FILTER-0000000004
> >>>>>>>       <-- KSTREAM-SOURCE-0000000000
> >>>>>>>     Processor: KSTREAM-FILTER-0000000004 (stores: [])
> >>>>>>>       --> KSTREAM-SINK-0000000003
> >>>>>>>       <-- KSTREAM-MAP-0000000001
> >>>>>>>     Sink: KSTREAM-SINK-0000000003 (topic: table-repartition)
> >>>>>>>       <-- KSTREAM-FILTER-0000000004
> >>>>>>>
> >>>>>>>   Sub-topology: 1
> >>>>>>>     Source: KSTREAM-SOURCE-0000000005 (topics: [table-repartition])
> >>>>>>>       --> KSTREAM-REDUCE-0000000002
> >>>>>>>     Processor: KSTREAM-REDUCE-0000000002 (stores: [table])
> >>>>>>>       --> KTABLE-SELECT-0000000006, KTABLE-TOSTREAM-0000000012
> >>>>>>>       <-- KSTREAM-SOURCE-0000000005
> >>>>>>>     Processor: KTABLE-SELECT-0000000006 (stores: [])
> >>>>>>>       --> KSTREAM-SINK-0000000007
> >>>>>>>       <-- KSTREAM-REDUCE-0000000002
> >>>>>>>     Processor: KTABLE-TOSTREAM-0000000012 (stores: [])
> >>>>>>>       --> KSTREAM-SINK-0000000013
> >>>>>>>       <-- KSTREAM-REDUCE-0000000002
> >>>>>>>     Sink: KSTREAM-SINK-0000000007 (topic: 
> >>>>>>> aggregated-table-repartition)
> >>>>>>>       <-- KTABLE-SELECT-0000000006
> >>>>>>>     Sink: KSTREAM-SINK-0000000013 (topic: slope-table)
> >>>>>>>       <-- KTABLE-TOSTREAM-0000000012
> >>>>>>>
> >>>>>>>   Sub-topology: 2
> >>>>>>>     Source: KSTREAM-SOURCE-0000000008 (topics:
> >>>>>>> [aggregated-table-repartition])
> >>>>>>>       --> KTABLE-REDUCE-0000000009
> >>>>>>>     Processor: KTABLE-REDUCE-0000000009 (stores: [aggregated-table])
> >>>>>>>       --> KTABLE-TOSTREAM-0000000010
> >>>>>>>       <-- KSTREAM-SOURCE-0000000008
> >>>>>>>     Processor: KTABLE-TOSTREAM-0000000010 (stores: [])
> >>>>>>>       --> KSTREAM-SINK-0000000011
> >>>>>>>       <-- KTABLE-REDUCE-0000000009
> >>>>>>>     Sink: KSTREAM-SINK-0000000011 (topic: slope-aggregated-table)
> >>>>>>>       <-- KTABLE-TOSTREAM-0000000010
> >>>>>>>
> >>>>>>> Here's the source of the sub-topology that does table aggregation:
> >>>>>>>
> >>>>>>>     slopeTable
> >>>>>>>       .groupBy(
> >>>>>>>         (key, value) => (new Windowed(StringWrapper("dummykey"),
> >>>>>>> key.window()), value)
> >>>>>>>       )
> >>>>>>>       .reduceMat(adder = (aggValue, newValue) => {
> >>>>>>>         log.info(s"Called ADD: newValue=$newValue aggValue=$aggValue")
> >>>>>>>         val agg = Option(aggValue)
> >>>>>>>         TimedValueWithKey(
> >>>>>>>           value = agg.map(_.value).getOrElse(0) + newValue.value,
> >>>>>>>           timestamp =
> >>>>>>>
> >>>>>>> Utils.latestTimestamp(agg.map(_.timestamp).getOrElse(zeroTimestamp),
> >>>>>>> newValue.timestamp),
> >>>>>>>           key = "reduced"
> >>>>>>>         )
> >>>>>>>       }, subtractor = (aggValue, newValue) => {
> >>>>>>>         log.info(s"Called SUB: newValue=$newValue aggValue=$aggValue")
> >>>>>>>         val agg = Option(aggValue)
> >>>>>>>         TimedValueWithKey(
> >>>>>>>           value = agg.map(_.value).getOrElse(0) - newValue.value,
> >>>>>>>           timestamp =
> >>>>>>>
> >>>>>>> Utils.latestTimestamp(agg.map(_.timestamp).getOrElse(zeroTimestamp),
> >>>>>>> newValue.timestamp),
> >>>>>>>           key = "reduced"
> >>>>>>>         )
> >>>>>>>       }, "aggregated-table")
> >>>>>>>       .toStream
> >>>>>>>       .to("slope-aggregated-table")
> >>>>>>>
> >>>>>>> I log all calls to adder and subtractor, so I am able to see what's
> >>>>>>> going on there, as well as I track the original keys of the aggregated
> >>>>>>> values and their timestamps, so it's relatively easy to see how the
> >>>>>>> data goes through this topology
> >>>>>>>
> >>>>>>> In order to reproduce this behavior I need to:
> >>>>>>> 1. Start a full topology (with table aggregation)
> >>>>>>> 2. Start without table aggregation (no app reset)
> >>>>>>> 3. Start with table aggregation (no app reset)
> >>>>>>>
> >>>>>>> Bellow is an interpretation of the adder/subtractor logs for a given
> >>>>>>> key/window in the chronological order
> >>>>>>>
> >>>>>>> SUB: newValue=(key2, 732, 10:50:40) aggValue=null
> >>>>>>> ADD: newValue=(key2, 751, 10:50:59) aggValue=(-732, 10:50:40)
> >>>>>>> SUB: newValue=(key1, 732, 10:50:40) aggValue=(19, 10:50:59)
> >>>>>>> ADD: newValue=(key1, 751, 10:50:59) aggValue=(-713, 10:50:59)
> >>>>>>> SUB: newValue=(key3, 732, 10:50:40) aggValue=(38, 10:50:59)
> >>>>>>> ADD: newValue=(key3, 751, 10:50:59) aggValue=(-694, 10:50:59)
> >>>>>>>
> >>>>>>> And in the end the last value that's materialized for that window
> >>>>>>> (i.e. windowed key) in the kafka topic is 57, i.e. a increase in value
> >>>>>>> for a single key between some point in the middle of the window and at
> >>>>>>> the end of the window, times 3. As opposed to the expected value of
> >>>>>>> 751 * 3 = 2253 (sum of last values in a time window for all keys being
> >>>>>>> aggregated).
> >>>>>>>
> >>>>>>> It's clear to me that I should do an application reset, but I also
> >>>>>>> would like to understand, should I expect adder/subtractor being
> >>>>>>> called with null aggValue, or is it a clear sign that something went
> >>>>>>> horribly wrong?
> >>>>>>>
> >>>>>>> On Fri, Jul 13, 2018 at 12:19 AM John Roesler <j...@confluent.io>
> >>>>> wrote:
> >>>>>>>>
> >>>>>>>> Hi Vasily,
> >>>>>>>>
> >>>>>>>> Thanks for the email.
> >>>>>>>>
> >>>>>>>> To answer your question: you should reset the application basically
> >>>>> any
> >>>>>>>> time you change the topology. Some transitions are safe, but others
> >>>>> will
> >>>>>>>> result in data loss or corruption. Rather than try to reason about
> >>>>> which
> >>>>>>> is
> >>>>>>>> which, it's much safer just to either reset the app or not change it
> >>>>> (if
> >>>>>>> it
> >>>>>>>> has important state).
> >>>>>>>>
> >>>>>>>> Beyond changes that you make to the topology, we spend a lot of
> >>>>> effort to
> >>>>>>>> try and make sure that different versions of Streams will produce the
> >>>>>>> same
> >>>>>>>> topology, so unless the release notes say otherwise, you should be
> >>>>> able
> >>>>>>> to
> >>>>>>>> upgrade without a reset.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> I can't say right now whether those wacky behaviors are bugs or the
> >>>>>>> result
> >>>>>>>> of changing the topology without a reset. Or if they are correct but
> >>>>>>>> surprising behavior somehow. I'll look into it tomorrow. Do feel
> >>>>> free to
> >>>>>>>> open a Jira ticket if you think you have found a bug, especially if
> >>>>> you
> >>>>>>> can
> >>>>>>>> describe a repro. Knowing your topology before and after the change
> >>>>> would
> >>>>>>>> also be immensely helpful. You can print it with Topology.describe().
> >>>>>>>>
> >>>>>>>> Regardless, I'll make a note to take a look at the code tomorrow and
> >>>>> try
> >>>>>>> to
> >>>>>>>> decide if you should expect these behaviors with "clean" topology
> >>>>>>> changes.
> >>>>>>>>
> >>>>>>>> Thanks,
> >>>>>>>> -John
> >>>>>>>>
> >>>>>>>> On Thu, Jul 12, 2018 at 11:51 AM Vasily Sulatskov <
> >>>>> vas...@sulatskov.net>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi,
> >>>>>>>>>
> >>>>>>>>> I am doing some experiments with kafka-streams KGroupedTable
> >>>>>>>>> aggregation, and admittedly I am not wiping data properly on each
> >>>>>>>>> restart, partially because I also wonder what would happen if you
> >>>>>>>>> change a streams topology without doing a proper reset.
> >>>>>>>>>
> >>>>>>>>> I've noticed that from time to time, kafka-streams
> >>>>>>>>> KGroupedTable.reduce() can call subtractor function with null
> >>>>>>>>> aggregator value, and if you try to work around that, by
> >>>>> interpreting
> >>>>>>>>> null aggregator value as zero for numeric value you get incorrect
> >>>>>>>>> aggregation result.
> >>>>>>>>>
> >>>>>>>>> I do understand that the proper way of handling this is to do a
> >>>>> reset
> >>>>>>>>> on topology changes, but I'd like to understand if there's any
> >>>>>>>>> legitmate case when kafka-streams can call an adder or a
> >>>>> substractor
> >>>>>>>>> with null aggregator value, and should I plan for this, or should I
> >>>>>>>>> interpret this as an invalid state, and terminate the application,
> >>>>> and
> >>>>>>>>> do a proper reset?
> >>>>>>>>>
> >>>>>>>>> Also, I can't seem to find a guide which explains when application
> >>>>>>>>> reset is necessary. Intuitively it seems that it should be done
> >>>>> every
> >>>>>>>>> time a topology changes. Any other cases?
> >>>>>>>>>
> >>>>>>>>> I tried to debug where the null value comes from and it seems that
> >>>>>>>>> KTableReduce.process() is getting called with Change<V> value with
> >>>>>>>>> newValue == null, and some non-null oldValue. Which leads to and to
> >>>>>>>>> subtractor being called with null aggregator value. I wonder how
> >>>>> it is
> >>>>>>>>> possible to have an old value for a key without a new value (does
> >>>>> it
> >>>>>>>>> happen because of the auto commit interval?).
> >>>>>>>>>
> >>>>>>>>> I've also noticed that it's possible for an input value from a
> >>>>> topic
> >>>>>>>>> to bypass aggregation function entirely and be directly
> >>>>> transmitted to
> >>>>>>>>> the output in certain cases: oldAgg is null, newValue is not null
> >>>>> and
> >>>>>>>>> oldValue is null - in that case newValue will be transmitted
> >>>>> directly
> >>>>>>>>> to the output. I suppose it's the correct behaviour, but feels a
> >>>>> bit
> >>>>>>>>> weird nonetheless. And I've actually been able to observe this
> >>>>>>>>> behaviour in practice. I suppose it's also caused by this happening
> >>>>>>>>> right before a commit happens, and the message is sent to a
> >>>>> changelog
> >>>>>>>>> topic.
> >>>>>>>>>
> >>>>>>>>> Please can someone with more knowledge shed some light on these
> >>>>> issues?
> >>>>>>>>>
> >>>>>>>>> --
> >>>>>>>>> Best regards,
> >>>>>>>>> Vasily Sulatskov
> >>>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> --
> >>>>>>> Best regards,
> >>>>>>> Vasily Sulatskov
> >>>>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>> --
> >>>>> Best regards,
> >>>>> Vasily Sulatskov
> >>>>>
> >>>
> >>>
> >>>
> >>
> >
> >
>


-- 
Best regards,
Vasily Sulatskov

Reply via email to