Skip to content

KAFKA-9707: Fix InsertField.Key should apply to keys of tombstone records#8280

Merged
kkonstantine merged 6 commits intoapache:trunkfrom
gharris1727:insert-field-tombstone
Mar 26, 2020
Merged

KAFKA-9707: Fix InsertField.Key should apply to keys of tombstone records#8280
kkonstantine merged 6 commits intoapache:trunkfrom
gharris1727:insert-field-tombstone

Conversation

@gharris1727
Copy link
Copy Markdown
Contributor

  • Fix typo that hardcoded .value() instead of abstract operatingValue
  • Add test for Key transform that was previously not tested

Signed-off-by: Greg Harris gregh@confluent.io

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested

Signed-off-by: Greg Harris <gregh@confluent.io>
@gharris1727
Copy link
Copy Markdown
Contributor Author

This was a typo introduced in #6914.

cc @ncliang @C0urante for review

Copy link
Copy Markdown
Contributor

@ncliang ncliang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch, @gharris1727 ! It would be bad if we transformed the record but not its corresponding tombstone representing the delete.

@gharris1727
Copy link
Copy Markdown
Contributor Author

cc @rhauch @kkonstantine for review

Copy link
Copy Markdown
Contributor

@kkonstantine kkonstantine left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This behavior, of applying the SMT but leaving the tombstone records unaffected, seems to be intentional if you read the details of KAFKA-8523

Tbh, while I was reading this PR I also found counter-intuitive that someone would use this SMT to transform tombstones to non-tombstone records. Indeed, tombstone records are special in that they represent deletion in several downstream systems as well as Kafka often times.

In the light of the previous discussions, any reasons to consider keeping this fix @gharris1727 @ncliang ?

@gharris1727
Copy link
Copy Markdown
Contributor Author

@kkonstantine Yes, by the literal description, "leaving the tombstone record unaffected" is the current behavior, and not what this PR would implement.

However, I think that the (1) "leave the tombstone record unaffected" strategy is being used in contrast to the other strategies discussed in the previous issue, which were to (2) drop the tombstone, or (3) create a new empty map with only the inserted fields present. I think the decision to implement Strategy (1) among these three choices is correct, since (2) and (3) discard the meaning of the tombstone while (1) keeps that meaning in the output stream.

I think that decision was made without considering the behavior of InsertField.Key, which coincidentally had no unit tests. I only happened to notice this error because I copied this fix to another SMT which did have key tests, and those tests were broken by this behavior. I think the behavior that this PR implements is another option (4) "leave tombstone values unaffected, leave null keys unaffected, and otherwise transform keys".

With the current behavior, tombstones in the output stream would have different keys than non-tombstone values that had the same initial key. If the output stream were written back to a topic, it would not trigger compaction. If the system consuming the output stream assumed that the inserted field would exist in every record, tombstone events would violate that assumption. I think these behaviors are not correct semantically, and are bugs, not features.

@kkonstantine
Copy link
Copy Markdown
Contributor

Oh, that's right. I missed that this referred only to the key of a record. It's interesting that this distinction was missed in the previous PR.

I agree in that case. We should do the right thing and transform the key of a tombstone record, if that's what the configuration is aiming to do.

Given that this is not a regression, I'll wait for 2.5.0 to be released before I merge and backport as far back as it makes sense.

@kkonstantine
Copy link
Copy Markdown
Contributor

retest this please

Copy link
Copy Markdown
Contributor

@rhauch rhauch left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice work, @gharris1727. However, I think we should clean the previous fix up even more, and more fully test the changes.


