Skip to content

MINOR: clarify why suppress can sometimes drop tombstones#6195

Merged
guozhangwang merged 2 commits intoapache:trunkfrom
vvcephei:minor-clarify-suppression-dropping-tombstones
Jan 25, 2019
Merged

MINOR: clarify why suppress can sometimes drop tombstones#6195
guozhangwang merged 2 commits intoapache:trunkfrom
vvcephei:minor-clarify-suppression-dropping-tombstones

Conversation

@vvcephei
Copy link
Copy Markdown
Contributor

Clarify when and why suppress would drop tombstone records. No behavioral change.

Committer Checklist (excluded from commit message)

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

@vvcephei
Copy link
Copy Markdown
Contributor Author

A contributor asked me what shouldSuppressTombstones is for, and I couldn't remember. After figuring it out, it was clear that this logic needs clarification.

Please take a look when you get the chance, @guozhangwang @mjsax @bbejeck @ableegoldman

Copy link
Copy Markdown
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

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

Retest this please.

@mjsax
Copy link
Copy Markdown
Member

mjsax commented Jan 24, 2019

Retest this please.

@@ -66,7 +79,7 @@ Duration timeToWaitForMoreEvents() {
}

boolean shouldSuppressTombstones() {
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Consider renaming to safeToDropTombstones:

Suggested change
boolean shouldSuppressTombstones() {
boolean safeToDropTombstones() {

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Ah, missed one. Thanks!

@vvcephei
Copy link
Copy Markdown
Contributor Author

retest this, please

Copy link
Copy Markdown
Contributor

@guozhangwang guozhangwang left a comment

Choose a reason for hiding this comment

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

LGTM. Left a meta comment which is actually out of the scope of this PR.

* idempotent and correct). We decided that the unnecessary tombstones would not be
* desirable in the output stream, though, hence the ability to drop them.
*
* A alternative is to remember whether a result has previously been emitted
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.

Sounds good.

Actually I've seen the same situation for our current caching layer flushing logic as well: e.g. put(A, v) -> delete(A) and both only hit the cache layer. When flushing we tried to read the old value and found its null bytes, so we know nothing was flushed for A and nothing written to downstream before so we can skip putting a tombstone to underlying store as well as downstream.

For suppression buffer though, it is harder since you do not have an underlying store to fetch the old value, and of course reading the whole changelog to see if there's any updates on this key A costs you everything. But suppose we always have a persistent buffer, this may be an easier task.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Thanks @guozhangwang ,

Yes, I think that's why the record cache keeps a set of "dirty keys", just like the suppression buffer. If you have a dirty-key, but don't find any value for it, then you know it was a delete and you can send a tombstone.

It is indeed harder for the suppression buffer, since we try not to completely duplicate all the data. I guess it's not all the data, but it is all the keys at least. For example, let's say we store the data as the heterogeneous type [nonTombstoneValueHasBeenEmittedFlag] | [nonTombstoneValueHasBeenEmittedFlag, valueToBeEmitted]. Then, when we get a tombstone, if the pre-existing value has nonTombstoneValueHasBeenEmittedFlag == true, we know we must emit the tombstone. If there is no value, or if nonTombstoneValueHasBeenEmittedFlag == false, we know that we don't need to send the tombstone. Upon sending the tombstone, we would simply delete the record from the store. Upon emitting a non-tombstone value, we would drop the value from the store and only store [nonTombstoneValueHasBeenEmittedFlag := true].

Not sure I would do this in memory, but as you point out, it could be an option for the persistent version. I guess if you think the whole "live" key space would fit in memory (plus one bit per key for the flag), then in-memory would work too.

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.

Right. But note that the current dirty-key in cache is not enough determining if we have, ever, write for a key to the underlying store which is not deleted yet: dirty-key only contains the dirty-keys since last flush, i.e. the key not in the dirty-key is only a necessary, not sufficient condition. And that's why we can only consider not writing the tombstone if the read on this key returns null-bytes, indicating nothing was there.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Oh, right, I guess we would need to do something like what I described above, or some equivalent solution...

Copy link
Copy Markdown
Member

@bbejeck bbejeck left a comment

Choose a reason for hiding this comment

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

Thanks for the clarification, LGTM

@vvcephei
Copy link
Copy Markdown
Contributor Author

Some core integration test failed...

Retest this, please.

@guozhangwang
Copy link
Copy Markdown
Contributor

I've ran the unit tests locally and it passed.

@guozhangwang guozhangwang merged commit a65940c into apache:trunk Jan 25, 2019
@vvcephei vvcephei deleted the minor-clarify-suppression-dropping-tombstones branch January 25, 2019 22:18
@vvcephei
Copy link
Copy Markdown
Contributor Author

Thanks, @guozhangwang !

jarekr pushed a commit to confluentinc/kafka that referenced this pull request Apr 18, 2019
* ak/trunk:
  MINOR: Update usage of deprecated API (apache#6146)
  KAFKA-4217: Add KStream.flatTransform (apache#5273)
  MINOR: Update Gradle to 5.1.1 (apache#6160)
  KAFKA-3522: Generalize Segments (apache#6170)
  Added quotes around the class path (apache#4469)
  KAFKA-7837: Ensure offline partitions are picked up as soon as possible when shrinking ISR (apache#6202)
  MINOR: In the MetadataResponse schema, ignorable should be a boolean
  KAFKA-7838: Log leader and follower end offsets when shrinking ISR (apache#6168)
  KAFKA-5692: Change PreferredReplicaLeaderElectionCommand to use Admin… (apache#3848)
  MINOR: clarify why suppress can sometimes drop tombstones (apache#6195)
  MINOR: Upgrade ducktape to 0.7.5 (apache#6197)
  MINOR: Improve IntegrationTestUtils documentation (apache#5664)
  MINOR: upgrade to jdk8 8u202
  KAFKA-7693; Fix SequenceNumber overflow in producer (apache#5989)
  KAFKA-7692; Fix ProducerStateManager SequenceNumber overflow (apache#5990)
  MINOR: update copyright year in the NOTICE file. (apache#6196)
  KAFKA-7793: Improve the Trogdor command line. (apache#6133)
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
Reviewers: Jonathan Gordon, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants