kfaraz commented on code in PR #16358:
URL: https://github.com/apache/druid/pull/16358#discussion_r1587418967
##########
sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java:
##########
@@ -81,12 +83,18 @@ public DruidExpression toDruidExpression(
final DruidExpression arg = inputExpressions.get(0);
final Expr lookupNameExpr =
plannerContext.parseExpression(inputExpressions.get(1).getExpression());
final String replaceMissingValueWith =
getReplaceMissingValueWith(inputExpressions, plannerContext);
+ String lookupName = (String) lookupNameExpr.getLiteralValue();
+
+ // Put the lookup names in the query context to facilitate selective
loading of lookups.
+
plannerContext.queryContextMap().putIfAbsent(PlannerContext.CTX_LOOKUPS_TO_LOAD,
new HashSet<>());
+ Set<String> lookupsToLoad = (Set<String>)
plannerContext.queryContextMap().get(PlannerContext.CTX_LOOKUPS_TO_LOAD);
+ lookupsToLoad.add(lookupName);
Review Comment:
```suggestion
plannerContext.queryContextMap().computeIfAbsent(PlannerContext.CTX_LOOKUPS_TO_LOAD,
key -> new HashSet<>()).add(lookupName);
```
##########
sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlResourceCollectorShuttle.java:
##########
@@ -87,13 +88,21 @@ public SqlNode visit(SqlIdentifier id)
if (qualifiedNameParts.size() == 2) {
final String schema = qualifiedNameParts.get(0);
final String resourceName = qualifiedNameParts.get(1);
+
+ // Put the lookup names in the query context to facilitate selective
loading of lookups.
+ if (schema.equals(NamedLookupSchema.NAME)) {
+
plannerContext.queryContextMap().putIfAbsent(PlannerContext.CTX_LOOKUPS_TO_LOAD,
new HashSet<>());
+ Set<String> lookupsToLoad = (Set<String>)
plannerContext.queryContextMap().get(PlannerContext.CTX_LOOKUPS_TO_LOAD);
+ lookupsToLoad.add(resourceName);
Review Comment:
```suggestion
plannerContext.queryContextMap().computeIfAbsent(PlannerContext.CTX_LOOKUPS_TO_LOAD,
key -> new HashSet<>()).add(resourceName);
```
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java:
##########
@@ -333,4 +336,15 @@ public static boolean writeResultsToDurableStorage(final
MSQSpec querySpec)
{
return querySpec.getDestination() instanceof DurableStorageMSQDestination;
}
+
+ @Override
+ public LookupLoadingSpec getLookupLoadingSpec()
+ {
+ if
(getQuerySpec().getQuery().getContext().containsKey(PlannerContext.CTX_LOOKUPS_TO_LOAD))
{
+ List<String> lookupsToLoad = (List<String>)
getQuerySpec().getQuery().getContext().get(PlannerContext.CTX_LOOKUPS_TO_LOAD);
Review Comment:
I think we are putting a set in the context, not a list.
##########
sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java:
##########
@@ -81,12 +83,18 @@ public DruidExpression toDruidExpression(
final DruidExpression arg = inputExpressions.get(0);
final Expr lookupNameExpr =
plannerContext.parseExpression(inputExpressions.get(1).getExpression());
final String replaceMissingValueWith =
getReplaceMissingValueWith(inputExpressions, plannerContext);
+ String lookupName = (String) lookupNameExpr.getLiteralValue();
Review Comment:
```suggestion
final String lookupName = (String)
lookupNameExpr.getLiteralValue();
```
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java:
##########
@@ -333,4 +336,15 @@ public static boolean writeResultsToDurableStorage(final
MSQSpec querySpec)
{
return querySpec.getDestination() instanceof DurableStorageMSQDestination;
}
+
+ @Override
+ public LookupLoadingSpec getLookupLoadingSpec()
+ {
+ if
(getQuerySpec().getQuery().getContext().containsKey(PlannerContext.CTX_LOOKUPS_TO_LOAD))
{
+ List<String> lookupsToLoad = (List<String>)
getQuerySpec().getQuery().getContext().get(PlannerContext.CTX_LOOKUPS_TO_LOAD);
+ return LookupLoadingSpec.loadOnly(new HashSet<>(lookupsToLoad));
+ } else {
+ return LookupLoadingSpec.NONE;
Review Comment:
The default behaviour being `NONE` might cause issues during a rolling
upgrade since Broker is typically upgraded after Peons and MMs.
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java:
##########
@@ -269,6 +270,14 @@ public static Map<String, Object> makeTaskContext(
.put(MultiStageQueryContext.CTX_IS_REINDEX,
MSQControllerTask.isReplaceInputDataSourceTask(querySpec))
.put(MultiStageQueryContext.CTX_MAX_CONCURRENT_STAGES,
queryKernelConfig.getMaxConcurrentStages());
+ // Put the lookup names in the query context to facilitate selective
loading of lookups.
+ if
(querySpec.getQuery().getContext().containsKey(PlannerContext.CTX_LOOKUPS_TO_LOAD)
&& querySpec.getQuery().getContext().get(PlannerContext.CTX_LOOKUPS_TO_LOAD)
!= null) {
Review Comment:
The first condition seems redundant:
```suggestion
if
(querySpec.getQuery().getContext().get(PlannerContext.CTX_LOOKUPS_TO_LOAD) !=
null) {
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]