Skip to content
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

[REVIEW NEEDED] - KAFKA-3705 Added a foreignKeyJoin implementation for KTable. #5527

Open
wants to merge 2 commits into
base: trunk
from

Conversation

Projects
None yet
8 participants
@bellemare
Copy link

commented Aug 17, 2018

https://issues.apache.org/jira/browse/KAFKA-3705

Foreign Key Join:

Allows for a KTable to map its value to a given foreign key and join on another KTable keyed on that foreign key. Applies the joiner, then returns the tuples keyed on the original key. This supports updates from both sides of the join.

Design Philosophy:

The intent of this design was to build a totally encapsulated function that operates very similarly to the regular join function. No further work is required by the user to obtain their foreignKeyJoin results after calling the function. That being said, there is increased cost in some of the topology components, especially due to resolving out-of-order arrival due to foreign key changes. I would appreciate any and all feedback on this approach, as my understanding of the Kafka Streams DSL is to provide higher level functionality without requiring the users to know exactly what's going on under the hood.

Some points of note:

  1. Requires an additional materialized State Store for the prefixScanning of the repartitioned CombinedKey events.

  2. ReadOnlyKeyValueStore interface was modified to contain prefixScan. This requires that all implementations support this, but follows an existing precedent where some store functions are already stubbed out with exceptions.

  3. Currently limited to Inner Join (can do more join logic in future - just limiting the focus of this KIP).

  4. Application Reset does not seem to delete the new internal topics that I have added. (only tested with Kafka 1.0).

  5. Only works with identical number of input partitions at the moment, though it may be possible to get it working with KTables of varying input partition count.

Testing:

Testing is covered by a two integration tests that exercises the foreign key join.
The first test exercises the out-of-order resolution and partitioning strategies by running three streams instances on three partitions. This demonstrates the scalability of the proposed solution.

important The second test (KTableKTableForeignKeyInnerJoinMultiIntegrationTest) attempts to join using foreign key twice. This results in a NullPointerException regarding a missing task, and must be resolved before committing this.

final ValueMapper<V, KO> keyExtractor,
final ValueJoiner<V, VO, VR> joiner,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized,
final Serde<K> thisKeySerde,

This comment has been minimized.

Copy link
@Kaiserchen

Kaiserchen Aug 17, 2018

Serialized wrapper?

This comment has been minimized.

Copy link
@bellemare

bellemare Aug 20, 2018

Author

Will do.


