This is an automated email from the ASF dual-hosted git repository.

rubenql pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git


The following commit(s) were added to refs/heads/master by this push:
     new ce25311  [CALCITE-4737] Add RelOptPlanner visualizer for debugging 
(Zuozhi Wang, Thomas Rebele)
ce25311 is described below

commit ce2531148f0c9990792068b12000deac827fc831
Author: Thomas Rebele <thomas.reb...@gmail.com>
AuthorDate: Fri Dec 17 10:35:59 2021 +0100

    [CALCITE-4737] Add RelOptPlanner visualizer for debugging (Zuozhi Wang, 
Thomas Rebele)
---
 .../plan/visualizer/InputExcludedRelWriter.java    |  89 ++++
 .../calcite/plan/visualizer/NodeUpdateHelper.java  | 107 +++++
 .../plan/visualizer/RuleMatchVisualizer.java       | 487 +++++++++++++++++++++
 .../apache/calcite/plan/visualizer/StepInfo.java   |  50 +++
 .../calcite/plan/visualizer/package-info.java      |  23 +
 .../calcite/plan/visualizer/viz-template.html      | 421 ++++++++++++++++++
 .../org/apache/calcite/test/RelOptTestBase.java    |   4 +-
 .../calcite/test/RuleMatchVisualizerTest.java      | 138 ++++++
 .../calcite/test/RuleMatchVisualizerTest.xml       | 264 +++++++++++
 9 files changed, 1581 insertions(+), 2 deletions(-)

diff --git 
a/core/src/main/java/org/apache/calcite/plan/visualizer/InputExcludedRelWriter.java
 
b/core/src/main/java/org/apache/calcite/plan/visualizer/InputExcludedRelWriter.java
new file mode 100644
index 0000000..5608da8
--- /dev/null
+++ 
b/core/src/main/java/org/apache/calcite/plan/visualizer/InputExcludedRelWriter.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.visualizer;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Pair;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An implement of RelWriter for explaining a single RelNode.
+ * The result only contains the properties of the RelNode,
+ * but does not explain the children.
+ *
+ * <pre>{@code
+ * InputExcludedRelWriter relWriter = new InputExcludedRelWriter();
+ * rel.explain(relWriter);
+ * String digest = relWriter.toString();
+ * }</pre>
+ *
+ */
+class InputExcludedRelWriter implements RelWriter {
+
+  private final Map<String, @Nullable Object> values = new LinkedHashMap<>();
+
+  InputExcludedRelWriter() {
+  }
+
+
+  @Override public void explain(RelNode rel, List<Pair<String, @Nullable 
Object>> valueList) {
+    valueList.forEach(pair -> {
+      assert pair.left != null;
+      this.values.put(pair.left, pair.right);
+    });
+  }
+
+  @Override public SqlExplainLevel getDetailLevel() {
+    return SqlExplainLevel.EXPPLAN_ATTRIBUTES;
+  }
+
+  @Override public RelWriter input(String term, RelNode input) {
+    // do nothing, ignore input
+    return this;
+  }
+
+  @Override public RelWriter item(String term, @Nullable Object value) {
+    this.values.put(term, value);
+    return this;
+  }
+
+  @Override public RelWriter itemIf(String term, @Nullable Object value, 
boolean condition) {
+    if (condition) {
+      this.values.put(term, value);
+    }
+    return this;
+  }
+
+  @Override public RelWriter done(RelNode node) {
+    return this;
+  }
+
+  @Override public boolean nest() {
+    return false;
+  }
+
+  @Override public String toString() {
+    return values.toString();
+  }
+}
diff --git 
a/core/src/main/java/org/apache/calcite/plan/visualizer/NodeUpdateHelper.java 
b/core/src/main/java/org/apache/calcite/plan/visualizer/NodeUpdateHelper.java
new file mode 100644
index 0000000..cdbbb9d
--- /dev/null
+++ 
b/core/src/main/java/org/apache/calcite/plan/visualizer/NodeUpdateHelper.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.visualizer;
+
+import org.apache.calcite.rel.RelNode;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Helper class to create the node update.
+ */
+class NodeUpdateHelper {
+
+  private final String key;
+  private final @Nullable RelNode rel;
+  private final NodeUpdateInfo state;
+  private @Nullable NodeUpdateInfo update = null;
+
+  NodeUpdateHelper(String key, @Nullable RelNode rel) {
+    this.key = key;
+    this.rel = rel;
+    this.state = new NodeUpdateInfo();
+  }
+
+  String getKey() {
+    return key;
+  }
+
+  @Nullable RelNode getRel() {
+    return this.rel;
+  }
+
+  void updateAttribute(final String attr, final Object newValue) {
+    if (Objects.equals(newValue, state.get(attr))) {
+      return;
+    }
+
+    state.put(attr, newValue);
+
+    if (update == null) {
+      update = new NodeUpdateInfo();
+    }
+
+    if (newValue instanceof List
+        && ((List<?>) newValue).size() == 0
+        && !update.containsKey(attr)) {
+      return;
+    }
+
+    update.put(attr, newValue);
+  }
+
+  boolean isEmptyUpdate() {
+    return this.update == null || update.isEmpty();
+  }
+
+  /**
+   * Gets an object representing all the changes since the last call to this 
method.
+   *
+   * @return an object or null if there are no changes.
+   */
+  @Nullable Object getAndResetUpdate() {
+    if (isEmptyUpdate()) {
+      return null;
+    }
+    NodeUpdateInfo update = this.update;
+    this.update = null;
+    return update;
+  }
+
+  Map<String, Object> getState() {
+    return Collections.unmodifiableMap(this.state);
+  }
+
+  /**
+   * Get the current value for the attribute.
+   */
+  @Nullable Object getValue(final String attr) {
+    return this.state.get(attr);
+  }
+
+  /**
+   * Type alias.
+   */
+  private static class NodeUpdateInfo extends LinkedHashMap<String, Object> {
+  }
+}
diff --git 
a/core/src/main/java/org/apache/calcite/plan/visualizer/RuleMatchVisualizer.java
 
