-
Notifications
You must be signed in to change notification settings - Fork 1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
feat: avoid spurious tombstones in table output #6405
Merged
big-andy-coates
merged 5 commits into
confluentinc:master
from
big-andy-coates:ifx_master
Oct 12, 2020
Merged
Changes from 1 commit
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
a10e5eb
chore: fix master build
big-andy-coates cc7f1da
feat: avoid supurious tombstones
big-andy-coates fcee2d4
test: updated test & historical plans
big-andy-coates 8bc96c1
test: temp disable historical plans that fail
big-andy-coates 64480bf
test: disable _correct_ tests ;)
big-andy-coates File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
63 changes: 0 additions & 63 deletions
63
ksqldb-engine/src/test/java/io/confluent/ksql/function/KudafUndoAggregatorTest.java
This file was deleted.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -51,17 +51,19 @@ public KudafUndoAggregator( | |
@SuppressWarnings({"unchecked", "rawtypes"}) | ||
@Override | ||
public GenericRow apply(final Struct k, final GenericRow rowValue, final GenericRow aggRowValue) { | ||
final GenericRow result = GenericRow.fromList(aggRowValue.values()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. As above. |
||
|
||
for (int idx = 0; idx < nonAggColumnCount; idx++) { | ||
aggRowValue.set(idx, rowValue.get(idx)); | ||
result.set(idx, rowValue.get(idx)); | ||
} | ||
|
||
for (int idx = nonAggColumnCount; idx < columnCount; idx++) { | ||
final TableAggregationFunction function = aggregateFunctions.get(idx - nonAggColumnCount); | ||
final Object argument = rowValue.get(function.getArgIndexInValue()); | ||
final Object previous = aggRowValue.get(idx); | ||
aggRowValue.set(idx, function.undo(argument, previous)); | ||
final Object previous = result.get(idx); | ||
result.set(idx, function.undo(argument, previous)); | ||
} | ||
|
||
return aggRowValue; | ||
return result; | ||
} | ||
} |
107 changes: 107 additions & 0 deletions
107
...xecution/src/test/java/io/confluent/ksql/execution/function/udaf/KudafAggregatorTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,107 @@ | ||
/* | ||
* Copyright 2020 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community License (the "License"); you may not use | ||
* this file except in compliance with the License. You may obtain a copy of the | ||
* License at | ||
* | ||
* http://www.confluent.io/confluent-community-license | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
* WARRANTIES OF ANY KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations under the License. | ||
*/ | ||
|
||
package io.confluent.ksql.execution.function.udaf; | ||
|
||
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.is; | ||
import static org.hamcrest.Matchers.not; | ||
import static org.mockito.ArgumentMatchers.any; | ||
import static org.mockito.Mockito.when; | ||
|
||
import com.google.common.collect.ImmutableList; | ||
import io.confluent.ksql.GenericRow; | ||
import io.confluent.ksql.execution.transform.KsqlProcessingContext; | ||
import io.confluent.ksql.function.KsqlAggregateFunction; | ||
import java.util.function.Function; | ||
import org.apache.kafka.connect.data.Struct; | ||
import org.apache.kafka.streams.kstream.Merger; | ||
import org.junit.Before; | ||
import org.junit.Test; | ||
import org.junit.runner.RunWith; | ||
import org.mockito.Mock; | ||
import org.mockito.junit.MockitoJUnitRunner; | ||
|
||
@RunWith(MockitoJUnitRunner.class) | ||
public class KudafAggregatorTest { | ||
|
||
@Mock | ||
private KsqlAggregateFunction<Long, String, String> func1; | ||
@Mock | ||
private Struct key; | ||
@Mock | ||
private Merger<Struct, String> func1Merger; | ||
@Mock | ||
private Function<String, String> func1ResultMapper; | ||
@Mock | ||
private KsqlProcessingContext ctx; | ||
private KudafAggregator<String> aggregator; | ||
|
||
@Before | ||
public void setUp() { | ||
aggregator = new KudafAggregator<>(2, ImmutableList.of(func1)); | ||
|
||
when(func1.getMerger()).thenReturn(func1Merger); | ||
when(func1.getResultMapper()).thenReturn(func1ResultMapper); | ||
|
||
when(func1.aggregate(any(), any())).thenReturn("func1-result"); | ||
when(func1Merger.apply(any(), any(), any())).thenReturn("func1-merged"); | ||
when(func1ResultMapper.apply(any())).thenReturn("func1-result"); | ||
} | ||
|
||
@Test | ||
public void shouldNotMutateParametersOnApply() { | ||
// Given: | ||
final GenericRow value = GenericRow.genericRow(1, 2L); | ||
final GenericRow agg = GenericRow.genericRow(1, 2L, 3); | ||
|
||
// When: | ||
final GenericRow result = aggregator.apply("key", value, agg); | ||
|
||
// Then: | ||
assertThat(value, is(GenericRow.genericRow(1, 2L))); | ||
assertThat(agg, is(GenericRow.genericRow(1, 2L, 3))); | ||
assertThat("invalid test", result, is(not(GenericRow.genericRow(1, 2L, 3)))); | ||
} | ||
|
||
@Test | ||
public void shouldNotMutateParametersOnMerge() { | ||
// Given: | ||
final GenericRow aggOne = GenericRow.genericRow(1, 2L, 4); | ||
final GenericRow aggTwo = GenericRow.genericRow(1, 2L, 3); | ||
|
||
// When: | ||
final GenericRow result = aggregator.getMerger().apply(key, aggOne, aggTwo); | ||
|
||
// Then: | ||
assertThat(aggOne, is(GenericRow.genericRow(1, 2L, 4))); | ||
assertThat(aggTwo, is(GenericRow.genericRow(1, 2L, 3))); | ||
assertThat("invalid test", result, is(not(GenericRow.genericRow(1, 2L, 4)))); | ||
assertThat("invalid test", result, is(not(GenericRow.genericRow(1, 2L, 3)))); | ||
} | ||
|
||
@Test | ||
public void shouldNotMutateParametersOnResultsMap() { | ||
// Given: | ||
final GenericRow agg = GenericRow.genericRow(1, 2L, 4); | ||
|
||
// When: | ||
final GenericRow result = aggregator.getResultMapper().transform("k", agg, ctx); | ||
|
||
// Then: | ||
assertThat(agg, is(GenericRow.genericRow(1, 2L, 4))); | ||
assertThat("invalid test", result, is(not(GenericRow.genericRow(1, 2L, 4)))); | ||
} | ||
} |
79 changes: 79 additions & 0 deletions
79
...tion/src/test/java/io/confluent/ksql/execution/function/udaf/KudafUndoAggregatorTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,79 @@ | ||
/* | ||
* Copyright 2020 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community License (the "License"); you may not use | ||
* this file except in compliance with the License. You may obtain a copy of the | ||
* License at | ||
* | ||
* http://www.confluent.io/confluent-community-license | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
* WARRANTIES OF ANY KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations under the License. | ||
*/ | ||
|
||
package io.confluent.ksql.execution.function.udaf; | ||
|
||
import static io.confluent.ksql.GenericRow.genericRow; | ||
import static org.hamcrest.CoreMatchers.equalTo; | ||
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.is; | ||
import static org.hamcrest.Matchers.not; | ||
import static org.mockito.ArgumentMatchers.any; | ||
import static org.mockito.Mockito.when; | ||
|
||
import com.google.common.collect.ImmutableList; | ||
import io.confluent.ksql.GenericRow; | ||
import io.confluent.ksql.execution.function.TableAggregationFunction; | ||
import org.apache.kafka.connect.data.Struct; | ||
import org.junit.Before; | ||
import org.junit.Test; | ||
import org.junit.runner.RunWith; | ||
import org.mockito.Mock; | ||
import org.mockito.junit.MockitoJUnitRunner; | ||
|
||
@RunWith(MockitoJUnitRunner.class) | ||
public class KudafUndoAggregatorTest { | ||
|
||
@Mock | ||
private TableAggregationFunction<Long, String, String> func1; | ||
@Mock | ||
private Struct key; | ||
private KudafUndoAggregator aggregator; | ||
|
||
@Before | ||
public void setUp() { | ||
aggregator = new KudafUndoAggregator(2, ImmutableList.of(func1)); | ||
|
||
when(func1.undo(any(), any())).thenReturn("func1-undone"); | ||
} | ||
|
||
@Test | ||
public void shouldNotMutateParametersOnApply() { | ||
// Given: | ||
final GenericRow value = GenericRow.genericRow(1, 2L); | ||
final GenericRow agg = GenericRow.genericRow(1, 2L, 3); | ||
|
||
// When: | ||
final GenericRow result = aggregator.apply(key, value, agg); | ||
|
||
// Then: | ||
assertThat(value, is(GenericRow.genericRow(1, 2L))); | ||
assertThat(agg, is(GenericRow.genericRow(1, 2L, 3))); | ||
assertThat("invalid test", result, is(not(GenericRow.genericRow(1, 2L, 3)))); | ||
} | ||
|
||
@Test | ||
public void shouldApplyUndoableAggregateFunctions() { | ||
// Given: | ||
final GenericRow value = genericRow(1, 2L); | ||
final GenericRow aggRow = genericRow(1, 2L, 3); | ||
|
||
// When: | ||
final GenericRow resultRow = aggregator.apply(key, value, aggRow); | ||
|
||
// Then: | ||
assertThat(resultRow, equalTo(genericRow(1, 2L, "func1-undone"))); | ||
} | ||
} |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Kafka Streams does not expect the aggregator to mutate its parameters. The streams code is passing in the "old value", which ksqlDB was then mutating and returning as the "new value". This meant, when then function returned, the old and new values matched. This is obviously bad!
Code now takes a copy and mutates that. There is a perf hit, obviously, but it's unavoidable.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not sure I understand -- why did the old code work, in that case? Or did something change on the Streams side recently?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The old code works because we were never enabling the sending of old values. We now do, to avoid the spurious tombstones.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry, still not understanding. What was being sent before, if not the old values? Was this method even being called, previously?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The processing nodes in the streams topology can optionally include the old/previous value, as well as the new/current value, to child nodes. This is not on by default. An upstream change to how table filters is handled means this is now turned on.
The streams code for aggregation looks something like:
The two calls:
remove.apply(key, value.oldValue, oldAgg)
andadd.apply(key, value.newValue, initializedAgg)
are calling out to ksqlDB code. If these calls directly mutate theoldAgg
orinitializedAgg
parameters passed, rather than creating copies, then the old and new values forwarded to child nodes will match. i.e. intupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null)
, the parametersnewAgg
andoldAgg
will have the same updated value, rather thanoldAgg
holding the previous value. This breaks downstream processes, which expect the old and new value.Previously the nodes weren't configured to send old values, so where just sending
null
for the old value and downstream could handle this correctly.