@Override
public CombinedKey<KF, KP> deserialize(final String topic, final byte[] data) {
//{4-byte foreignKeyLength}{foreignKeySerialized}{4-bytePrimaryKeyLength}{primaryKeySerialized}

This comment has been minimized.

Copy link
@Kaiserchen

Kaiserchen Aug 17, 2018

can skip the second length, its known anyway.

This comment has been minimized.

Copy link
@bellemare

bellemare Aug 20, 2018

Author

True, I could just do it as the remainder. Thanks

//{4-byte foreignKeyLength}{foreignKeySerialized}{4-bytePrimaryKeyLength}{primaryKeySerialized}

final byte[] fkCount = Arrays.copyOfRange(data, 0, 4);
final int foreignKeyLength = fourBytesToInt(fkCount);

This comment has been minimized.

Copy link
@Kaiserchen

Kaiserchen Aug 17, 2018

treating the whole thing as Buffer?

This comment has been minimized.

Copy link
@bellemare

bellemare Aug 20, 2018

Author

I don't understand your question, can you elaborate?

This comment has been minimized.

Copy link
@Kaiserchen

Kaiserchen Aug 21, 2018

I think the whole section could look nicer if you would start with ByteBuffer.allocate(totallength).asIntBuffer(keylength).asbyteBuffer.put(key).put(key)...
something



//TODO - Can reduce some of the parameters, but < 13 is not possible at the moment.
//Would likely need to split into two graphNodes - ie: foreignKeyJoinNode and foreignKeyJoinOrderResolutionNode.

This comment has been minimized.

Copy link
@Kaiserchen

Kaiserchen Aug 17, 2018

I think the step here an optimizer could potentially exploit is the repartitioning. So one could try to only factor out the repartitioning

This comment has been minimized.

Copy link
@bellemare

bellemare Aug 20, 2018

Author

I'll have to look more into the optimizer. TBH I built this originally in 1.0 and just did a functional port, not necessarily a best practices one. Thanks

This comment has been minimized.

Copy link
@Kaiserchen

Kaiserchen Aug 21, 2018

I would not recommend to spend to much energy. At the moment I really don't expect the optimizer to be able to exploit any of this. Probably also not in the future. Was just a though popping into my head


@Override
public KeyValueIterator<K, V> prefixScan(final K prefix) {
return this.inner.prefixScan(prefix);

This comment has been minimized.

Copy link
@Kaiserchen

Kaiserchen Aug 17, 2018

probably need to wrap into
DelegatingPeekingKeyValueIterator

@mjsax mjsax added the streams label Aug 17, 2018

final Materialized foreignMaterialized = Materialized.<CombinedKey<KO, K>, V, KeyValueStore<Bytes, byte[]>>as(prefixScannableDBRef.name())
//Need all values to be immediately available in the rocksDB store.
//No easy way to flush cache prior to prefixScan, so caching is disabled on this store.
.withCachingDisabled()

This comment has been minimized.

Copy link
@bellemare

bellemare Aug 20, 2018

Author

I notice that in 2.x that I may be able to rework this to allow for enabled cache using a prefixScan function similar to ThreadCache.range. I will have to look into this a bit more, though I don't think it will affect performance much since I anticipate RocksDB prefixScan to take the longest overall.

This comment has been minimized.

Copy link
@Kaiserchen

Kaiserchen Aug 21, 2018

Might be, its one of the places I got stuck once. From experience I can tell that its working sufficiently well w/o cache. I think rocks does a pretty good job in not seeking around to randomly on the disk

This comment has been minimized.

Copy link
@bellemare

bellemare Sep 3, 2018

Author

I'll leave it out for now. If someone else thinks otherwise, they can speak up or it can be done in a subsequent PR.

final byte[] offset = longSerializer.serialize(null, context().offset());
context().headers().add(KTableRepartitionerProcessorSupplier.this.offset, offset);
context().headers().add(propagate, falseByteArray);
context().forward(combinedOldKey, change.newValue);

This comment has been minimized.

Copy link
@Kaiserchen

Kaiserchen Sep 3, 2018

would need to forward null here?

This comment has been minimized.

Copy link
@bellemare

bellemare Sep 3, 2018

Author

Yes, cleaner to do so. The value is not relevant. I have fixed that and added a clarification comment (I can remove all comments if required before final submission).


import org.apache.kafka.streams.processor.ProcessorSupplier;

abstract public class BaseForeignKeyJoinProcessorSupplier<K, V> implements ProcessorSupplier<K, V> {

This comment has been minimized.

Copy link
@Kaiserchen

Kaiserchen Sep 3, 2018

would remove this class and avoid protected final here. Doesn't seem like a convincing java programming style.

This comment has been minimized.

Copy link
@bellemare

bellemare Sep 3, 2018

Author

I will change this - but I do appreciate any advice on how I should changee that. I'll post my updated code and we can determine if it's any better, or if there is a more specific approach I should follow.

@Kaiserchen

This comment has been minimized.

Copy link

commented Sep 3, 2018

We should just remove all the final keywords, I don't think they add any benefit?

@bellemare

This comment has been minimized.

Copy link
Author

commented Sep 3, 2018

I had to add all the final keywords to pass the linting check - IIRC, my first run had dozens of linting errors preventing compilation.

@bellemare bellemare force-pushed the bellemare:trunk-oneToMany branch from a36f5ee to 3d465f5 Sep 3, 2018

@bellemare bellemare force-pushed the bellemare:trunk-oneToMany branch from 8719e16 to 56b76fa Sep 25, 2018

@bellemare bellemare force-pushed the bellemare:trunk-oneToMany branch 5 times, most recently from 3e1f62d to e718610 Oct 26, 2018

@bellemare bellemare force-pushed the bellemare:trunk-oneToMany branch 3 times, most recently from 8024097 to 3925a3a Dec 3, 2018

@mjsax

This comment has been minimized.

Copy link
Member

commented Mar 7, 2019

@bellemare What is your JIRA ID? Would like to assign the ticket to you.

@bellemare

This comment has been minimized.

Copy link
Author

commented Mar 8, 2019

@mjsax JIRA ID is abellemare

@sachabest

This comment has been minimized.

Copy link

commented Apr 9, 2019

Hi, sorry to intrude on a potentially stale PR, but is this functionality still in development? Would be extraordinarily useful for joining two changelog-like entities.

@pgwhalen

This comment has been minimized.

Copy link

commented Apr 9, 2019

I sure hope so, my team is looking forward to it as well! Given that the KIP was accepted a few weeks ago, I think it's safe to say it will make it in fairly soon. I would definitely pick up development if @bellemare can't continue.

@bellemare

This comment has been minimized.

Copy link
Author

commented Apr 9, 2019

Hey folks - I'm still trying to get the code put together and finalize some of the changes that were outlined in the KIP. Stay tuned!

@bellemare

This comment has been minimized.

Copy link
Author

commented Apr 12, 2019

Hi All - I'm at a point where I need some feedback on a couple of things:

  1. The organization of the code

  2. The organization of the graph nodes in KTableImpl

  3. Insights into why I am getting NullPointerException in KTableKTableForeignKeyInnerJoinMultiIntegrationTest (though not consistently). I believe this is a misunderstanding on my part as to how partitions are co-partitioned, but there may be more to it that I am missing. Basically, it seems that depending on how the tasks and partitions are assigned, we either get a java.lang.NullPointerException: Task was unexpectedly missing for partition table1-1 or it we don't.
    This must be resolved if we wish to have flexible partition counts for joining, ie: FK join a topic with 3 partitions and a topic with 7 partitions

  4. Anything else.

Feedback is very much appreciated, as this is the first PR I've put up against Kafka and I'm sure I've violated a number of things.

@bellemare bellemare changed the title [DO NOT MERGE] - KAFKA-3705 Added a foreignKeyJoin implementation for KTable. [REVIEW NEEDED] - KAFKA-3705 Added a foreignKeyJoin implementation for KTable. Apr 12, 2019

@Override
public byte[] serialize(String topic, SubscriptionResponseWrapper<V> data) {
//{16-bytes Hash}{n-bytes serialized data}
byte[] serializedData = serializer.serialize(null, data.getForeignValue());

This comment has been minimized.

Copy link
@adaniline-traderev

adaniline-traderev Apr 16, 2019

Why is the topic passed as null? It causes issues with GenericRecord AVRO serializer, since it tries to register schemas under "null-value" subject, and the schema registry responds with "version not compatible" error

This comment has been minimized.

Copy link
@bellemare

bellemare Apr 16, 2019

Author

The issue is actually with the Confluent implementation of the SerDe, as they incorrectly attempt to register when null topics are passed in. Read confluentinc/schema-registry#1061 for more details. That being said, it has been extremely quiet in that git repo, I am not sure how much effort Confluent puts into supporting work on that product.

This comment has been minimized.

Copy link
@adaniline-traderev

adaniline-traderev Apr 16, 2019

If this does not gets fixed either way, this PR will be unusable for most of the practical use cases. What is the downside of passing the topic name to the serializer? I tried it, and it seemed to work as expected.
Is there a workaround if confluentinc/schema-registry#1061 is not fixed?

This comment has been minimized.

Copy link
@bellemare

bellemare Apr 16, 2019

Author

I think the main issue would be the large amount of internal topic schemas registered to the schema registry. This, combined with any breaking changes to the schema (due to normal business requirement changes) would make it such that you are now needing to manually delete schema registry entries made to internal topics. This is a workflow that I do not believe was ever intended to be done with the Confluent Serde.

As it stands right now, there are allegedly other functionalities that require null serialization ("There are several places in Streams where we need to serialize a value for purposes other than sending it to a topic (KTableSuppressProcessor comes to mind), and using null for the topic is the convention we have."). These too will not work with the confluent Serde.

If they do not fix it, then the next best thing to do would be wrap it in your own implementation and intercept null-topic values to avoid registration. I do not see why it wouldn't be fixed since the current behaviour of registering "null-topic" is fundamentally useless.

Anyways, with all that being said, for this particular line I can certainly pass in the topic since it's fairly well-defined. If you wish to have your internal topics registered to the schema registry, no big deal. For other parts, such as

, there is no solution using the current Confluent Serde.

This comment has been minimized.

Copy link
@adaniline-traderev

adaniline-traderev Apr 16, 2019

Confluent serde needs a schema id, and looks like it is not stored in GenericData.Record instance - it may not be trivial to fix confluentinc/schema-registry#1061...

This comment has been minimized.

Copy link
@mjsax

mjsax Apr 27, 2019

Member

I discussed this with some other people, and somebody mentioned, that for the value we serialize, this value is actually also store in RocksDB (input KTable). We also know, that the corresponding byte[] are written into the store changelog topic. Hence, instead of using the repartition topic, using the changelog topic should be a better option, as it does not leak anything into SR (or whatever other Serdes might do with the topic name).

Even if there is not changelog topic for the input KTable (we do some optimizations and sometimes don't create one (eg, the store might actual be a logical view and is not materialized). But even for this case, using the changelog topic name seems to be save.

This comment has been minimized.

Copy link
@bellemare

bellemare Apr 29, 2019

Author

context().topic() gives the repartition topic name in the serializer, which is what I want. In the processor sections, where I use null, context().topic() gives me the input-topic name for the KTable... which is also fine, since the serializer will check against the input topic schema, which must be valid by definition of the data being within the topic... so I suspect this issue can be laid to rest, in line with adaniline-traderev's suggestion.

This removes any requirement for the upstream serializer to have to do special work for null values.

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang May 1, 2019

Contributor

@mjsax I'm not sure I can fully follow the suggestion of using changelog topic v.s. the repartition topic here: are you suggesting to do it universally or just for this case? If it is the latter case, I felt it a bit awkward due to inconsistency with other source KTable cases where we will just follow the SourceNode / RecordDeserializer path to deserialize using the source topic; it if it the first case, that also has some drawbacks since with today's topology generation not all source KTables will need to be materialized to a store and hence not necessary having a changelog topic.

I still feel that using the source topic name (and i.e. in this case, the repartition topic) admittedly exposed to SR but is philosophically the right thing to do, and we should consider fixing it on serde APIs in the future. WDYT

This comment has been minimized.

Copy link
@mjsax

mjsax May 1, 2019

Member

@guozhangwang I was just talking about the foreign-key case (not sure why you thought it might be anything else?). My understanding is the following: The contract is that we should pass a topic name into the serializer of which we want to write the data into. This contract breaks if we pass in the repartition topic name, because we write something different into the repartition topic.

You are right that the changelog topic might not exist, however, my personal take is, that registering for a non-existing topic, is a smaller violation of the contract that passing in the "wrong" repartition topic name. Note, that the changelog topic name is conceptually the "right" topic name. However, this case would not happen very often anyway (compare examples below).

Your comment trigger one more thought: the optimization framework could actually check for different cases, and if there is an upstream topic (either changelog or source topic that has the same schema), we could actually use this name.

Some examples (does not cover all cases):

builder.table("table-topic").foreignKeyJoin(...)

For this case we need to materialize the base table (that is also the join-table), and the schema is registered on table-topic already, so we can pass in table-topic to avoid leaking anything.

builder.table("table-topic").filter(...).foreignKeyJoin(...)

For this case we materialize the derived table from the filter() and we get a proper filter-changelog-topic and we can pass this one.

builder.stream("stream-topic").groupBy().aggregate().foreignKeyJoin(...)

For this case, the agg result KTable is materialized and we can pass the agg-changelog-topic as name.

builder.stream("stream-topic").groupBy().aggregate().filter().foreignKeyJoin(...)

For this case, the agg result KTable is materialized and we can pass the agg-changelog-topic as name, because the filter() does not change the schema. Thus, even if the join-input KTable is not materialized, we can avoid to leak anything by "borrowing" the upstream changelog topic name of the filter input KTable.

builder.table("table-topic").mapValues(...).foreignKeyJoin(...)

For this case, we need to materialize the result of mapValues() and get a proper changelog topic for the join-input table.

builder.table("table-topic", Materialized.as("foo")).mapValues(...).foreignKeyJoin(...)

This might be a weird case, for which the base table is materialized, while the input join-table would not be materialized, and also the type changes via mapValues(). Hence, the table-topic schema is not the same as the join schema and we also don't have a changelog topic for the join-input KTable. We still use the changelog-topic name of the non-existent changelog topic (of the mapValues() result KTable).

As you can see, we can cover a large scope of cases for which we don't leak anything and can always use a topic name that contains data corresponding to the schema. Does this explain my thoughts?

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang May 2, 2019

Contributor

I understand your reasoning now, but still I felt Streams should not fix it trying to piggy-back on another topic that happens to be of the same schema that this serde is used for; or rather, I'd prefer to use a non-exist dummy topic than an existing topic if we do not like repartition topics (again, I agree that repartition topic is not ideal, since we are, in fact, not sending the bytes serialized in that way to the topic).

@vvcephei

This comment has been minimized.

Copy link
Contributor

commented Apr 16, 2019

Hi @bellemare ,

Thanks for your PR! I'll review this as soon as I get the chance, and pay particular attention to the points you called out.

-John

//we do not want that at all. Will be removed in final version, but this just illustrates the
//main issue with the confluent avro serde.
//Murmur3.hash128(valueSerializer.serialize(new Random().nextLong() + "", currentValue)));
Murmur3.hash128(valueSerializer.serialize(null, currentValue)));

This comment has been minimized.

Copy link
@adaniline-traderev

adaniline-traderev Apr 17, 2019

Here is another instance where a serializer is called with null topic. I was just wondering if it is possible to pass

context().topic()

as topic name - seemed to worked in my testing scenario...

This comment has been minimized.

Copy link
@bellemare

bellemare Apr 30, 2019

Author

After a long discussion, yes, this is what I think is currently the best option. Thanks for the suggestion in the first place!

@vvcephei
Copy link
Contributor

left a comment

Hey! I've made a fractional pass, and ran out of time, so I'm sending my comments so far.

@@ -417,6 +418,18 @@ public static int murmur2(final byte[] data) {
return h;
}

//TODO - Bellemare - This can be removed if we stick with Murmur3, which I think we should.

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 1, 2019

Contributor

Might as well just remove it. I think Murmur3 is fine.

This comment has been minimized.

Copy link
@bellemare
* "All MurmurHash versions are public domain software, and the author disclaims all copyright
* to their code."
*/
public class Murmur3 {

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 1, 2019

Contributor

Wouldn't hurt to have some tests for this. Maybe copy those from Hive as well.

This comment has been minimized.

Copy link
@bellemare

bellemare May 14, 2019

Author

Done. I copied the tests over. They depend on Guava so I added that as a test dependency for the kafka common package. It's also Apache-2.0 license so I don't think there's a problem with this, but if there is please let me know.

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 15, 2019

Contributor

I think this is fine, but of course I'm not a lawyer. :)


public class CombinedKey<KF, KP> {
private final KF foreignKey;
private KP primaryKey = null;

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 1, 2019

Contributor

Recommend making this final as well, and just moving the null initialization to the fk constructor.

This comment has been minimized.

Copy link
@bellemare
}

public KF getForeignKey() {
return this.foreignKey;

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 1, 2019

Contributor
Suggested change
return this.foreignKey;
return foreignKey;

We try to avoid unnecessary qualifiers. Also applies elsewhere. I won't call them out further at this time.

This comment has been minimized.

Copy link
@bellemare

bellemare May 14, 2019

Author

Roger that. Will review code + apply.

this.foreignKey = foreignKey;
}

public KF getForeignKey() {

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 1, 2019

Contributor
Suggested change
public KF getForeignKey() {
KF getForeignKey() {

In general, we try to retrict visibility to the minimum required. I won't call these out further at this time.

private final CombinedKeySerde<KO, K> keySerde;
private final boolean byPrimaryKey;

//Use a custom partitioner.

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 1, 2019

Contributor

not sure what this means...

This comment has been minimized.

Copy link
@bellemare

bellemare May 14, 2019

Author

Deprecated comment. Should have been removed. Fixed.

private final boolean byPrimaryKey;

//Use a custom partitioner.
public DefaultCombinedKeyPartitioner(final CombinedKeySerde<KO, K> keySerde, final boolean byPrimaryKey) {

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 1, 2019

Contributor

My immediate reaction is that it's a little surprising that byPrimaryKey := false means "use only the foreign key". Boolean flags like this generally contribute to code complexity/obscurity. What do you think about just making separate partitioners for the primary and foreign keys?

This comment has been minimized.

Copy link
@bellemare

bellemare May 14, 2019

Author

Good point. I think you are correct and that it will be clearer. The irony here is that I never use it with byPrimaryKey=True... I think I abandoned that part of the code and forgot to clean it up. So, no more boolean, and I will clean up the name.

super.init(context);
foreignValues = foreignValueGetterSupplier.get();
foreignValues.init(context);
store = (KeyValueStore<CombinedKey<KO, K>, SubscriptionWrapper>) context.getStateStore(topicName);

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 1, 2019

Contributor

it's a little surprising that the store's name is the same as the topic name

This comment has been minimized.

Copy link
@bellemare

bellemare May 14, 2019

Author

Bit of a typo there. It should be the state store name. This state store is materializing the CombinedKey and so should not be associated with any particular topic name. I will clear this up.

store.put(key, value);
}

//What to do if the foreign key actually is null? Can't partition correctly...

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 1, 2019

Contributor

generally, we forbid null keys in KTable APIs for this reason. Feel free to just drop the record, log a message, and increment the right metric (look around for other null-key checks in the KTable processors).

This comment has been minimized.

Copy link
@bellemare
((KTableImpl<?, ?, ?>) other).enableSendingOldValues();
enableSendingOldValues();

final String repartitionProcessorName = builder.newProcessorName(REPARTITION_NAME);

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 1, 2019

Contributor

Just dropping this in before I forget, following KIP-307, we need to provide a mechanism to name these operators and internal states.

This comment has been minimized.

Copy link
@bellemare

bellemare May 14, 2019

Author

The wiki says it's currently under discussion - has it already been accepted? Just curious as to if this work needs to be done for 2.3 if it's not going to be a part of it.

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 15, 2019

Contributor

Woah, it looks like the contributor forgot to update the KIP. It was accepted. I'll update the wiki.

@vvcephei
Copy link
Contributor

left a comment

Hi @bellemare , what's the status of this PR? It seems there was some hanging discussion...

Here's another very small batch of review comments:

@@ -1140,6 +1141,25 @@
final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);

/**
* Joins the records of this KTable to another table keyed on a different key. Updates from this table will join
* 1 to 1 on the other table. Updates to the other table will induce a join on each record in this table that has

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 10, 2019

Contributor

This reads a little funny to me. Can we just say that it's a many:1 join with the other table, and the foreignKeyExtractor selects the key in the other table to join with?

This comment has been minimized.

Copy link
@bellemare

bellemare May 14, 2019

Author

Done. Let me know if it's still unclear.

@@ -641,4 +669,180 @@ boolean sendingOldValueEnabled() {
return (ProcessorParameters<K, VR>) kObjectProcessorParameters;
}

public <VR, KO, VO> KTable<K, VR> join(final KTable<KO, VO> other,

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 10, 2019

Contributor

Missing @Override

This comment has been minimized.

Copy link
@bellemare
final ValueMapper<V, KO> foreignKeyExtractor,
final ValueJoiner<V, VO, VR> joiner,
final MaterializedInternal<K, VR, KeyValueStore<Bytes, byte[]>> materialized) {
((KTableImpl<?, ?, ?>) other).enableSendingOldValues();

This comment has been minimized.

Copy link
@vvcephei

vvcephei May 10, 2019

Contributor

Feels like this deserves a comment.

This comment has been minimized.

Copy link
@bellemare
@bellemare

This comment has been minimized.

Copy link
Author

commented May 11, 2019

@vvcephei Hi John - thanks for the feedback so far! I haven't had time to attend to this due to some recent personal matters, but I should be able to take a crack at it this upcoming week. I think that (in my mind) the discussion about the topic names has been resolved, so I don't think there are any impediments other than me getting this cleaned up and then rebased to trunk.

@bellemare

This comment has been minimized.

Copy link
Author

commented May 14, 2019

@vvcephei - Completed all your feedback so far John. Thanks so much.
I also updated the Wiki page to more clearly show the underlying implementation in the diagram. I renamed some of the processors and hopefully added more clarity with the comments and other work.

Currently, I do not know enough about the underlying mechanisms to get variable-partition counts working (ie: join a KTable with 7 partitions with that of 11 partitions). This is explained above in the April 12th post on KTableKTableForeignKeyInnerJoinMultiIntegrationTest. Multi-partition support could be added in a later revision if we wish to get this in for 2.3.

I will rebase this to trunk and commit that too shortly.

Merged and resolved differences. Left a number of TODOs, and now tryi…
…ng to get the integration test to pass again.
@bellemare

This comment has been minimized.

Copy link
Author

commented May 15, 2019

Rebasing to trunk has been considerably longer than I planned. Dealing with the new timestamped data stores has been a bit of a nightmare. Additionally, data which used to be present in the KTableImpl class is no longer available. In the KTableImpl constructor,storebuilder and isQueryable have been replaced by materialized.queryableStoreName(), which means that I do not have the ability to attach my resolver to the original, "this" materialized instance in the case where a queryable name is not set. I will look at ways to resolve this, but I do not anticipate being done before 2.3. I have spent considerable time on it in the past day and it's looking like much more is required.

@vvcephei

This comment has been minimized.

Copy link
Contributor

commented May 15, 2019

Hey @bellemare ,

Thanks for the update, and for the rebase work. Yes, the new timestamped stores changed a lot of implementation classes. It's a bummer that it happened to get merged after you forked. I agree, it's unlikely that this be able to get merged by Friday (the feature freeze for 2.3).

But no worries, it just means that we'll have more time to review it, write lots of tests, system tests, work on docs and blogs, etc., before it does get released, which decreases the overall risk of such a big feature.

Once you finish up the rebase, I'll take another pass. I didn't make it all the way through last time, and wound up just commenting on the little things I noticed along the way.

-John

@bellemare

This comment has been minimized.

Copy link
Author

commented May 16, 2019

There's an issue with the changing of the KTableImpl API

2.3-trunk:

public KTableImpl(final String name,
                  final Serde<K> keySerde,
                  final Serde<V> valSerde,
                  final Set<String> sourceNodes,
                  final String queryableStoreName,
                  final ProcessorSupplier<?, ?> processorSupplier,
                  final StreamsGraphNode streamsGraphNode,
                  final InternalStreamsBuilder builder) {
    super(name, keySerde, valSerde, sourceNodes, streamsGraphNode, builder);
    this.processorSupplier = processorSupplier;
    this.queryableStoreName = queryableStoreName;
}

2.0-trunk (what I had rebased it to last)

public KTableImpl(final String name,
                  final Serde<K> keySerde,
                  final Serde<V> valSerde,
                  final Set<String> sourceNodes,
                  final String queryableStoreName,
                  final boolean isQueryable,
                  final ProcessorSupplier<?, ?> processorSupplier,
                  final StreamsGraphNode streamsGraphNode,
                  final InternalStreamsBuilder builder) {
    super(name, keySerde, valSerde, sourceNodes, streamsGraphNode, builder);
    this.processorSupplier = processorSupplier;
    this.queryableStoreName = queryableStoreName;
    this.isQueryable = isQueryable;
}

isQueryable and queryableStoreName in 2.0 have been replaced by just queryableStoreName in 2.3, with a null value intending to mean that it is not queryable.

The problem is that the oneToMany joiner needs to query the underlying statestore previously represented by this.queryableStoreName during resolution of the hash code values. Previously, queryableStoreName was populated even if the underlying state store was anonymous (ie: not passed in as a Materialized parameter). Now, however, if the user does not materialize a state store, the anonymous one is NOT passed in as this.queryableStoreName. I think that it was only my code which would be affected by this unfortunately. I think I will need to change it back to how it was in 2.0 unless someone has any other ideas about how to get the underlying queryableStoreName when it is anonymous.

oneToMany join Working, with a number of prefixScans stubbed out. Als…
…o requires that user pass in materialized stores.

@bellemare bellemare force-pushed the bellemare:trunk-oneToMany branch from 3de35df to 0d6d92e May 16, 2019

@bellemare
Copy link
Author

left a comment

I will go back through it and clean up the comments, the commented-out code, etc. As it stands currently though, it does have a few issues that I have highlighted that I would love to get some specific feedback on.


public interface KTablePrefixValueGetter<K, V> extends KTableValueGetter<K, V> {

KeyValueIterator<K, V> prefixScan(final K prefix);

This comment has been minimized.

Copy link
@bellemare

bellemare May 16, 2019

Author

I put this in its own interface for now because it's not immediately clear where it should live. It needs to be part of StateStore, given how all of the various wrapper classes, like the metered, caching and logging ones work. However, I do not want it to be in the ReadOnlyKeyValueStore as it shouldn't be exposed to the outside world.

*/
@Override
public KeyValueIterator<Bytes, byte[]> prefixScan(final Bytes prefix) {
throw new UnsupportedOperationException("prefixScan() not supported in " + getClass().getName());

This comment has been minimized.

Copy link
@bellemare

bellemare May 16, 2019

Author

One of the effects of prefixScan needing to be attached to StateStore

@Override
public KeyValueIterator<Bytes, byte[]> prefix(Bytes prefix) {
//TODO - Bellemare - How do we support this here?
throw new UnsupportedOperationException("prefixScan() not supported in " + getClass().getName());

This comment has been minimized.

Copy link
@bellemare

bellemare May 16, 2019

Author

I'm not really sure how the Timestamped statestores work, nor if I even need to support this (since my intention is to only use the non-timestamped versions).

if (!open) {
throw new InvalidStateStoreException(String.format("RocksDB iterator for store %s has closed", storeName));
}
if (!super.hasNext()) {

This comment has been minimized.

Copy link
@bellemare

bellemare May 16, 2019

Author

This is a weird one. I end up with a stack overflow error as the above function calls just go back and forth. It seems like it's getting some weird scope problem. This works around it, but I am not sure if I stumbled on an existing bug or if it's something I introduced. I'm just not really sure how what I did could have caused this.

//
// byte[] md5Null = Utils.md5(new byte[]{});
// String foo = bytesToHex(md5Null).toString();
// System.out.println(foo);

This comment has been minimized.

Copy link
@bellemare

bellemare May 16, 2019

Author

All the above will be cleaned up.

private KafkaStreams prepareTopology(final String queryableName) {
final StreamsBuilder builder = new StreamsBuilder();

//TODO - bellemare - this appears to only work when I materialize the state... because I cannot access auto-materialized stateSupplier in internalStreamsBuilder!

This comment has been minimized.

Copy link
@bellemare

bellemare May 16, 2019

Author

I'm required to supply a materialized store with a name, otherwise the internals of KTableImpl cannot get the name for the state store, and therefore the SubscriptionResolverJoinProcessorSupplier cannot access it (fails with NPE). This is related to the aforementioned changes to the KTableImpl constructor dropping the isQueryable boolean.

// return new ProcessorContextImpl.WindowStoreReadWriteDecorator((WindowStore) store);
// } else if (store instanceof SessionStore) {
// return new ProcessorContextImpl.SessionStoreReadWriteDecorator((SessionStore) store);
// }

This comment has been minimized.

Copy link
@bellemare

bellemare May 16, 2019

Author

Am I going to need to handle all of these statestore types? I suspect I'll need to handle both TimestampedKeyValueStore and regular KeyValueStore, but given that context.getStateStore can return any of them, I don't know if I should expect to handle the remainder.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
You can’t perform that action at this time.