b/core/src/main/java/org/apache/calcite/plan/visualizer/RuleMatchVisualizer.java
new file mode 100644
index 0000000..37d88a2
--- /dev/null
+++ 
b/core/src/main/java/org/apache/calcite/plan/visualizer/RuleMatchVisualizer.java
@@ -0,0 +1,487 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.visualizer;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptListener;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+
+import org.apache.commons.io.IOUtils;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.util.DefaultPrettyPrinter;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Charsets;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.text.DecimalFormat;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * This is a tool to visualize the rule match process of a RelOptPlanner.
+ *
+ * <pre>{@code
+ * // create the visualizer
+ * RuleMatchVisualizer viz = new RuleMatchVisualizer("/path/to/output/dir", 
"file-name-suffix");
+ * viz.attachTo(planner)
+ *
+ * planner.findBestExpr();
+ *
+ * // extra step for HepPlanner: write the output to files
+ * // a VolcanoPlanner will call it automatically
+ * viz.writeToFile();
+ * }</pre>
+ */
+public class RuleMatchVisualizer implements RelOptListener {
+
+  private static final String INITIAL = "INITIAL";
+  private static final String FINAL = "FINAL";
+  public static final String DEFAULT_SET = "default";
+
+  // default HTML template can be edited at
+  // 
core/src/main/resources/org/apache/calcite/plan/visualizer/viz-template.html
+  private final String templateDirectory = 
"org/apache/calcite/plan/visualizer";
+  private final @Nullable String outputDirectory;
+  private final @Nullable String outputSuffix;
+
+  private String latestRuleID = "";
+  private int latestRuleTransformCount = 1;
+  private boolean initialized = false;
+
+  private @Nullable RelOptPlanner planner = null;
+
+  private boolean includeTransitiveEdges = false;
+  private boolean includeIntermediateCosts = false;
+
+  private final List<StepInfo> steps = new ArrayList<>();
+  private final Map<String, NodeUpdateHelper> allNodes = new LinkedHashMap<>();
+
+  /**
+   * Use this constructor to save the result on disk at the end of the 
planning phase.
+   * <p>
+   * Note: when using HepPlanner, {@link #writeToFile()} needs to be called 
manually.
+   * </p>
+   */
+  public RuleMatchVisualizer(
+      String outputDirectory,
+      String outputSuffix) {
+    this.outputDirectory = Objects.requireNonNull(outputDirectory, 
"outputDirectory");
+    this.outputSuffix = Objects.requireNonNull(outputSuffix, "outputSuffix");
+  }
+
+  /**
+   * Use this constructor when the result shall not be written to disk.
+   */
+  public RuleMatchVisualizer() {
+    this.outputDirectory = null;
+    this.outputSuffix = null;
+  }
+
+  /**
+   * Attaches the visualizer to the planner.
+   * Must be called before applying the rules.
+   * Must be called exactly once.
+   */
+  public void attachTo(RelOptPlanner planner) {
+    assert this.planner == null;
+    planner.addListener(this);
+    this.planner = planner;
+  }
+
+  /**
+   * Output edges from a subset to the nodes of all subsets that satisfy it.
+   */
+  public void setIncludeTransitiveEdges(final boolean includeTransitiveEdges) {
+    this.includeTransitiveEdges = includeTransitiveEdges;
+  }
+
+  /**
+   * Output intermediate costs, including all cost updates.
+   */
+  public void setIncludeIntermediateCosts(final boolean 
includeIntermediateCosts) {
+    this.includeIntermediateCosts = includeIntermediateCosts;
+  }
+
+  @Override public void ruleAttempted(RuleAttemptedEvent event) {
+    // HepPlanner compatibility
+    if (!initialized) {
+      assert planner != null;
+      RelNode root = planner.getRoot();
+      assert root != null;
+      initialized = true;
+      updateInitialPlan(root);
+    }
+  }
+
+  /**
+   * Register initial plan.
+   * (Workaround for HepPlanner)
+   */
+  private void updateInitialPlan(RelNode node) {
+    if (node instanceof HepRelVertex) {
+      HepRelVertex v = (HepRelVertex) node;
+      updateInitialPlan(v.getCurrentRel());
+      return;
+    }
+    this.registerRelNode(node);
+    for (RelNode input : getInputs(node)) {
+      updateInitialPlan(input);
+    }
+  }
+
+  /**
+   * Get the inputs for a node, unwrapping {@link HepRelVertex} nodes.
+   * (Workaround for HepPlanner)
+   */
+  private Collection<RelNode> getInputs(final RelNode node) {
+    return node.getInputs().stream().map(n -> {
+      if (n instanceof HepRelVertex) {
+        return ((HepRelVertex) n).getCurrentRel();
+      }
+      return n;
+    }).collect(Collectors.toList());
+  }
+
+  @Override public void relChosen(RelChosenEvent event) {
+    if (event.getRel() == null) {
+      assert this.planner != null;
+      RelNode root = this.planner.getRoot();
+      assert root != null;
+      updateFinalPlan(root);
+      this.addStep(FINAL, null);
+      this.writeToFile();
+    }
+  }
+
+  /**
+   * Mark nodes that are part of the final plan.
+   */
+  private void updateFinalPlan(RelNode node) {
+    int size = this.steps.size();
+    if (size > 0 && FINAL.equals(this.steps.get(size - 1).getId())) {
+      return;
+    }
+
+    this.registerRelNode(node).updateAttribute("inFinalPlan", Boolean.TRUE);
+    if (node instanceof RelSubset) {
+      RelNode best = ((RelSubset) node).getBest();
+      if (best == null) {
+        return;
+      }
+      updateFinalPlan(best);
+    } else {
+      for (RelNode input : getInputs(node)) {
+        updateFinalPlan(input);
+      }
+    }
+  }
+
+  @Override public void ruleProductionSucceeded(RuleProductionEvent event) {
+    // method is called once before ruleMatch, and once after ruleMatch
+    if (event.isBefore()) {
+      // add the initialState
+      if (latestRuleID.isEmpty()) {
+        this.addStep(INITIAL, null);
+        this.latestRuleID = INITIAL;
+      }
+      return;
+    }
+
+    // we add the state after the rule is applied
+    RelOptRuleCall ruleCall = event.getRuleCall();
+    String ruleID = Integer.toString(ruleCall.id);
+    String displayRuleName = ruleCall.id + "-" + ruleCall.getRule();
+
+    // a rule might call transform to multiple times, handle it by modifying 
the rule name
+    if (ruleID.equals(this.latestRuleID)) {
+      latestRuleTransformCount++;
+      displayRuleName += "-" + latestRuleTransformCount;
+    } else {
+      latestRuleTransformCount = 1;
+    }
+    this.latestRuleID = ruleID;
+
+    this.addStep(displayRuleName, ruleCall);
+  }
+
+  @Override public void relDiscarded(RelDiscardedEvent event) {
+  }
+
+  @Override public void relEquivalenceFound(RelEquivalenceEvent event) {
+    RelNode rel = event.getRel();
+    assert rel != null;
+    Object eqClass = event.getEquivalenceClass();
+    if (eqClass instanceof String) {
+      String eqClassStr = (String) eqClass;
+      eqClassStr = eqClassStr.replace("equivalence class ", "");
+      String setId = "set-" + eqClassStr;
+      registerSet(setId);
+      registerRelNode(rel).updateAttribute("set", setId);
+    }
+    // register node
+    this.registerRelNode(rel);
+  }
+
+  /**
+   * Add a set.
+   */
+  private void registerSet(final String setID) {
+    this.allNodes.computeIfAbsent(setID, k -> {
+      NodeUpdateHelper h = new NodeUpdateHelper(setID, null);
+      h.updateAttribute("label", DEFAULT_SET.equals(setID) ? "" : setID);
+      h.updateAttribute("kind", "set");
+      return h;
+    });
+  }
+
+  /**
+   * Add a RelNode to track its changes.
+   */
+  private NodeUpdateHelper registerRelNode(final RelNode rel) {
+    return this.allNodes.computeIfAbsent(key(rel), k -> {
+      NodeUpdateHelper h = new NodeUpdateHelper(key(rel), rel);
+      // attributes that need to be set only once
+      h.updateAttribute("label", getNodeLabel(rel));
+      h.updateAttribute("explanation", getNodeExplanation(rel));
+      h.updateAttribute("set", DEFAULT_SET);
+
+      if (rel instanceof RelSubset) {
+        h.updateAttribute("kind", "subset");
+      }
+      return h;
+    });
+  }
+
+  /**
+   * Check and store the changes of the rel node.
+   */
+  private void updateNodeInfo(final RelNode rel, final boolean isLastStep) {
+    NodeUpdateHelper helper = registerRelNode(rel);
+    if (this.includeIntermediateCosts || isLastStep) {
+      RelOptPlanner planner = this.planner;
+      assert planner != null;
+      RelMetadataQuery mq = rel.getCluster().getMetadataQuery();
+      RelOptCost cost = planner.getCost(rel, mq);
+      Double rowCount = mq.getRowCount(rel);
+      helper.updateAttribute("cost", formatCost(rowCount, cost));
+    }
+
+    List<String> inputs = new ArrayList<>();
+    if (rel instanceof RelSubset) {
+      RelSubset relSubset = (RelSubset) rel;
+      relSubset.getRels().forEach(input -> inputs.add(key(input)));
+      Set<String> transitive = new HashSet<>();
+      relSubset.getSubsetsSatisfyingThis()
+          .filter(other -> !other.equals(relSubset))
+          .forEach(input -> {
+            inputs.add(key(input));
+            if (!includeTransitiveEdges) {
+              input.getRels().forEach(r -> transitive.add(key(r)));
+            }
+          });
+      inputs.removeAll(transitive);
+    } else {
+      getInputs(rel).forEach(input -> inputs.add(key(input)));
+    }
+
+    helper.updateAttribute("inputs", inputs);
+  }
+
+  /**
+   * Add the updates since the last step to {@link #steps}.
+   */
+  private void addStep(String stepID, @Nullable RelOptRuleCall ruleCall) {
+    Map<String, Object> nextNodeUpdates = new LinkedHashMap<>();
+
+    // HepPlanner compatibility
+    boolean usesDefaultSet = this.allNodes.values()
+        .stream()
+        .anyMatch(h -> DEFAULT_SET.equals(h.getValue("set")));
+    if (usesDefaultSet) {
+      this.registerSet(DEFAULT_SET);
+    }
+
+    for (NodeUpdateHelper h : allNodes.values()) {
+      RelNode rel = h.getRel();
+      if (rel != null) {
+        updateNodeInfo(rel, FINAL.equals(stepID));
+      }
+      if (h.isEmptyUpdate()) {
+        continue;
+      }
+      Object update = h.getAndResetUpdate();
+      if (update != null) {
+        nextNodeUpdates.put(h.getKey(), update);
+      }
+    }
+
+    List<String> matchedRels = ruleCall == null
+        ? Collections.emptyList()
+        : 
Arrays.stream(ruleCall.rels).map(this::key).collect(Collectors.toList());
+    this.steps.add(new StepInfo(stepID, nextNodeUpdates, matchedRels));
+  }
+
+  public String getJsonStringResult() {
+    try {
+      LinkedHashMap<String, Object> data = new LinkedHashMap<>();
+      data.put("steps", steps);
+      ObjectMapper objectMapper = new ObjectMapper();
+      DefaultPrettyPrinter printer = new DefaultPrettyPrinter();
+      printer = printer.withoutSpacesInObjectEntries();
+      return objectMapper.writer(printer).writeValueAsString(data);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Writes the HTML and JS files of the rule match visualization.
+   * <p>
+   * The old files with the same name will be replaced.
+   */
+  public void writeToFile() {
+    if (outputDirectory == null || outputSuffix == null) {
+      return;
+    }
+
+    try {
+      String templatePath = 
Paths.get(templateDirectory).resolve("viz-template.html").toString();
+      ClassLoader cl = getClass().getClassLoader();
+      assert cl != null;
+      InputStream resourceAsStream = cl.getResourceAsStream(templatePath);
+      assert resourceAsStream != null;
+      String htmlTemplate = IOUtils.toString(resourceAsStream, 
StandardCharsets.UTF_8);
+
+      String htmlFileName = "planner-viz" + outputSuffix + ".html";
+      String dataFileName = "planner-viz-data" + outputSuffix + ".js";
+
+      String replaceString = "src=\"planner-viz-data.js\"";
+      int replaceIndex = htmlTemplate.indexOf(replaceString);
+      String htmlContent = htmlTemplate.substring(0, replaceIndex)
+          + "src=\"" + dataFileName + "\""
+          + htmlTemplate.substring(replaceIndex + replaceString.length());
+
+      String dataJsContent = "var data = " + getJsonStringResult() + ";\n";
+
+      Path outputDirPath = Paths.get(outputDirectory);
+      Path htmlOutput = outputDirPath.resolve(htmlFileName);
+      Path dataOutput = outputDirPath.resolve(dataFileName);
+
+      if (!Files.exists(outputDirPath)) {
+        Files.createDirectories(outputDirPath);
+      }
+
+      Files.write(htmlOutput, htmlContent.getBytes(Charsets.UTF_8), 
StandardOpenOption.CREATE,
+          StandardOpenOption.TRUNCATE_EXISTING);
+      Files.write(dataOutput, dataJsContent.getBytes(Charsets.UTF_8), 
StandardOpenOption.CREATE,
+          StandardOpenOption.TRUNCATE_EXISTING);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  
//--------------------------------------------------------------------------------
+  // methods related to string representation
+  
//--------------------------------------------------------------------------------
+
+  private String key(final RelNode rel) {
+    return "" + rel.getId();
+  }
+
+  private String getNodeLabel(final RelNode relNode) {
+    if (relNode instanceof RelSubset) {
+      final RelSubset relSubset = (RelSubset) relNode;
+      String setId = getSetId(relSubset);
+      return "subset#" + relSubset.getId() + "-set" + setId + "-\n"
+          + relSubset.getTraitSet();
+    }
+
+    return "#" + relNode.getId() + "-" + relNode.getRelTypeName();
+  }
+
+  private String getSetId(final RelSubset relSubset) {
+    String explanation = getNodeExplanation(relSubset);
+    int start = explanation.indexOf("RelSubset") + "RelSubset".length();
+    if (start < 0) {
+      return "";
+    }
+    int end = explanation.indexOf(".", start);
+    if (end < 0) {
+      return "";
+    }
+    return explanation.substring(start, end);
+  }
+
+  private String getNodeExplanation(final RelNode relNode) {
+    InputExcludedRelWriter relWriter = new InputExcludedRelWriter();
+    relNode.explain(relWriter);
+    return relWriter.toString();
+  }
+
+  private static String formatCost(Double rowCount, @Nullable RelOptCost cost) 
{
+    if (cost == null) {
+      return "null";
+    }
+    String originalStr = cost.toString();
+    if (originalStr.contains("inf") || originalStr.contains("huge")
+        || originalStr.contains("tiny")) {
+      return originalStr;
+    }
+    return new MessageFormat("\nrowCount: {0}\nrows: {1}\ncpu:  {2}\nio:   
{3}",
+        Locale.ROOT).format(new String[]{
+            formatCostScientific(rowCount),
+            formatCostScientific(cost.getRows()),
+            formatCostScientific(cost.getCpu()),
+            formatCostScientific(cost.getIo())
+        }
+    );
+  }
+
+  private static String formatCostScientific(double costNumber) {
+    long costRounded = Math.round(costNumber);
+    DecimalFormat formatter = (DecimalFormat) 
DecimalFormat.getInstance(Locale.ROOT);
+    
formatter.applyPattern("#.#############################################E0");
+    return formatter.format(costRounded);
+  }
+
+}
diff --git 
a/core/src/main/java/org/apache/calcite/plan/visualizer/StepInfo.java 
b/core/src/main/java/org/apache/calcite/plan/visualizer/StepInfo.java
new file mode 100644
index 0000000..ce91058
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/plan/visualizer/StepInfo.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.plan.visualizer;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A step in the visualizer represents one rule call of the planner.
+ */
+class StepInfo {
+  private final String id;
+  private final Map<String, Object> updates;
+  private final List<String> matchedRels;
+
+  StepInfo(final String id,
+      final Map<String, Object> updates,
+      final List<String> matchedRels) {
+    this.id = id;
+    this.updates = Collections.unmodifiableMap(updates);
+    this.matchedRels = Collections.unmodifiableList(matchedRels);
+  }
+
+  public String getId() {
+    return id;
+  }
+
+  public Map<String, Object> getUpdates() {
+    return updates;
+  }
+
+  public List<String> getMatchedRels() {
+    return matchedRels;
+  }
+}
diff --git 
a/core/src/main/java/org/apache/calcite/plan/visualizer/package-info.java 
b/core/src/main/java/org/apache/calcite/plan/visualizer/package-info.java
new file mode 100644
index 0000000..4f23efb
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/plan/visualizer/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * A visualizer showing how the rules are applied step-by-step.
+ *
+ * @see org.apache.calcite.plan.visualizer.RuleMatchVisualizer
+ */
+package org.apache.calcite.plan.visualizer;
diff --git 
a/core/src/main/resources/org/apache/calcite/plan/visualizer/viz-template.html 
b/core/src/main/resources/org/apache/calcite/plan/visualizer/viz-template.html
new file mode 100644
index 0000000..dfaf6ae
--- /dev/null
+++ 
b/core/src/main/resources/org/apache/calcite/plan/visualizer/viz-template.html
@@ -0,0 +1,421 @@
+<!doctype html>
+<html lang="en">
+<!--
+{% comment %}
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+{% endcomment %}
+-->
+<meta charset="utf-8">
+<title>Calcite Rule Match Visualization</title>
+
+<script src="https://unpkg.com/d3@7.2.1/dist/d3.min.js"; 
charset="utf-8"></script>
+<script src="https://unpkg.com/dagre-d3@0.6.4/dist/dagre-d3.min.js";></script>
+<script src="https://unpkg.com/tippy.js@3/dist/tippy.all.min.js";></script>
+<script src="planner-viz-data.js"></script>
+
+<style id="css">
+    body {
+        height: 100vh;
+        width: 100vw;
+        margin: 0 0;
+        color: #333;
+        font-weight: 300;
+        font-family: "Helvetica Neue", Helvetica, Arial, sans-serf;
+    }
+
+    li a {
+        display: block;
+        padding: 5px 20px;
+    }
+
+    section {
+        margin-bottom: 3em;
+    }
+
+    section p {
+        text-align: justify;
+    }
+
+    svg {
+        overflow: hidden;
+        margin: 0 auto;
+    }
+
+    pre {
+        border: 1px solid #ccc;
+    }
+
+    #step-list-column {
+        border-left: 1px solid #ccc;
+    }
+
+    .clusters rect {
+        fill: #FFFFE0;
+        stroke: #999;
+        stroke-width: 1.5px;
+    }
+
+    text {
+        font-weight: 300;
+        font-family: "Helvetica Neue", Helvetica, Arial, sans-serf;
+        font-size: 2em;
+    }
+
+    .node rect {
+        stroke: #999;
+        fill: #fff;
+        stroke-width: 1.5px;
+    }
+
+    .edgePath path {
+        stroke: #333;
+        stroke-width: 2px;
+    }
+
+    .container {
+        overflow: hidden;
+        width: 100%;
+        height: 100%;
+        display: flex;
+        flex-flow: row nowrap;
+    }
+
+    .column1 {
+        display: flex;
+        flex-flow: column nowrap;
+    }
+
+    .column2 {
+        flex: 0 1 100%;
+    }
+
+    .tippy-content {
+        word-break: break-all;
+        word-wrap: break-word;
+    }
+
+    button {
+        padding: 0.1em;
+        display: inline-block;
+        font-size: 2em;
+        min-width: 1.5em;
+    }
+</style>
+
+<div class="container">
+    <div class="column2">
+        <div id="toolbar">
+            <button id="left-button" style="display:inline-block">⤿</button>
+            <button id="right-button" style="display:inline-block">⤾</button>
+            <button id="fit-content-button" 
style="display:inline-block">⇿</button>
+            <span>&nbsp;</span>
+            <button id="toggle-list-button">≡</button>
+            <button id="prev-button" disabled>◀</button>
+            <button id="next-button" disabled>▶</button>
+            <div id="current-step" style="display: inline-block"></div>
+        </div>
+        <svg id="svg-canvas" width="100%" height="100%" ></svg>
+    </div>
+    <div id="step-list-column" class="column1">
+        <div style="width: 100%; text-align: center">
+        </div>
+        <ol id="step-list" style="overflow: auto" start="0">
+        </ul>
+    </div>
+</div>
+
+<script id="js">
+
+    var rankDirs = ["BT", "RL", "TB", "LR"];
+
+    /*
+     * Graph data and D3 JS render related variables
+     */
+
+    // Create the input graph
+    var g = new dagreD3.graphlib.Graph({
+            compound: true
+        })
+        .setGraph({
+            rankdir: 'LR',
+        })
+        .setDefaultEdgeLabel(function () {
+            return {};
+        });
+
+    // Create the renderer
+    var render = new dagreD3.render();
+
+    // Set up an SVG group so that we can translate the final graph.
+    var svg = d3.select("svg");
+    var svgGroup = svg.append("g");
+
+    // Set up zoom support
+    const zoom = d3.zoom().on('zoom', (e) => svgGroup.attr('transform', 
e.transform));
+    var svg = d3.select('svg')
+        .call(zoom);
+
+    var fitContent = () => {
+        const { x, y, width, height } = svgGroup.node().getBBox();
+        const { clientWidth, clientHeight } = svg.node();
+        if (width && height) {
+            const scale = Math.min(clientWidth / width, clientHeight / height) 
* 0.98
+            zoom.scaleTo(svg, scale)
+            zoom.translateTo(svg, width/2+x , height/2+y )
+        }
+    };
+
+    /*
+     * Global State
+     */
+
+    var currentStepIndex = 0;
+    var currentRankDirIdx = 0;
+
+    /*
+     * Event Handler functions
+     */
+
+    var updateLocation = () => {
+        var urlParams = new URLSearchParams(location.search); 
+        urlParams.set("step", currentStepIndex);
+        urlParams.set("dir", currentRankDirIdx);
+        window.history.pushState({}, "", "?" + urlParams.toString());
+    };
+
+    var parseLocation = () => {
+        var urlParams = new URLSearchParams(location.search); 
+        if (urlParams.has("step")) {
+            var stepIdx = Number(urlParams.get("step"))
+            if (Number.isInteger(stepIdx))
+                currentStepIndex = stepIdx;
+        }
+        if (urlParams.has("dir")) {
+            var dirIdx = Number(urlParams.get("dir"))
+            if (Number.isInteger(dirIdx) && dirIdx >= 0 && dirIdx < 
rankDirs.length)
+                currentRankDirIdx = dirIdx;
+        }
+    };
+
+    var setCurrentStep = (stepIndex) => {
+        // un-highlight previous entry
+        var prevStepIndex = currentStepIndex;
+        if (prevStepIndex !== undefined) {
+            var prevStepElement = 
document.getElementById(data.steps[prevStepIndex]["id"]);
+            prevStepElement.style.backgroundColor = "#FFFFFF";
+        }
+
+        currentStepIndex = stepIndex;
+        var currentStepID = data.steps[stepIndex]["id"];
+        document.getElementById('current-step').innerText = currentStepIndex + 
": " + currentStepID;
+
+        var currentStepElement = document.getElementById(currentStepID);
+        currentStepElement.style.backgroundColor = "#D3D3D3";
+
+        document.getElementById("prev-button").disabled = false;
+        document.getElementById("next-button").disabled = false;
+
+        if (currentStepIndex === 0) {
+            document.getElementById("prev-button").disabled = true;
+        }
+        if (currentStepIndex === data.steps.length - 1) {
+            document.getElementById("next-button").disabled = true;
+        }
+
+        updateGraph();
+    }
+
+    var getCurrentState = () => {
+        var nodes = {};
+        for(var i=0; i<=currentStepIndex; i++) {
+            // recreate state by merging all updates
+            var updates = data.steps[i]["updates"];
+            Object.entries(updates).forEach(e => {
+                const [key, value] = e;
+                var nodeInfo = nodes[key] ??= {
+                    id: key,
+                    addedInStep: i,
+                };
+                Object.assign(nodeInfo, value);
+            });
+        }
+        //var newNodes = Object.values(nodes).filter(n => n.addedInStep === 
currentStepIndex);
+        var matchedRels = data.steps[currentStepIndex]["matchedRels"] ?? [];
+        return { nodes, matchedRels };
+    };
+
+    var updateGraph = () => {
+        updateLocation();
+
+        var state = getCurrentState();
+        var stepID = data.steps[currentStepIndex]["id"];
+
+        // remove previous rendered view and clear graph model
+        d3.select("svg g").selectAll("*").remove();
+        g.nodes().slice().forEach(nodeID => g.removeNode(nodeID));
+
+        for(var n of Object.values(state.nodes)) {
+            var nodeID = n.id;
+            if(n.kind === "set") {
+                // add set
+                var setLabel = n.Label;
+                if (setLabel === null || setLabel === undefined) {
+                    setLabel = nodeID;
+                }
+                g.setNode(nodeID, {
+                    label: setLabel,
+                    clusterLabelPos: 'top'
+                });
+            }
+            else {
+                var nodeLabel;
+                if (stepID === "FINAL") {
+                    nodeLabel = n.label + "\n" + n.cost;
+                } else {
+                    nodeLabel = n.label;
+                }
+                var nodeStyle;
+                if (stepID === "FINAL" && n.inFinalPlan === true) {
+                    if(n.kind === "subset")
+                        nodeStyle = "fill: #E0FFFF";
+                    else
+                        nodeStyle = "fill: #C8C8F3";
+                }
+                else if (stepID !== "INITIAL" && n.addedInStep == 
currentStepIndex) {
+                    nodeStyle = "fill: #E0FFFF";
+                } else if (state.matchedRels.includes(nodeID)) {
+                    nodeStyle = "fill: #C8C8F3";
+                } else {
+                    nodeStyle = "fill: #FFFFFF";
+                }
+                g.setNode(nodeID, {
+                    label: nodeLabel,
+                    style: nodeStyle
+                });
+                // node-set parent relationship
+                g.setParent(nodeID, n.set);
+
+                // create links
+                if(n.inputs) 
+                for(var inputID of n.inputs) {
+                    var input = state.nodes[inputID];
+                    var edgeOptions = { arrowheadStyle: "normal" };
+                    if (n.kind === "subset" && input.kind === "subset") {
+                        edgeOptions = { style: "stroke-dasharray: 5, 5; fill: 
none;" };
+                    }
+                    g.setEdge(inputID, nodeID, edgeOptions);
+                }
+            }
+        }
+
+        g.setGraph({
+            rankdir: rankDirs[currentRankDirIdx]
+        })
+
+        // re-render
+        render(d3.select("svg g"), g);
+
+        // register tooltip popup
+        const allD3Nodes = d3.select('svg').selectAll('.node');
+        const allD3NodeElements = allD3Nodes.nodes();
+
+        tippy.setDefaults({
+            trigger: "click",
+            interactive: true,
+        });
+
+        var i = 0;
+        allD3Nodes.each(nodeID => {
+            var nodeElement = allD3NodeElements[i];
+            var node = state.nodes[nodeID];
+            var stepName = data.steps[node.addedInStep].id;
+            var popupContent;
+            if(node.kind === "subset")
+                popupContent = "inputs: " + node.inputs;
+            else
+                popupContent = node.explanation;
+            popupContent += "<br>Added in Step '"  + stepName + "'";
+
+            tippy(nodeElement, { content: popupContent })
+            i++;
+        });
+    }
+
+    /*
+     * render HTML Element and add event hanlders
+     */
+
+    // populate UI list
+    var stepListElement = document.getElementById("step-list");
+    data.steps.forEach((step, index) => {
+        var stepID = step["id"];
+        var listItem = document.createElement("li");
+        var textItem = document.createElement("a");
+        textItem.innerText = stepID;
+        textItem.id = stepID;
+        textItem.setAttribute("href", "#");
+
+        listItem.appendChild(textItem);
+        stepListElement.appendChild(listItem);
+        listItem.addEventListener("click", event => {
+            setCurrentStep(index);
+        })
+    })
+
+    document.getElementById("prev-button").addEventListener("click", event => {
+        if (currentStepIndex !== 0) {
+            setCurrentStep(currentStepIndex - 1);
+        }
+    });
+
+    document.getElementById("next-button").addEventListener("click", event => {
+        if (currentStepIndex !== data.steps.length - 1) {
+            setCurrentStep(currentStepIndex + 1);
+        }
+    });
+
+    document.getElementById("left-button").addEventListener("click", event => {
+        currentRankDirIdx += 1;
+        if (currentRankDirIdx >= rankDirs.length)
+            currentRankDirIdx = 0;
+        updateGraph();
+    });
+
+    document.getElementById("right-button").addEventListener("click", event => 
{
+        currentRankDirIdx -= 1;
+        if (currentRankDirIdx < 0)
+            currentRankDirIdx = 3;
+        updateGraph();
+    });
+
+    document.getElementById("fit-content-button").addEventListener("click", 
fitContent);
+
+    document.getElementById("toggle-list-button").addEventListener("click", () 
=> {
+        var col1 = document.getElementById("step-list-column");
+        if (col1.style.display === "none") 
+            col1.style.display = "";
+        else 
+            col1.style.display = "none";
+    });
+
+    // render initial state
+    
+    parseLocation();
+    setCurrentStep(currentStepIndex);
+    fitContent();
+</script>
+
+</html>
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java 
b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
index f94242a..e599bc7 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
@@ -221,8 +221,8 @@ abstract class RelOptTestBase extends SqlToRelTestBase {
       return withPre(builder.build());
     }
 
-    public Sql with(HepPlanner hepPlanner) {
-      return new Sql(tester, sql, preProgram, hepPlanner, hooks, transforms, 
relFn);
+    public Sql with(RelOptPlanner planner) {
+      return new Sql(tester, sql, preProgram, planner, hooks, transforms, 
relFn);
     }
 
     public Sql with(HepProgram program) {
diff --git 
a/core/src/test/java/org/apache/calcite/test/RuleMatchVisualizerTest.java 
b/core/src/test/java/org/apache/calcite/test/RuleMatchVisualizerTest.java
new file mode 100644
index 0000000..52445db
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/test/RuleMatchVisualizerTest.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test;
+
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.hep.HepPlanner;
+import org.apache.calcite.plan.hep.HepProgram;
+import org.apache.calcite.plan.visualizer.RuleMatchVisualizer;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.rules.CoreRules;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Check the output of {@link RuleMatchVisualizer}.
+ */
+public class RuleMatchVisualizerTest extends RelOptTestBase {
+
+  protected DiffRepository getDiffRepos() {
+    return DiffRepository.lookup(RuleMatchVisualizerTest.class);
+  }
+
+  @Test void testHepPlanner() {
+    final String sql = "select a.name from dept a\n"
+        + "union all\n"
+        + "select b.name from dept b\n"
+        + "order by name limit 10";
+
+    final HepProgram program = HepProgram.builder()
+        .addRuleInstance(CoreRules.PROJECT_SET_OP_TRANSPOSE)
+        .addRuleInstance(CoreRules.SORT_UNION_TRANSPOSE)
+        .build();
+    HepPlanner planner = new HepPlanner(program);
+
+    RuleMatchVisualizer viz = new RuleMatchVisualizer();
+    viz.attachTo(planner);
+
+    sql(sql).with(planner).check();
+
+    String result = normalize(viz.getJsonStringResult());
+    getDiffRepos().assertEquals("visualizer", "${visualizer}", result);
+  }
+
+  @Test void testVolcanoPlanner() {
+    final String sql = "select a.name from dept a";
+
+    VolcanoPlanner planner = new VolcanoPlanner();
+    planner.setTopDownOpt(false);
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+    planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+
+    RelOptUtil.registerDefaultRules(planner, false, false);
+
+    RuleMatchVisualizer viz = new RuleMatchVisualizer();
+    viz.attachTo(planner);
+
+
+    sql(sql)
+        .with(planner)
+        .withTester(
+            t -> t.withClusterFactory(
+              cluster -> RelOptCluster.create(planner, 
cluster.getRexBuilder())))
+        .check();
+
+    String result = normalize(viz.getJsonStringResult());
+    getDiffRepos().assertEquals("visualizer", "${visualizer}", result);
+  }
+
+  /**
+   * Normalize the visualizer output, so that it is independent of other tests.
+   */
+  private String normalize(String str) {
+    // rename rel ids
+    str = renameMatches(
+        str, Pattern.compile("\"([0-9]+)\"|"
+        + "\"label\" *: *\"#([0-9]+)-|"
+        + "\"label\" *: *\"subset#([0-9]+)-|"
+        + "\"explanation\" *: *\"\\{subset=rel#([0-9]+):"), 1000);
+    // rename rule call ids
+    str = renameMatches(str, Pattern.compile("\"id\" *: *\"([0-9]+)-"), 100);
+    return str;
+  }
+
+  /**
+   * Rename the first group of each match to a consecutive index, starting at 
the offset.
+   */
+  private String renameMatches(final String str,
+      final Pattern pattern, int offset) {
+    Map<String, String> rename = new HashMap<>();
+    StringBuilder sb = new StringBuilder();
+    Matcher m = pattern.matcher(str);
+
+    int last = 0;
+    while (m.find()) {
+      int start = -1;
+      int end = -1;
+      String oldName = null;
+      for (int i = 1; i <= m.groupCount(); i++) {
+        if (m.group(i) != null) {
+          oldName = m.group(i);
+          start = m.start(i);
+          end = m.end(i);
+          break;
+        }
+      }
+      assert oldName != null;
+      String newName = rename.computeIfAbsent(oldName, k -> "" + 
(rename.size() + offset));
+      sb.append(str, last, start);
+      sb.append(newName);
+      last = end;
+    }
+    sb.append(str.substring(last));
+    return sb.toString();
+  }
+
+}
diff --git 
a/core/src/test/resources/org/apache/calcite/test/RuleMatchVisualizerTest.xml 
b/core/src/test/resources/org/apache/calcite/test/RuleMatchVisualizerTest.xml
new file mode 100644
index 0000000..7f07d0b
--- /dev/null
+++ 
b/core/src/test/resources/org/apache/calcite/test/RuleMatchVisualizerTest.xml
@@ -0,0 +1,264 @@
+<?xml version="1.0" ?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to you under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~ http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+<Root>
+  <TestCase name="testHepPlanner">
+    <Resource name="sql">
+      <![CDATA[select a.name from dept a
+union all
+select b.name from dept b
+order by name limit 10]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSort(sort0=[$0], dir0=[ASC], fetch=[10])
+  LogicalProject(NAME=[$0])
+    LogicalUnion(all=[true])
+      LogicalProject(NAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+      LogicalProject(NAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalSort(sort0=[$0], dir0=[ASC], fetch=[10])
+  LogicalUnion(all=[true])
+    LogicalSort(sort0=[$0], dir0=[ASC], fetch=[10])
+      LogicalProject(NAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+    LogicalSort(sort0=[$0], dir0=[ASC], fetch=[10])
+      LogicalProject(NAME=[$1])
+        LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+    </Resource>
+    <Resource name="visualizer">
+      <![CDATA[{
+  "steps":[ {
+    "id":"INITIAL",
+    "updates":{
+      "1000":{
+        "label":"#1000-LogicalTableScan",
+        "explanation":"{table=[CATALOG, SALES, DEPT]}",
+        "set":"default"
+      },
+      "1001":{
+        "label":"#1001-LogicalProject",
+        "explanation":"{NAME=$1}",
+        "set":"default",
+        "inputs":[ "1000" ]
+      },
+      "1002":{
+        "label":"#1002-LogicalUnion",
+        "explanation":"{all=true}",
+        "set":"default",
+        "inputs":[ "1001", "1001" ]
+      },
+      "1003":{
+        "label":"#1003-LogicalProject",
+        "explanation":"{NAME=$0}",
+        "set":"default",
+        "inputs":[ "1002" ]
+      },
+      "1004":{
+        "label":"#1004-LogicalSort",
+        "explanation":"{sort0=$0, dir0=ASC, fetch=10}",
+        "set":"default",
+        "inputs":[ "1003" ]
+      },
+      "default":{
+        "label":"",
+        "kind":"set"
+      }
+    },
+    "matchedRels":[ ]
+  }, {
+    "id":"100-ProjectSetOpTransposeRule",
+    "updates":{
+      "1004":{
+        "inputs":[ "1002" ]
+      }
+    },
+    "matchedRels":[ "1003", "1002" ]
+  }, {
+    "id":"101-SortUnionTransposeRule",
+    "updates":{
+      "1005":{
+        "label":"#1005-LogicalSort",
+        "explanation":"{sort0=$0, dir0=ASC, fetch=10}",
+        "set":"default",
+        "inputs":[ "1001" ]
+      },
+      "1006":{
+        "label":"#1006-LogicalUnion",
+        "explanation":"{all=true}",
+        "set":"default",
+        "inputs":[ "1005", "1005" ]
+      },
+      "1007":{
+        "label":"#1007-LogicalSort",
+        "explanation":"{sort0=$0, dir0=ASC, fetch=10}",
+        "set":"default",
+        "inputs":[ "1006" ]
+      }
+    },
+    "matchedRels":[ "1004", "1002" ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testVolcanoPlanner">
+    <Resource name="sql">
+      <![CDATA[select a.name from dept a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(NAME=[$1])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+EnumerableProject(NAME=[$1])
+  EnumerableTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+    </Resource>
+    <Resource name="visualizer">
+      <![CDATA[{
+  "steps":[ {
+    "id":"INITIAL",
+    "updates":{
+      "set-0":{
+        "label":"set-0",
+        "kind":"set"
+      },
+      "1000":{
+        "label":"subset#1000-set#0-\nNONE.[]",
+        "explanation":"{subset=rel#1000:RelSubset#0.NONE.[]}",
+        "set":"set-0",
+        "kind":"subset",
+        "inputs":[ "1001" ]
+      },
+      "1001":{
+        "label":"#1001-LogicalTableScan",
+        "explanation":"{table=[CATALOG, SALES, DEPT]}",
+        "set":"set-0"
+      },
+      "set-1":{
+        "label":"set-1",
+        "kind":"set"
+      },
+      "1002":{
+        "label":"subset#1002-set#1-\nNONE.[]",
+        "explanation":"{subset=rel#1002:RelSubset#1.NONE.[]}",
+        "set":"set-1",
+        "kind":"subset",
+        "inputs":[ "1003" ]
+      },
+      "1003":{
+        "label":"#1003-LogicalProject",
+        "explanation":"{NAME=$1}",
+        "set":"set-1",
+        "inputs":[ "1000" ]
+      },
+      "1004":{
+        "label":"subset#1004-set#1-\nENUMERABLE.[]",
+        "explanation":"{subset=rel#1004:RelSubset#1.ENUMERABLE.[], NAME=$1}",
+        "set":"set-1",
+        "kind":"subset",
+        "inputs":[ "1005" ]
+      },
+      "1005":{
+        "label":"#1005-AbstractConverter",
+        "explanation":"{convention=ENUMERABLE, sort=[]}",
+        "set":"set-1",
+        "inputs":[ "1002" ]
+      }
+    },
+    "matchedRels":[ ]
+  }, {
+    "id":"100-EnumerableTableScanRule(in:NONE,out:ENUMERABLE)",
+    "updates":{
+      "1006":{
+        "label":"subset#1006-set#0-\nENUMERABLE.[]",
+        "explanation":"{subset=rel#1006:RelSubset#0.ENUMERABLE.[], 
table=[CATALOG, SALES, DEPT]}",
+        "set":"set-0",
+        "kind":"subset",
+        "inputs":[ "1007" ]
+      },
+      "1007":{
+        "label":"#1007-EnumerableTableScan",
+        "explanation":"{table=[CATALOG, SALES, DEPT]}",
+        "set":"set-0"
+      }
+    },
+    "matchedRels":[ "1001" ]
+  }, {
+    "id":"101-EnumerableProjectRule(in:NONE,out:ENUMERABLE)",
+    "updates":{
+      "1004":{
+        "inputs":[ "1005", "1008" ]
+      },
+      "1008":{
+        "label":"#1008-EnumerableProject",
+        "explanation":"{NAME=$1}",
+        "set":"set-1",
+        "inputs":[ "1006" ]
+      }
+    },
+    "matchedRels":[ "1003" ]
+  }, {
+    "id":"FINAL",
+    "updates":{
+      "1000":{
+        "cost":"{inf}"
+      },
+      "1001":{
+        "cost":"{inf}"
+      },
+      "1002":{
+        "cost":"{inf}"
+      },
+      "1003":{
+        "cost":"{inf}"
+      },
+      "1004":{
+        "inFinalPlan":true,
+        "cost":"\nrowCount: 4E0\nrows: 8E0\ncpu:  9E0\nio:   0E0"
+      },
+      "1005":{
+        "cost":"{inf}"
+      },
+      "1006":{
+        "inFinalPlan":true,
+        "cost":"\nrowCount: 4E0\nrows: 4E0\ncpu:  5E0\nio:   0E0"
+      },
+      "1007":{
+        "inFinalPlan":true,
+        "cost":"\nrowCount: 4E0\nrows: 4E0\ncpu:  5E0\nio:   0E0"
+      },
+      "1008":{
+        "inFinalPlan":true,
+        "cost":"\nrowCount: 4E0\nrows: 8E0\ncpu:  9E0\nio:   0E0"
+      }
+    },
+    "matchedRels":[ ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+</Root>

Reply via email to