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

feat: build ks app from an execution plan visitor #3418

Merged

Conversation

rodesai
Copy link
Contributor

@rodesai rodesai commented Sep 26, 2019

Description

This patch implements a visitor that iterates over the execution plan
and builds the final kstreams app. In addition to defining and
implementing the visitor, this required updating the type built by
many of the plan nodes to a wrapper class that includes both a kstream/
ktable, and a factory for building key serdes.

Now that we have this visior, we no longer need the code in SchemaKX
that makes calls into kafka streams, so that's all cleaned up.

Finally, we need to actually call the visitor to build the streams app.
For now that's happening in PhysicalPlanBuilder, but that will get moved
very soon.

Testing done

Adapted execution plan builder tests to incorporate the new visitor.

Reviewer checklist

  • Ensure docs are updated if necessary. (eg. if a user visible feature is being added or changed).
  • Ensure relevant issues are linked (description should include text like "Fixes #")

@rodesai rodesai requested a review from a team as a code owner September 26, 2019 00:08
final StreamSource<?> streamSource
) {
if (streamSource.getFormats().getKeyFormat().isWindowed()) {
return (StreamWithKeySerdeFactory) buildWindowed(queryBuilder,streamSource);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is not ideal. Planning to clean this up in a follow-on that adds a separate step type for a windowed source so we can have it strictly typed (similar to aggregates).

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

@rodesai rodesai force-pushed the build-streams-from-visitor-rebase branch 2 times, most recently from ec682fe to d688767 Compare October 1, 2019 02:14
Copy link
Contributor

@purplefox purplefox left a comment

Choose a reason for hiding this comment

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

Looks great! Just a few small comments, mainly on naming.

if (sourceType == DataSourceType.KTABLE) {
((SchemaKTable) schemaKStream).getSourceTableStep().build(planBuilder);
Copy link
Contributor

Choose a reason for hiding this comment

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

As above

import org.apache.kafka.streams.kstream.KGroupedTable;
import org.apache.kafka.streams.kstream.Windowed;

public interface PlanBuilder {
Copy link
Contributor

Choose a reason for hiding this comment

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

Would be nice to have a few lines of class header comment explaining what the purpose of this interface is (Not expecting full javadoc!)

import java.util.Objects;
import org.apache.kafka.streams.kstream.KStream;

public final class StreamWithKeySerdeFactory<K> {
Copy link
Contributor

Choose a reason for hiding this comment

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

This class doesn't seems to be a StreamWithKeySerdeFactory (i.e. it doesn't create instances of StreamWithKeySerde) but more of a holder class for a stream and a KeySerdeFactory. Perhaps it could have a better name? Maybe StreamHolder?

import java.util.Objects;
import org.apache.kafka.streams.kstream.KTable;

public final class TableWithKeySerdeFactory<K> {
Copy link
Contributor

Choose a reason for hiding this comment

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

Naming again. Maybe TableHolder?

testCase.getGeneratedSchemas().get(0).split(System.lineSeparator()));
assertThat("Schemas used by topology differ "
+ "from those used by previous versions"
+ " of KSQL - this likely means there is a non-backwards compatible change.\n"
Copy link
Contributor

Choose a reason for hiding this comment

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

"... this is likely to mean..."

import org.apache.kafka.streams.kstream.KGroupedTable;
import org.apache.kafka.streams.kstream.Windowed;

public final class KSPlanBuilder implements PlanBuilder {
Copy link
Contributor

Choose a reason for hiding this comment

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

Just a few lines of class header comment would be awesome "This is an implementation of PlanBuilder which uses Kafka streams...."

public static <K> KStream<K, GenericRow> build(
final KStream<K, GenericRow> kstream,
final StreamFilter<KStream<K, GenericRow>> step,
public static <K> StreamWithKeySerdeFactory<K> build(
Copy link
Contributor

Choose a reason for hiding this comment

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

This is kind of a general point. I notice that we use "build" in the codebase a lot. And often it's used where the builder pattern is not actually being used, rather, something is just being created. I'd suggest only using "build" where there's a real builder pattern in place, and using "create" in the case of a simple factory method.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fair enough. I'll defer this to another PR though to keep the scope of the changes here at a minimum.

final StreamSource<?> streamSource
) {
if (streamSource.getFormats().getKeyFormat().isWindowed()) {
return (StreamWithKeySerdeFactory) buildWindowed(queryBuilder,streamSource);
Copy link
Contributor

Choose a reason for hiding this comment

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

+1

@@ -79,15 +79,20 @@ private StreamStreamJoinBuilder() {
);
final KsqlValueJoiner joiner = new KsqlValueJoiner(leftSchema, rightSchema);
final JoinWindows joinWindows = JoinWindows.of(join.getBefore()).after(join.getAfter());
final KStream<K, GenericRow> result;
switch (join.getJoinType()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Kind of tangential - but I wonder why we don't support RIGHT joins? It would be a simple matter for us to just swap left and right here, rather than forcing the user to do this.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not sure. What's the value over just doing a left join?

@rodesai rodesai force-pushed the build-streams-from-visitor-rebase branch from d688767 to 8f41e9d Compare October 2, 2019 17:55
This patch implements a visitor that iterates over the execution plan
and builds the final kstreams app. In addition to defining and
implementing the visitor, this required updating the type built by
many of the plan nodes to a wrapper class that includes both a kstream/
ktable, and a factory for building key serdes.

Now that we have this visior, we no longer need the code in SchemaKX
that makes calls into kafka streams, so that's all cleaned up.

Finally, we need to actually call the visitor to build the streams app.
For now that's happening in PhysicalPlanBuilder, but that will get moved
very soon.
@rodesai rodesai force-pushed the build-streams-from-visitor-rebase branch from 8f41e9d to edfa8c9 Compare October 2, 2019 18:14
Copy link
Member

@stevenpyzhang stevenpyzhang left a comment

Choose a reason for hiding this comment

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

Overall, I think it looks good, left 2 minor comments.

) {
final DataSourceType sourceType = (schemaKStream instanceof SchemaKTable)
? DataSourceType.KTABLE
: DataSourceType.KSTREAM;

final DataSource<?> sinkDataSource;
final PlanBuilder planBuilder = new KSPlanBuilder(ksqlQueryBuilder);
Copy link
Member

Choose a reason for hiding this comment

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

suggestion (nit): Since a new new KSPlanBuilder() object is created in both buildPlanForStructuredOutputNode() and buildPlanForBareQuery(), this new object could instead be created in buildPlan() and passed into both methods. I don't feel too strongly about this though.

import org.apache.kafka.streams.kstream.KeyValueMapper;

public final class TableGroupByBuilder {
private TableGroupByBuilder() {
}

public static <K> KGroupedTable<Struct, GenericRow> build(
final KTable<K, GenericRow> ktable,
final KTableHolder<K> table,
Copy link
Member

Choose a reason for hiding this comment

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

[nit] Could this be renamed to tableHolder? This goes for other holder variables in all the other builders. It seems weird to have table.getTable() and stream.getStream() (it looks like we already have a table available, why would we need to call getTable() from it?)

@rodesai rodesai merged commit b57d194 into confluentinc:master Oct 2, 2019
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.

None yet

3 participants