use druidhook instead Hook

This commit is contained in:
Zoltan Haindrich 2024-06-11 15:55:55 +00:00
parent 6c6053eee0
commit 7a65938fd6
5 changed files with 30 additions and 23 deletions

View File

@ -23,7 +23,7 @@ import org.apache.druid.sql.calcite.run.DruidHook;
import java.io.PrintStream; import java.io.PrintStream;
public class QuidemRecorder implements AutoCloseable, DruidHook public class QuidemRecorder implements AutoCloseable, DruidHook<String>
{ {
private PrintStream printStream; private PrintStream printStream;
@ -42,10 +42,10 @@ public class QuidemRecorder implements AutoCloseable, DruidHook
} }
@Override @Override
public <T> void invoke(HookKey<T> key, T object) public void invoke(HookKey<String> key, String query)
{ {
if (DruidHook.SQL.equals(key)) { if (DruidHook.SQL.equals(key)) {
printStream.print(object); printStream.print(query);
printStream.println(";"); printStream.println(";");
printStream.println("!ok"); printStream.println("!ok");
return; return;

View File

@ -40,7 +40,6 @@ import org.apache.calcite.rel.rules.DateRangeRules;
import org.apache.calcite.rel.rules.JoinPushThroughJoinRule; import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
import org.apache.calcite.rel.rules.ProjectMergeRule; import org.apache.calcite.rel.rules.ProjectMergeRule;
import org.apache.calcite.rel.rules.PruneEmptyRules; import org.apache.calcite.rel.rules.PruneEmptyRules;
import org.apache.calcite.runtime.Hook;
import org.apache.calcite.sql.SqlExplainFormat; import org.apache.calcite.sql.SqlExplainFormat;
import org.apache.calcite.sql.SqlExplainLevel; import org.apache.calcite.sql.SqlExplainLevel;
import org.apache.calcite.sql2rel.RelDecorrelator; import org.apache.calcite.sql2rel.RelDecorrelator;
@ -68,6 +67,7 @@ import org.apache.druid.sql.calcite.rule.RewriteFirstValueLastValueRule;
import org.apache.druid.sql.calcite.rule.SortCollapseRule; import org.apache.druid.sql.calcite.rule.SortCollapseRule;
import org.apache.druid.sql.calcite.rule.logical.DruidAggregateRemoveRedundancyRule; import org.apache.druid.sql.calcite.rule.logical.DruidAggregateRemoveRedundancyRule;
import org.apache.druid.sql.calcite.rule.logical.DruidLogicalRules; import org.apache.druid.sql.calcite.rule.logical.DruidLogicalRules;
import org.apache.druid.sql.calcite.run.DruidHook;
import org.apache.druid.sql.calcite.run.EngineFeature; import org.apache.druid.sql.calcite.run.EngineFeature;
import java.util.ArrayList; import java.util.ArrayList;
@ -386,7 +386,7 @@ public class CalciteRulesManager
public RelNode run(RelOptPlanner planner, RelNode rel, RelTraitSet requiredOutputTraits, public RelNode run(RelOptPlanner planner, RelNode rel, RelTraitSet requiredOutputTraits,
List<RelOptMaterialization> materializations, List<RelOptLattice> lattices) List<RelOptMaterialization> materializations, List<RelOptLattice> lattices)
{ {
Hook.TRIMMED.run(rel); DruidHook.dispatch(DruidHook.LOGICAL_PLAN, rel);
return rel; return rel;
} }
} }

View File

@ -47,7 +47,6 @@ import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexNode;
import org.apache.calcite.runtime.Hook;
import org.apache.calcite.schema.ScannableTable; import org.apache.calcite.schema.ScannableTable;
import org.apache.calcite.sql.SqlExplain; import org.apache.calcite.sql.SqlExplain;
import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNode;
@ -73,6 +72,7 @@ import org.apache.druid.sql.calcite.rel.DruidRel;
import org.apache.druid.sql.calcite.rel.DruidUnionRel; import org.apache.druid.sql.calcite.rel.DruidUnionRel;
import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention;
import org.apache.druid.sql.calcite.rel.logical.DruidLogicalNode; import org.apache.druid.sql.calcite.rel.logical.DruidLogicalNode;
import org.apache.druid.sql.calcite.run.DruidHook;
import org.apache.druid.sql.calcite.run.EngineFeature; import org.apache.druid.sql.calcite.run.EngineFeature;
import org.apache.druid.sql.calcite.run.QueryMaker; import org.apache.druid.sql.calcite.run.QueryMaker;
import org.apache.druid.sql.calcite.table.DruidTable; import org.apache.druid.sql.calcite.table.DruidTable;
@ -155,7 +155,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
isPrepared = true; isPrepared = true;
SqlNode validatedQueryNode = validatedQueryNode(); SqlNode validatedQueryNode = validatedQueryNode();
rootQueryRel = handlerContext.planner().rel(validatedQueryNode); rootQueryRel = handlerContext.planner().rel(validatedQueryNode);
Hook.CONVERTED.run(rootQueryRel.rel); DruidHook.dispatch(DruidHook.CONVERTED_PLAN, rootQueryRel.rel);
handlerContext.hook().captureQueryRel(rootQueryRel); handlerContext.hook().captureQueryRel(rootQueryRel);
final RelDataTypeFactory typeFactory = rootQueryRel.rel.getCluster().getTypeFactory(); final RelDataTypeFactory typeFactory = rootQueryRel.rel.getCluster().getTypeFactory();
final SqlValidator validator = handlerContext.planner().getValidator(); final SqlValidator validator = handlerContext.planner().getValidator();
@ -563,7 +563,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
.plus(DruidLogicalConvention.instance()), .plus(DruidLogicalConvention.instance()),
newRoot newRoot
); );
Hook.JAVA_PLAN.run(newRoot); DruidHook.dispatch(DruidHook.DRUID_PLAN, newRoot);
DruidQueryGenerator generator = new DruidQueryGenerator(plannerContext, (DruidLogicalNode) newRoot, rexBuilder); DruidQueryGenerator generator = new DruidQueryGenerator(plannerContext, (DruidLogicalNode) newRoot, rexBuilder);
DruidQuery baseQuery = generator.buildQuery(); DruidQuery baseQuery = generator.buildQuery();
@ -591,7 +591,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
handlerContext.hook().captureDruidRel(druidRel); handlerContext.hook().captureDruidRel(druidRel);
Hook.JAVA_PLAN.run(druidRel); DruidHook.dispatch(DruidHook.DRUID_PLAN, druidRel);
if (explain != null) { if (explain != null) {
return planExplanation(possiblyLimitedRoot, druidRel, true); return planExplanation(possiblyLimitedRoot, druidRel, true);

View File

@ -27,7 +27,9 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
public interface DruidHook @FunctionalInterface
public interface DruidHook<T>
{ {
static class HookKey<T> static class HookKey<T>
{ {
@ -64,10 +66,12 @@ public interface DruidHook
} }
public static final HookKey<RelNode> CONVERTED_PLAN = new HookKey<>("converted", RelNode.class);
public static final HookKey<RelNode> LOGICAL_PLAN = new HookKey<>("logicalPlan", RelNode.class);
public static final HookKey<RelNode> DRUID_PLAN = new HookKey<>("druidPlan", RelNode.class); public static final HookKey<RelNode> DRUID_PLAN = new HookKey<>("druidPlan", RelNode.class);
public static final HookKey<String> SQL = new HookKey<>("sql", String.class); public static final HookKey<String> SQL = new HookKey<>("sql", String.class);
<T> void invoke(HookKey<T> key, T object); void invoke(HookKey<T> key, T object);
static Map<HookKey<?>, List<DruidHook>> GLOBAL = new HashMap<>(); static Map<HookKey<?>, List<DruidHook>> GLOBAL = new HashMap<>();
@ -81,7 +85,7 @@ public interface DruidHook
GLOBAL.get(key).remove(hook); GLOBAL.get(key).remove(hook);
} }
public static Closeable withHook(HookKey<?> key, DruidHook hook) public static <T> Closeable withHook(HookKey<T> key, DruidHook<T> hook)
{ {
register(key, hook); register(key, hook);
return new Closeable() return new Closeable()

View File

@ -34,13 +34,16 @@ import org.apache.calcite.util.Util;
import org.apache.druid.query.Query; import org.apache.druid.query.Query;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.rel.DruidRel; import org.apache.druid.sql.calcite.rel.DruidRel;
import org.apache.druid.sql.calcite.run.DruidHook;
import org.apache.druid.sql.calcite.run.DruidHook.HookKey;
import org.apache.druid.sql.calcite.util.QueryLogHook; import org.apache.druid.sql.calcite.util.QueryLogHook;
import java.io.Closeable;
import java.io.IOException;
import java.sql.ResultSet; import java.sql.ResultSet;
import java.sql.Statement; import java.sql.Statement;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.function.Consumer;
import java.util.stream.Collectors; import java.util.stream.Collectors;
public class DruidQuidemCommandHandler implements CommandHandler public class DruidQuidemCommandHandler implements CommandHandler
@ -56,7 +59,7 @@ public class DruidQuidemCommandHandler implements CommandHandler
return new LogicalPlanCommand(lines, content); return new LogicalPlanCommand(lines, content);
} }
if (line.startsWith("druidPlan")) { if (line.startsWith("druidPlan")) {
return new PhysicalPlanCommand(lines, content); return new DruidPlanCommand(lines, content);
} }
if (line.startsWith("nativePlan")) { if (line.startsWith("nativePlan")) {
return new NativePlanCommand(lines, content); return new NativePlanCommand(lines, content);
@ -155,19 +158,19 @@ public class DruidQuidemCommandHandler implements CommandHandler
*/ */
abstract static class AbstractRelPlanCommand extends AbstractPlanCommand abstract static class AbstractRelPlanCommand extends AbstractPlanCommand
{ {
Hook hook; HookKey<RelNode> hook;
AbstractRelPlanCommand(List<String> lines, List<String> content, Hook hook) AbstractRelPlanCommand(List<String> lines, List<String> content, DruidHook.HookKey<RelNode> hook)
{ {
super(lines, content); super(lines, content);
this.hook = hook; this.hook = hook;
} }
@Override @Override
protected final void executeExplain(Context x) protected final void executeExplain(Context x) throws IOException
{ {
List<RelNode> logged = new ArrayList<>(); List<RelNode> logged = new ArrayList<>();
try (final Hook.Closeable unhook = hook.add((Consumer<RelNode>) logged::add)) { try (Closeable unhook = DruidHook.withHook(hook, (key, relNode) -> { logged.add(relNode);})) {
executeQuery(x); executeQuery(x);
} }
@ -185,15 +188,15 @@ public class DruidQuidemCommandHandler implements CommandHandler
{ {
LogicalPlanCommand(List<String> lines, List<String> content) LogicalPlanCommand(List<String> lines, List<String> content)
{ {
super(lines, content, Hook.TRIMMED); super(lines, content, DruidHook.LOGICAL_PLAN);
} }
} }
static class PhysicalPlanCommand extends AbstractRelPlanCommand static class DruidPlanCommand extends AbstractRelPlanCommand
{ {
PhysicalPlanCommand(List<String> lines, List<String> content) DruidPlanCommand(List<String> lines, List<String> content)
{ {
super(lines, content, Hook.JAVA_PLAN); super(lines, content, DruidHook.DRUID_PLAN);
} }
} }
@ -201,7 +204,7 @@ public class DruidQuidemCommandHandler implements CommandHandler
{ {
ConvertedPlanCommand(List<String> lines, List<String> content) ConvertedPlanCommand(List<String> lines, List<String> content)
{ {
super(lines, content, Hook.CONVERTED); super(lines, content, DruidHook.CONVERTED_PLAN);
} }
} }
} }