private boolean isTombstoneRecord(R record) {
return record.value() == null;
return operatingValue(record) == null;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we please clean up the logic? This method no longer returns true if it's just a tombstone record; it also returns true if the key is null for InsertField$Key.

I'd suggest removing this method altogether and just changing the point where this method is called to simply be:

          if (operatingValue(record) == null) {

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Plus one for this; confusion around the term tombstone has already caused plenty of miscommunication around this issue and we should be careful to use it correctly.

@rhauch rhauch changed the title KAFKA-9707: Fix InsertField.Key not applying to tombstone events KAFKA-9707: Fix InsertField.Key not applying to keys of tombstone events Mar 25, 2020
@rhauch rhauch changed the title KAFKA-9707: Fix InsertField.Key not applying to keys of tombstone events KAFKA-9707: Fix InsertField.Key not applying to keys of tombstone records Mar 25, 2020
@rhauch rhauch changed the title KAFKA-9707: Fix InsertField.Key not applying to keys of tombstone records KAFKA-9707: Fix InsertField.Key should apply to keys of tombstone records Mar 25, 2020
…ed record.

Signed-off-by: Greg Harris <gregh@confluent.io>
final SourceRecord transformedRecord = xformKey.apply(record);

assertEquals(null, transformedRecord.key());
assertEquals(42L, ((Map<?, ?>) transformedRecord.value()).get("magic"));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wouldn't it be sufficient to replace these two asserts with the following?

   assertSame(record, transformedRecord);

This is a bit more correct, since we want to assert that in this case the transform returns the original record. WDYT?

Signed-off-by: Greg Harris <gregh@confluent.io>
Signed-off-by: Greg Harris <gregh@confluent.io>
Copy link
Copy Markdown
Contributor

@rhauch rhauch left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. Thanks for finding and fixing this regression, @gharris1727.

@kkonstantine
Copy link
Copy Markdown
Contributor

retest this please

@kkonstantine
Copy link
Copy Markdown
Contributor

@gharris1727 please fix the checkstyle issue (see ./jenkins.sh for example of the various build phases and pick what applies to this package).

I can trigger a build right after.

Signed-off-by: Greg Harris <gregh@confluent.io>
@kkonstantine
Copy link
Copy Markdown
Contributor

retest this please

@kkonstantine
Copy link
Copy Markdown
Contributor

Copy link
Copy Markdown
Contributor

@kkonstantine kkonstantine left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Latest builds failed abruptly on unrelated issues.
We had a green build and the changes are straightforward.

LGTM
Merging as is. Thanks @gharris1727 for catching this bug.

@kkonstantine kkonstantine merged commit 008a3b2 into apache:trunk Mar 26, 2020
kkonstantine pushed a commit that referenced this pull request Mar 26, 2020
…ords (#8280)

* KAFKA-9707: Fix InsertField.Key not applying to tombstone events

* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested

Signed-off-by: Greg Harris <gregh@confluent.io>

* Add null value assertion to tombstone test

* Remove mis-named function and add test for passing-through a null-keyed record.

Signed-off-by: Greg Harris <gregh@confluent.io>

* Simplify unchanged record assertion

Signed-off-by: Greg Harris <gregh@confluent.io>

* Replace assertEquals with assertSame

Signed-off-by: Greg Harris <gregh@confluent.io>

* Fix checkstyleTest indent issue

Signed-off-by: Greg Harris <gregh@confluent.io>
kkonstantine pushed a commit that referenced this pull request Mar 26, 2020
…ords (#8280)

* KAFKA-9707: Fix InsertField.Key not applying to tombstone events

* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested

Signed-off-by: Greg Harris <gregh@confluent.io>

* Add null value assertion to tombstone test

* Remove mis-named function and add test for passing-through a null-keyed record.

Signed-off-by: Greg Harris <gregh@confluent.io>

* Simplify unchanged record assertion

Signed-off-by: Greg Harris <gregh@confluent.io>

* Replace assertEquals with assertSame

Signed-off-by: Greg Harris <gregh@confluent.io>

* Fix checkstyleTest indent issue

Signed-off-by: Greg Harris <gregh@confluent.io>
kkonstantine pushed a commit that referenced this pull request Mar 26, 2020
…ords (#8280)

* KAFKA-9707: Fix InsertField.Key not applying to tombstone events

* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested

Signed-off-by: Greg Harris <gregh@confluent.io>

* Add null value assertion to tombstone test

* Remove mis-named function and add test for passing-through a null-keyed record.

Signed-off-by: Greg Harris <gregh@confluent.io>

* Simplify unchanged record assertion

Signed-off-by: Greg Harris <gregh@confluent.io>

* Replace assertEquals with assertSame

Signed-off-by: Greg Harris <gregh@confluent.io>

* Fix checkstyleTest indent issue

Signed-off-by: Greg Harris <gregh@confluent.io>
kkonstantine pushed a commit that referenced this pull request Mar 26, 2020
…ords (#8280)

* KAFKA-9707: Fix InsertField.Key not applying to tombstone events

* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested

Signed-off-by: Greg Harris <gregh@confluent.io>

* Add null value assertion to tombstone test

* Remove mis-named function and add test for passing-through a null-keyed record.

Signed-off-by: Greg Harris <gregh@confluent.io>

* Simplify unchanged record assertion

Signed-off-by: Greg Harris <gregh@confluent.io>

* Replace assertEquals with assertSame

Signed-off-by: Greg Harris <gregh@confluent.io>

* Fix checkstyleTest indent issue

Signed-off-by: Greg Harris <gregh@confluent.io>
kkonstantine pushed a commit that referenced this pull request Mar 26, 2020
…ords (#8280)

* KAFKA-9707: Fix InsertField.Key not applying to tombstone events

* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested

Signed-off-by: Greg Harris <gregh@confluent.io>

* Add null value assertion to tombstone test

* Remove mis-named function and add test for passing-through a null-keyed record.

Signed-off-by: Greg Harris <gregh@confluent.io>

* Simplify unchanged record assertion

Signed-off-by: Greg Harris <gregh@confluent.io>

* Replace assertEquals with assertSame

Signed-off-by: Greg Harris <gregh@confluent.io>

* Fix checkstyleTest indent issue

Signed-off-by: Greg Harris <gregh@confluent.io>
kkonstantine pushed a commit that referenced this pull request Mar 26, 2020
…ords (#8280)

* KAFKA-9707: Fix InsertField.Key not applying to tombstone events

* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested

Signed-off-by: Greg Harris <gregh@confluent.io>

* Add null value assertion to tombstone test

* Remove mis-named function and add test for passing-through a null-keyed record.

Signed-off-by: Greg Harris <gregh@confluent.io>

* Simplify unchanged record assertion

Signed-off-by: Greg Harris <gregh@confluent.io>

* Replace assertEquals with assertSame

Signed-off-by: Greg Harris <gregh@confluent.io>

* Fix checkstyleTest indent issue

Signed-off-by: Greg Harris <gregh@confluent.io>
kkonstantine pushed a commit that referenced this pull request Mar 26, 2020
…ords (#8280)

* KAFKA-9707: Fix InsertField.Key not applying to tombstone events

* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested

Signed-off-by: Greg Harris <gregh@confluent.io>

* Add null value assertion to tombstone test

* Remove mis-named function and add test for passing-through a null-keyed record.

Signed-off-by: Greg Harris <gregh@confluent.io>

* Simplify unchanged record assertion

Signed-off-by: Greg Harris <gregh@confluent.io>

* Replace assertEquals with assertSame

Signed-off-by: Greg Harris <gregh@confluent.io>

* Fix checkstyleTest indent issue

Signed-off-by: Greg Harris <gregh@confluent.io>
kkonstantine pushed a commit that referenced this pull request Mar 26, 2020
…ords (#8280)

* KAFKA-9707: Fix InsertField.Key not applying to tombstone events

* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested

Signed-off-by: Greg Harris <gregh@confluent.io>

* Add null value assertion to tombstone test

* Remove mis-named function and add test for passing-through a null-keyed record.

Signed-off-by: Greg Harris <gregh@confluent.io>

* Simplify unchanged record assertion

Signed-off-by: Greg Harris <gregh@confluent.io>

* Replace assertEquals with assertSame

Signed-off-by: Greg Harris <gregh@confluent.io>

* Fix checkstyleTest indent issue

Signed-off-by: Greg Harris <gregh@confluent.io>
@kkonstantine
Copy link
Copy Markdown
Contributor

The PR was merged to trunk and backported to 2.5, 2.4, 2.3, 2.2, 2.1, 2.0, 1.1, and 1.0 release branches.

Unfortunately when I tried to Squash and merge on github the initial attempt failed, and the option Try again that appeared discarded my edits in the commit message. This resulted in the reviewers not being recorded and the message being unformatted. Apologies. Will keep it in mind for the next time.

qq619618919 pushed a commit to qq619618919/kafka that referenced this pull request May 12, 2020
…ords (apache#8280)

* KAFKA-9707: Fix InsertField.Key not applying to tombstone events

* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested

Signed-off-by: Greg Harris <gregh@confluent.io>

* Add null value assertion to tombstone test

* Remove mis-named function and add test for passing-through a null-keyed record.

Signed-off-by: Greg Harris <gregh@confluent.io>

* Simplify unchanged record assertion

Signed-off-by: Greg Harris <gregh@confluent.io>

* Replace assertEquals with assertSame

Signed-off-by: Greg Harris <gregh@confluent.io>

* Fix checkstyleTest indent issue

Signed-off-by: Greg Harris <gregh@confluent.io>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants