mirror of https://github.com/apache/druid.git
use druidhook instead Hook
This commit is contained in:
parent
6c6053eee0
commit
7a65938fd6
|
@ -23,7 +23,7 @@ import org.apache.druid.sql.calcite.run.DruidHook;
|
|||
|
||||
import java.io.PrintStream;
|
||||
|
||||
public class QuidemRecorder implements AutoCloseable, DruidHook
|
||||
public class QuidemRecorder implements AutoCloseable, DruidHook<String>
|
||||
{
|
||||
private PrintStream printStream;
|
||||
|
||||
|
@ -42,10 +42,10 @@ public class QuidemRecorder implements AutoCloseable, DruidHook
|
|||
}
|
||||
|
||||
@Override
|
||||
public <T> void invoke(HookKey<T> key, T object)
|
||||
public void invoke(HookKey<String> key, String query)
|
||||
{
|
||||
if (DruidHook.SQL.equals(key)) {
|
||||
printStream.print(object);
|
||||
printStream.print(query);
|
||||
printStream.println(";");
|
||||
printStream.println("!ok");
|
||||
return;
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.calcite.rel.rules.DateRangeRules;
|
|||
import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
|
||||
import org.apache.calcite.rel.rules.ProjectMergeRule;
|
||||
import org.apache.calcite.rel.rules.PruneEmptyRules;
|
||||
import org.apache.calcite.runtime.Hook;
|
||||
import org.apache.calcite.sql.SqlExplainFormat;
|
||||
import org.apache.calcite.sql.SqlExplainLevel;
|
||||
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.logical.DruidAggregateRemoveRedundancyRule;
|
||||
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 java.util.ArrayList;
|
||||
|
@ -386,7 +386,7 @@ public class CalciteRulesManager
|
|||
public RelNode run(RelOptPlanner planner, RelNode rel, RelTraitSet requiredOutputTraits,
|
||||
List<RelOptMaterialization> materializations, List<RelOptLattice> lattices)
|
||||
{
|
||||
Hook.TRIMMED.run(rel);
|
||||
DruidHook.dispatch(DruidHook.LOGICAL_PLAN, rel);
|
||||
return rel;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,7 +47,6 @@ import org.apache.calcite.rel.type.RelDataType;
|
|||
import org.apache.calcite.rel.type.RelDataTypeFactory;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.runtime.Hook;
|
||||
import org.apache.calcite.schema.ScannableTable;
|
||||
import org.apache.calcite.sql.SqlExplain;
|
||||
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.logical.DruidLogicalConvention;
|
||||
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.QueryMaker;
|
||||
import org.apache.druid.sql.calcite.table.DruidTable;
|
||||
|
@ -155,7 +155,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
isPrepared = true;
|
||||
SqlNode validatedQueryNode = validatedQueryNode();
|
||||
rootQueryRel = handlerContext.planner().rel(validatedQueryNode);
|
||||
Hook.CONVERTED.run(rootQueryRel.rel);
|
||||
DruidHook.dispatch(DruidHook.CONVERTED_PLAN, rootQueryRel.rel);
|
||||
handlerContext.hook().captureQueryRel(rootQueryRel);
|
||||
final RelDataTypeFactory typeFactory = rootQueryRel.rel.getCluster().getTypeFactory();
|
||||
final SqlValidator validator = handlerContext.planner().getValidator();
|
||||
|
@ -563,7 +563,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
.plus(DruidLogicalConvention.instance()),
|
||||
newRoot
|
||||
);
|
||||
Hook.JAVA_PLAN.run(newRoot);
|
||||
DruidHook.dispatch(DruidHook.DRUID_PLAN, newRoot);
|
||||
|
||||
DruidQueryGenerator generator = new DruidQueryGenerator(plannerContext, (DruidLogicalNode) newRoot, rexBuilder);
|
||||
DruidQuery baseQuery = generator.buildQuery();
|
||||
|
@ -591,7 +591,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
|
||||
handlerContext.hook().captureDruidRel(druidRel);
|
||||
|
||||
Hook.JAVA_PLAN.run(druidRel);
|
||||
DruidHook.dispatch(DruidHook.DRUID_PLAN, druidRel);
|
||||
|
||||
if (explain != null) {
|
||||
return planExplanation(possiblyLimitedRoot, druidRel, true);
|
||||
|
|
|
@ -27,7 +27,9 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public interface DruidHook
|
||||
@FunctionalInterface
|
||||
|
||||
public interface DruidHook<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<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<>();
|
||||
|
||||
|
@ -81,7 +85,7 @@ public interface DruidHook
|
|||
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);
|
||||
return new Closeable()
|
||||
|
|
|
@ -34,13 +34,16 @@ import org.apache.calcite.util.Util;
|
|||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
|
||||
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 java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.Statement;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class DruidQuidemCommandHandler implements CommandHandler
|
||||
|
@ -56,7 +59,7 @@ public class DruidQuidemCommandHandler implements CommandHandler
|
|||
return new LogicalPlanCommand(lines, content);
|
||||
}
|
||||
if (line.startsWith("druidPlan")) {
|
||||
return new PhysicalPlanCommand(lines, content);
|
||||
return new DruidPlanCommand(lines, content);
|
||||
}
|
||||
if (line.startsWith("nativePlan")) {
|
||||
return new NativePlanCommand(lines, content);
|
||||
|
@ -155,19 +158,19 @@ public class DruidQuidemCommandHandler implements CommandHandler
|
|||
*/
|
||||
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);
|
||||
this.hook = hook;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final void executeExplain(Context x)
|
||||
protected final void executeExplain(Context x) throws IOException
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -185,15 +188,15 @@ public class DruidQuidemCommandHandler implements CommandHandler
|
|||
{
|
||||
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)
|
||||
{
|
||||
super(lines, content, Hook.CONVERTED);
|
||||
super(lines, content, DruidHook.CONVERTED_PLAN);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue