Skip to content

KAFKA-7652: Part II; Add single-point query for SessionStore and use for flushing / getter#6161

Merged
guozhangwang merged 14 commits intoapache:trunkfrom
guozhangwang:K7652-orderbytes-upper-range
Jan 31, 2019
Merged

KAFKA-7652: Part II; Add single-point query for SessionStore and use for flushing / getter#6161
guozhangwang merged 14 commits intoapache:trunkfrom
guozhangwang:K7652-orderbytes-upper-range

Conversation

@guozhangwang
Copy link
Copy Markdown
Contributor

#2972 tried to fix a bug about flushing operation, but it was not complete, since findSessions(key, earliestEnd, latestStart) does not guarantee to only return a single entry since its semantics are to return any sessions whose end > earliestEnd and whose start < latestStart.

I've tried various ways to fix it completely and I ended up having to add a single-point query to the public ReadOnlySessionStore API for the exact needed semantics. It is used for flushing to read the old values (otherwise the wrong old values will be sent downstreams, hence it is a correctness issue) and also for getting the value for value-getters (it is for perf only).

Committer Checklist (excluded from commit message)

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

final AGG newValue = serdes.valueFrom(entry.newValue());
final AGG oldValue = newValue == null || sendOldValues ? fetchPrevious(rawKey, key.window()) : null;
final AGG oldValue = newValue == null || sendOldValues ?
serdes.valueFrom(bytesStore.fetch(rawKey, key.window().start(), key.window().end())) :
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.

This is the actual fix.

}
return value;
}
return store.fetch(key.key(), key.window().start(), key.window().end());
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.

This is a good-to-have for perf.

final Windowed<Bytes> a = new Windowed<>(keyA, new SessionWindow(2, 4));
final Windowed<Bytes> b = new Windowed<>(keyA, new SessionWindow(1, 2));
final Windowed<String> aDeserialized = new Windowed<>("a", new SessionWindow(2, 4));
final Windowed<String> bDeserialized = new Windowed<>("a", new SessionWindow(1, 2));
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.

This additional record is for illustrating the bug. Without the fix the flushed entries would be incorrect.


public static byte[] toBinary(final Windowed<Bytes> sessionKey) {
final byte[] bytes = sessionKey.key().get();
public static Bytes toBinary(final Bytes key,
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.

This is just some refactoring of the code to 1) merge the logic into a single function and 2) do Bytes.wrap internally so that callers do not need to call.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nice addition!

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 patch @guozhangwang. I made a pass over this, and I have a couple of minor comments, but otherwise, this looks good to me.

I do think we need a KIP for the addition to the ReadonlySessionStore, but the use case clean and straightforward so IMHO we should be able to get this in for 2.2.

}
cache.addDirtyEntryFlushListener(cacheName, entries -> {
for (final ThreadCache.DirtyEntry entry : entries) {
putAndMaybeForward(entry, context);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nice cleanup!

validateStoreOpen();
final Bytes bytesKey = SessionKeySchema.toBinary(key, startTime, endTime);
final Bytes cacheKey = cacheFunction.cacheKey(bytesKey);
if (cache == null) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: move the cache == null check two lines up and save creating the bytesKey and cacheKey operations when the cache is null. I know this is a minor point, but IMHO still worth the change.

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.

ack.

} catch (final InvalidStateStoreException ise) {
throw new InvalidStateStoreException("State store [" + storeName + "] is not available anymore" +
" and may have been migrated to another instance; " +
"please re-discover its location from the state metadata.");
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: do we want to include the original InvalidStateStoreException (ise) in the new exception we are throwing?

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.

We intentionally change the error message with the same InvalidStateStoreException type in the original design. But thinking about it again, I think it still worth exposing the original exception for debugging purposes.


@Override
public AGG fetch(final K key, final long startTime, final long endTime) {
return serdes.valueFrom(bytesStore.get(SessionKeySchema.toBinary(Bytes.wrap(serdes.rawKey(key)), startTime, endTime)));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: validate key is not null here?

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.

This is validated in higher in the call trace (MeteredXX) already.


public static byte[] toBinary(final Windowed<Bytes> sessionKey) {
final byte[] bytes = sessionKey.key().get();
public static Bytes toBinary(final Bytes key,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nice addition!

@guozhangwang
Copy link
Copy Markdown
Contributor Author

@bbejeck updated per comments.

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 @guozhangwang LGTM

@mjsax mjsax added the streams label Jan 27, 2019
Copy link
Copy Markdown
Contributor

@vvcephei vvcephei 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!

" and may have been migrated to another instance; " +
"please re-discover its location from the state metadata.");
"please re-discover its location from the state metadata. " +
"Original error message: " + ise.toString());
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.

Why not just include ise as the cause? Do we want to hide its stacktrace of something?

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.

Yes this is intentional indeed. Since this is a user-facing interface we were abstracting the internal stacktrace from users.

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.

LGTM.

@guozhangwang
Copy link
Copy Markdown
Contributor Author

KIP-420 (https://cwiki.apache.org/confluence/display/KAFKA/KIP-420%3A+Add+Single+Value+Fetch+in+Session+Stores) has been voted before the 2.2 KIP deadline. Merging to trunk now.

@guozhangwang guozhangwang merged commit 9d5893d into apache:trunk Jan 31, 2019
jarekr pushed a commit to confluentinc/kafka that referenced this pull request Apr 18, 2019
* AK/trunk:
  fix typo (apache#5150)
  MINOR: Reduce replica.fetch.backoff.ms in ReassignPartitionsClusterTest (apache#5887)
  KAFKA-7766: Fail fast PR builds (apache#6059)
  KAFKA-7798: Expose embedded clientIds (apache#6107)
  KAFKA-7641; Introduce "group.max.size" config to limit group sizes (apache#6163)
  KAFKA-7433; Introduce broker options in TopicCommand to use AdminClient (KIP-377)
  MINOR: Fix some field definitions for ListOffsetReponse (apache#6214)
  KAFKA-7873; Always seek to beginning in KafkaBasedLog (apache#6203)
  KAFKA-7719: Improve fairness in SocketServer processors (KIP-402) (apache#6022)
  MINOR: fix checkstyle suppressions for generated RPC code to work on Windows
  KAFKA-7859: Use automatic RPC generation in LeaveGroups (apache#6188)
  KAFKA-7652: Part II; Add single-point query for SessionStore and use for flushing / getter (apache#6161)
  KAFKA-3522: Add RocksDBTimestampedStore (apache#6149)
  KAFKA-3522: Replace RecordConverter with TimestampedBytesStore (apache#6204)
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
…for flushing / getter (apache#6161)

apache#2972 tried to fix a bug about flushing operation, but it was not complete, since findSessions(key, earliestEnd, latestStart) does not guarantee to only return a single entry since its semantics are to return any sessions whose end > earliestEnd and whose start < latestStart.

I've tried various ways to fix it completely and I ended up having to add a single-point query to the public ReadOnlySessionStore API for the exact needed semantics. It is used for flushing to read the old values (otherwise the wrong old values will be sent downstreams, hence it is a correctness issue) and also for getting the value for value-getters (it is for perf only).
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.

4 participants