-
Notifications
You must be signed in to change notification settings - Fork 1k
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: Add support for IN clause to pull queries #6409
Changes from 1 commit
74f95f8
332017d
6b89705
db742d1
e02ab18
41f34c1
40ee19a
cd573e0
f505f71
4b662c8
fb493c2
1a55d1f
c13a9c8
4888c57
54f8359
f721ccb
09a06dd
d59cdfb
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -233,11 +233,11 @@ public PullQueryResult execute( | |
.getMaterialization(queryId, contextStacker) | ||
.orElseThrow(() -> notMaterializedException(getSourceName(analysis))); | ||
|
||
List<Optional<KsqlNode>> sourceNodes = new ArrayList<>(); | ||
List<List<?>> tableRows = new ArrayList<>(); | ||
List<LogicalSchema> schemas = new ArrayList<>(); | ||
List<Future<PullQueryResult>> futures = new ArrayList<>(); | ||
List<List<Struct>> keysByLocation = mat.locator().groupByLocation( | ||
final List<Optional<KsqlNode>> sourceNodes = new ArrayList<>(); | ||
final List<List<?>> tableRows = new ArrayList<>(); | ||
final List<LogicalSchema> schemas = new ArrayList<>(); | ||
final List<Future<PullQueryResult>> futures = new ArrayList<>(); | ||
final List<List<Struct>> keysByLocation = mat.locator().groupByLocation( | ||
whereInfo.keysBound.stream() | ||
.map(keyBound -> asKeyStruct(keyBound, query.getPhysicalSchema())) | ||
.collect(Collectors.toList())); | ||
|
@@ -286,7 +286,7 @@ public PullQueryResult execute( | |
} | ||
|
||
static void validateSchemas(final List<LogicalSchema> schemas) { | ||
LogicalSchema schema = Iterables.getLast(schemas); | ||
final LogicalSchema schema = Iterables.getLast(schemas); | ||
for (LogicalSchema s : schemas) { | ||
if (!schema.equals(s)) { | ||
throw new KsqlException("Schemas from different hosts should be identical"); | ||
|
@@ -334,7 +334,7 @@ private PullQueryResult handlePullQuery( | |
= routeQuery(node, statement, executionContext, serviceContext, pullQueryContext); | ||
final Optional<KsqlNode> debugNode = Optional.ofNullable( | ||
routingOptions.isDebugRequest() ? node : null); | ||
List<Optional<KsqlNode>> debugNodes = rows.getRows().stream() | ||
final List<Optional<KsqlNode>> debugNodes = rows.getRows().stream() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: probably makes sense to make this There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I had been thinking that maybe this was necessary because different machines can have different configs, but I now remember that this is a request config, so it should always work. Ok, made it |
||
.map(r -> debugNode) | ||
.collect(Collectors.toList()); | ||
return new PullQueryResult(rows, debugNodes); | ||
|
@@ -381,15 +381,15 @@ private static TableRows queryRowsLocally( | |
if (pullQueryContext.whereInfo.windowBounds.isPresent()) { | ||
final WindowBounds windowBounds = pullQueryContext.whereInfo.windowBounds.get(); | ||
|
||
ImmutableList.Builder<TableRow> allRows = ImmutableList.builder(); | ||
final ImmutableList.Builder<TableRow> allRows = ImmutableList.builder(); | ||
for (Struct key : pullQueryContext.keys) { | ||
final List<? extends TableRow> rows = pullQueryContext.mat.windowed() | ||
.get(key, windowBounds.start, windowBounds.end); | ||
allRows.addAll(rows); | ||
} | ||
result = new Result(pullQueryContext.mat.schema(), allRows.build()); | ||
} else { | ||
ImmutableList.Builder<TableRow> allRows = ImmutableList.builder(); | ||
final ImmutableList.Builder<TableRow> allRows = ImmutableList.builder(); | ||
for (Struct key : pullQueryContext.keys) { | ||
final List<? extends TableRow> rows = pullQueryContext.mat.nonWindowed() | ||
.get(key) | ||
|
@@ -622,8 +622,8 @@ private static List<Object> extractKeysFromInPredicate( | |
final boolean windowed, | ||
final LogicalSchema schema | ||
) { | ||
InPredicate inPredicate = Iterables.getLast(inPredicates); | ||
List<Object> result = new ArrayList<>(); | ||
final InPredicate inPredicate = Iterables.getLast(inPredicates); | ||
final List<Object> result = new ArrayList<>(); | ||
for (Expression expression : inPredicate.getValueList().getValues()) { | ||
if (!(expression instanceof Literal)) { | ||
throw new KsqlException("Ony comparison to literals is currently supported: " | ||
agavra marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
@@ -826,22 +826,6 @@ private static Instant asInstant(final Expression other) { | |
); | ||
} | ||
|
||
private static KeyAndWindowBounds extractWhereClauseTarget( | ||
final InPredicate inPredicate, | ||
final PersistentQueryMetadata query | ||
) { | ||
UnqualifiedColumnReferenceExp column = (UnqualifiedColumnReferenceExp) inPredicate.getValue(); | ||
final ColumnName keyColumn = Iterables.getOnlyElement(query.getLogicalSchema().key()).name(); | ||
if (column.getColumnName().equals(keyColumn)) { | ||
return new KeyAndWindowBounds().addInPredicate(inPredicate); | ||
} | ||
|
||
throw invalidWhereClauseException( | ||
"IN expression on unsupported column: " + column.getColumnName().text(), | ||
false | ||
); | ||
} | ||
|
||
private enum ComparisonTarget { | ||
WINDOWSTART, | ||
WINDOWEND | ||
|
@@ -853,30 +837,32 @@ private static class KeyAndWindowBounds { | |
private List<ComparisonExpression> windowEndExpression = new ArrayList<>(); | ||
private List<InPredicate> inPredicate = new ArrayList<>(); | ||
|
||
public KeyAndWindowBounds() { | ||
KeyAndWindowBounds() { | ||
} | ||
|
||
public KeyAndWindowBounds addKeyColExpression(ComparisonExpression keyColExpression) { | ||
public KeyAndWindowBounds addKeyColExpression(final ComparisonExpression keyColExpression) { | ||
this.keyColExpression.add(keyColExpression); | ||
return this; | ||
} | ||
|
||
public KeyAndWindowBounds addWindowStartExpression(ComparisonExpression windowStartExpression) { | ||
public KeyAndWindowBounds addWindowStartExpression( | ||
final ComparisonExpression windowStartExpression) { | ||
this.windowStartExpression.add(windowStartExpression); | ||
return this; | ||
} | ||
|
||
public KeyAndWindowBounds addWindowEndExpression(ComparisonExpression windowEndExpression) { | ||
public KeyAndWindowBounds addWindowEndExpression( | ||
final ComparisonExpression windowEndExpression) { | ||
this.windowEndExpression.add(windowEndExpression); | ||
return this; | ||
} | ||
|
||
public KeyAndWindowBounds addInPredicate(InPredicate inPredicate) { | ||
public KeyAndWindowBounds addInPredicate(final InPredicate inPredicate) { | ||
this.inPredicate.add(inPredicate); | ||
return this; | ||
} | ||
|
||
public KeyAndWindowBounds merge(KeyAndWindowBounds other) { | ||
public KeyAndWindowBounds merge(final KeyAndWindowBounds other) { | ||
keyColExpression.addAll(other.keyColExpression); | ||
windowStartExpression.addAll(other.windowStartExpression); | ||
windowEndExpression.addAll(other.windowEndExpression); | ||
|
@@ -962,6 +948,23 @@ private static KeyAndWindowBounds extractWhereClauseTarget( | |
); | ||
} | ||
|
||
private static KeyAndWindowBounds extractWhereClauseTarget( | ||
final InPredicate inPredicate, | ||
final PersistentQueryMetadata query | ||
) { | ||
final UnqualifiedColumnReferenceExp column | ||
= (UnqualifiedColumnReferenceExp) inPredicate.getValue(); | ||
final ColumnName keyColumn = Iterables.getOnlyElement(query.getLogicalSchema().key()).name(); | ||
if (column.getColumnName().equals(keyColumn)) { | ||
return new KeyAndWindowBounds().addInPredicate(inPredicate); | ||
} | ||
|
||
throw invalidWhereClauseException( | ||
"IN expression on unsupported column: " + column.getColumnName().text(), | ||
false | ||
); | ||
} | ||
|
||
private static boolean isSelectStar(final Select select) { | ||
final boolean someStars = select.getSelectItems().stream() | ||
.anyMatch(s -> s instanceof AllColumns); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why don't you return besides they keys also the active, standby per group? This way you wouldn't need to do
locate
twice basically.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, I think currently the routing is wrong and you have to return the <active,standby> per group
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, I had thought that a given set of partitions were grouped together at each active and standby, but I think you're right this isn't the case. I'll change it to
groupByActiveStandyList
or something similar. In practice, there aren't too many standbys, so this is likely to be a lot better than grouping by partition or just fetching by key.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I reworked this so that it now groups key together only if they share the same list of nodes, including active and all standbys. Most of the time if there's 1 or 2 standbys and lots of keys fetched, this will hopefully reduce unnecessary calls.