You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by GitBox <gi...@apache.org> on 2021/12/08 12:40:03 UTC

[GitHub] [calcite] zabetak commented on a change in pull request #2552: [CALCITE-4737] Add Volcano Visualizer for Debugging

zabetak commented on a change in pull request #2552:
URL: https://github.com/apache/calcite/pull/2552#discussion_r764816065



##########
File path: core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleMatchVisualizer.java
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.volcano;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.tools.visualizer.InputExcludedRelWriter;
+import org.apache.calcite.tools.visualizer.VisualizerNodeInfo;
+import org.apache.calcite.tools.visualizer.VisualizerRuleMatchInfo;
+import org.apache.calcite.tools.visualizer.VolcanoRuleMatchVisualizerListener;
+
+import org.apache.commons.io.IOUtils;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Charsets;
+
+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.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * This is tool to visualize the rule match process of the VolcanoPlanner.
+ *
+ *
+ * <p>To use the visualizer, add a listener before the VolcanoPlanner optimization phase.
+ * Then writes the output to a file after the optimization ends.
+ *
+ * <pre>
+ * // create the visualizer. This attaches a listener to VolcanoPlanner
+ * VolcanoRuleMatchVisualizer visualizer = VolcanoRuleMatchVisualizer.createAndAttach(planner);
+ *
+ * volcanoPlanner.findBestExpr();
+ *
+ * // writes the output to files
+ * visualizer.writeToFile(outputDirectory, "");
+ * </pre>
+ */
+public class VolcanoRuleMatchVisualizer {
+
+  public static VolcanoRuleMatchVisualizer createAndAttach(VolcanoPlanner volcanoPlanner) {
+    VolcanoRuleMatchVisualizer vis = new VolcanoRuleMatchVisualizer(volcanoPlanner);
+    // add listener outside of the constructor to make checker-framework happy
+    volcanoPlanner.addListener(new VolcanoRuleMatchVisualizerListener(vis));
+    return vis;
+  }
+
+  VolcanoPlanner volcanoPlanner;
+
+  // a sequence of ruleMatch ID to represent the order of rule match
+  List<String> ruleMatchSequence = new ArrayList<>();
+  // map of ruleMatch ID and the info, including the state snapshot at the time of ruleMatch
+  Map<String, VisualizerRuleMatchInfo> ruleInfoMap = new TreeMap<>();
+  // map of nodeID to the ruleID it's first added
+  Map<String, String> nodeAddedInRule = new TreeMap<>();
+
+  // a map of relNode ID to the actual RelNode object
+  // contains all the relNodes appear during the optimization
+  // all RelNode are immutable in Calcite, therefore only new nodes will be added
+  Map<String, RelNode> allNodes = new TreeMap<>();

Review comment:
       Reduce visibility?

##########
File path: core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleMatchVisualizer.java
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.volcano;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.tools.visualizer.InputExcludedRelWriter;
+import org.apache.calcite.tools.visualizer.VisualizerNodeInfo;
+import org.apache.calcite.tools.visualizer.VisualizerRuleMatchInfo;
+import org.apache.calcite.tools.visualizer.VolcanoRuleMatchVisualizerListener;
+
+import org.apache.commons.io.IOUtils;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Charsets;
+
+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.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * This is tool to visualize the rule match process of the VolcanoPlanner.
+ *
+ *
+ * <p>To use the visualizer, add a listener before the VolcanoPlanner optimization phase.
+ * Then writes the output to a file after the optimization ends.
+ *
+ * <pre>
+ * // create the visualizer. This attaches a listener to VolcanoPlanner
+ * VolcanoRuleMatchVisualizer visualizer = VolcanoRuleMatchVisualizer.createAndAttach(planner);
+ *
+ * volcanoPlanner.findBestExpr();
+ *
+ * // writes the output to files
+ * visualizer.writeToFile(outputDirectory, "");
+ * </pre>
+ */
+public class VolcanoRuleMatchVisualizer {
+
+  public static VolcanoRuleMatchVisualizer createAndAttach(VolcanoPlanner volcanoPlanner) {
+    VolcanoRuleMatchVisualizer vis = new VolcanoRuleMatchVisualizer(volcanoPlanner);
+    // add listener outside of the constructor to make checker-framework happy
+    volcanoPlanner.addListener(new VolcanoRuleMatchVisualizerListener(vis));
+    return vis;
+  }
+
+  VolcanoPlanner volcanoPlanner;
+
+  // a sequence of ruleMatch ID to represent the order of rule match
+  List<String> ruleMatchSequence = new ArrayList<>();
+  // map of ruleMatch ID and the info, including the state snapshot at the time of ruleMatch
+  Map<String, VisualizerRuleMatchInfo> ruleInfoMap = new TreeMap<>();
+  // map of nodeID to the ruleID it's first added
+  Map<String, String> nodeAddedInRule = new TreeMap<>();
+
+  // a map of relNode ID to the actual RelNode object
+  // contains all the relNodes appear during the optimization
+  // all RelNode are immutable in Calcite, therefore only new nodes will be added
+  Map<String, RelNode> allNodes = new TreeMap<>();
+
+  private VolcanoRuleMatchVisualizer(VolcanoPlanner volcanoPlanner) {
+    this.volcanoPlanner = volcanoPlanner;
+  }
+
+  public void addRuleMatch(String ruleCallID, Collection<? extends RelNode> matchedRels) {
+
+    // store the current state snapshot
+    // nodes contained in the sets
+    // and inputs of relNodes (and relSubsets)
+    Map<String, String> setLabels = new TreeMap<>();
+    Map<String, String> setOriginalRel = new TreeMap<>();
+    Map<String, Set<String>> nodesInSet = new TreeMap<>();
+    Map<String, Set<String>> nodeInputs = new TreeMap<>();
+
+    // newNodes appeared after this ruleCall
+    Set<String> newNodes = new TreeSet<>();
+
+    // populate current snapshot, and fill in the allNodes map
+    volcanoPlanner.allSets.forEach(set -> {
+      String setID = "set-" + set.id;
+      String setLabel = getSetLabel(set);
+      setLabels.put(setID, setLabel);
+      setOriginalRel.put(setID, set.rel == null ? "" : String.valueOf(set.rel.getId()));
+
+      nodesInSet.put(setID, nodesInSet.getOrDefault(setID, new TreeSet<>()));
+
+      Consumer<RelNode> addNode = rel -> {
+        String nodeID = String.valueOf(rel.getId());
+        nodesInSet.get(setID).add(nodeID);
+
+        if (!allNodes.containsKey(nodeID)) {
+          newNodes.add(nodeID);
+          allNodes.put(nodeID, rel);
+        }
+      };
+
+      Consumer<RelNode> addLink = rel -> {
+        String nodeID = String.valueOf(rel.getId());
+        Set<String> relInputs = nodeInputs.computeIfAbsent(nodeID, k -> new TreeSet<>());
+        if (rel instanceof RelSubset) {
+          RelSubset relSubset = (RelSubset) rel;
+          relSubset.getRels().forEach(input -> relInputs.add(String.valueOf(input.getId())));
+          relSubset.getSubsetsSatisfyingThis()
+              .filter(other -> !other.equals(relSubset))
+              .forEach(input -> relInputs.add(String.valueOf(input.getId())));
+        } else {
+          rel.getInputs().forEach(input -> relInputs.add(String.valueOf(input.getId())));
+        }
+      };
+
+      set.rels.forEach(addNode);
+      set.subsets.forEach(addNode);
+      set.rels.forEach(addLink);
+      set.subsets.forEach(addLink);
+    });
+
+    // get the matched nodes of this rule
+    Set<String> matchedNodeIDs = matchedRels.stream()
+        .map(rel -> String.valueOf(rel.getId()))
+        .collect(Collectors.toCollection(() -> new TreeSet<>()));
+
+    // get importance 0 rels as of right now
+    Set<String> importanceZeroNodes = new TreeSet<>();
+    volcanoPlanner.prunedNodes
+        .forEach(rel -> importanceZeroNodes.add(Integer.toString(rel.getId())));
+
+    VisualizerRuleMatchInfo ruleMatchInfo =
+        new VisualizerRuleMatchInfo(setLabels, setOriginalRel, nodesInSet,
+            nodeInputs, matchedNodeIDs, newNodes, importanceZeroNodes);
+
+    ruleMatchSequence.add(ruleCallID);
+    ruleInfoMap.put(ruleCallID, ruleMatchInfo);
+
+    newNodes.forEach(newNode -> nodeAddedInRule.put(newNode, ruleCallID));
+  }
+
+  /**
+   * Add a final plan to the variable.
+   */
+  private void addFinalPlan() {
+    if (ruleMatchSequence.contains("FINAL")) {
+      return;
+    }
+
+    Set<RelNode> finalPlanNodes = new LinkedHashSet<>();
+    Deque<RelSubset> subsetsToVisit = new ArrayDeque<>();
+    RelSubset root = (RelSubset) volcanoPlanner.getRoot();
+    assert root != null;
+    subsetsToVisit.add(root);
+
+    RelSubset subset;
+    while ((subset = subsetsToVisit.poll()) != null) {
+      // add subset itself to the highlight list
+      finalPlanNodes.add(subset);
+      // highlight its best node if it exists
+      RelNode best = subset.getBest();
+      if (best == null) {
+        continue;
+      }
+      finalPlanNodes.add(best);
+      // recursively visit the input relSubsets of the best node
+      best.getInputs().stream().map(rel -> (RelSubset) rel).forEach(subsetsToVisit::add);
+    }
+
+    this.addRuleMatch("FINAL", new ArrayList<>(finalPlanNodes));
+  }
+
+  private String getSetLabel(RelSet set) {
+    return "set-" + set.id + "    ";
+  }
+
+  private String getJsonStringResult() {
+    try {
+      RelNode root = volcanoPlanner.getRoot();
+      if (root == null) {
+        throw new RuntimeException("volcano planner root is null");
+      }
+      RelMetadataQuery mq = root.getCluster().getMetadataQuery();
+
+      Map<String, VisualizerNodeInfo> nodeInfoMap = new TreeMap<>();
+      for (Map.Entry<String, RelNode> entry : allNodes.entrySet()) {
+        RelNode relNode = entry.getValue();
+        RelOptCost cost = volcanoPlanner.getCost(relNode, mq);
+        Double rowCount = mq.getRowCount(relNode);
+
+        VisualizerNodeInfo nodeInfo;
+        if (relNode instanceof RelSubset) {
+          RelSubset relSubset = (RelSubset) relNode;
+          String nodeLabel = "subset#" + relSubset.getId() + "-set#" + relSubset.set.id + "-\n"
+              + relSubset.getTraitSet().toString();
+          String relIDs = relSubset.getRelList().stream()
+              .map(i -> "#" + i.getId()).collect(joining(", "));
+          String explanation = "rels: [" + relIDs + "]";
+          nodeInfo =
+              new VisualizerNodeInfo(nodeLabel, true, explanation, cost, rowCount);
+        } else {
+          InputExcludedRelWriter relWriter = new InputExcludedRelWriter();
+          relNode.explain(relWriter);
+          String inputIDs = relNode.getInputs().stream()
+              .map(i -> "#" + i.getId()).collect(joining(", "));
+          String explanation = relWriter.toString() + ", inputs: [" + inputIDs + "]";
+
+          String nodeLabel = "#" + relNode.getId() + "-" + relNode.getRelTypeName();
+          nodeInfo = new VisualizerNodeInfo(nodeLabel, false, explanation, cost,
+              rowCount);
+        }
+
+        nodeInfoMap.put(entry.getKey(), nodeInfo);
+      }
+
+      LinkedHashMap<String, Object> data = new LinkedHashMap<>();
+      data.put("allNodes", nodeInfoMap);
+      data.put("ruleMatchSequence", ruleMatchSequence);
+      data.put("ruleMatchInfoMap", ruleInfoMap);
+      data.put("nodeAddedInRule", nodeAddedInRule);
+
+      ObjectMapper objectMapper = new ObjectMapper();
+      return objectMapper.writerWithDefaultPrettyPrinter().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.
+   *
+   * @param outputDirectory directory of the output files
+   * @param suffix          file name suffix, can be null
+   */
+  public void writeToFile(String outputDirectory, String suffix) {
+    // default HTML template is under "resources"
+    writeToFile("volcano-viz", outputDirectory, suffix);
+  }
+
+  public void writeToFile(String templateDirectory, String outputDirectory, String suffix) {

Review comment:
       Instead of forcing the client to keep a reference to the visualizer and call these methods explicitly after optimization can't we do it automatically at the end of the optimization phase (e.g., by relying on `RelOptListener#relChosen`)? 
   If this gets integrated to the planner we should find a trick to avoid replacing the visualizations of previous queries; in other words avoid writing to the same file/directory.

##########
File path: core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleMatchVisualizer.java
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.volcano;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.tools.visualizer.InputExcludedRelWriter;
+import org.apache.calcite.tools.visualizer.VisualizerNodeInfo;
+import org.apache.calcite.tools.visualizer.VisualizerRuleMatchInfo;
+import org.apache.calcite.tools.visualizer.VolcanoRuleMatchVisualizerListener;
+
+import org.apache.commons.io.IOUtils;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Charsets;
+
+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.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * This is tool to visualize the rule match process of the VolcanoPlanner.
+ *
+ *
+ * <p>To use the visualizer, add a listener before the VolcanoPlanner optimization phase.
+ * Then writes the output to a file after the optimization ends.
+ *
+ * <pre>
+ * // create the visualizer. This attaches a listener to VolcanoPlanner
+ * VolcanoRuleMatchVisualizer visualizer = VolcanoRuleMatchVisualizer.createAndAttach(planner);
+ *
+ * volcanoPlanner.findBestExpr();
+ *
+ * // writes the output to files
+ * visualizer.writeToFile(outputDirectory, "");
+ * </pre>
+ */
+public class VolcanoRuleMatchVisualizer {
+
+  public static VolcanoRuleMatchVisualizer createAndAttach(VolcanoPlanner volcanoPlanner) {
+    VolcanoRuleMatchVisualizer vis = new VolcanoRuleMatchVisualizer(volcanoPlanner);
+    // add listener outside of the constructor to make checker-framework happy

Review comment:
       I guess that if we incorporate by default in Calcite the method `createAndAttach` can be removed.

##########
File path: core/src/main/java/org/apache/calcite/tools/visualizer/VolcanoRuleMatchVisualizerListener.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.tools.visualizer;
+
+import org.apache.calcite.plan.RelOptListener;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.plan.volcano.VolcanoRuleMatchVisualizer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+/**
+ * Visualizer Listener listens to events of Volcano Planner.
+ * Whenever a rule is successfully applied, it adds the information to the visualizer.
+ */
+public class VolcanoRuleMatchVisualizerListener implements RelOptListener {
+
+  private final VolcanoRuleMatchVisualizer visualizer;
+  private String latestRuleID = "";
+  private int latestRuleTransformCount = 1;
+
+  public VolcanoRuleMatchVisualizerListener(VolcanoRuleMatchVisualizer visualizer) {
+    this.visualizer = visualizer;
+  }
+
+  /**
+   * After a rule is matched, record the rule and the state after matching.
+   */
+  @Override public void ruleAttempted(RuleAttemptedEvent event) {
+  }
+
+  @Override public void relChosen(RelChosenEvent event) {
+  }
+
+  @Override public void ruleProductionSucceeded(RuleProductionEvent event) {
+    RelOptPlanner planner = event.getRuleCall().getPlanner();
+    if (!(planner instanceof VolcanoPlanner)) {
+      return;
+    }
+
+    // ruleAttempted is called once before ruleMatch, and once after ruleMatch
+    if (event.isBefore()) {
+      // add the initialState
+      if (latestRuleID.isEmpty()) {
+        visualizer.addRuleMatch("INITIAL", new ArrayList<>());
+        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().toString();
+
+    // 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;
+
+    visualizer.addRuleMatch(displayRuleName, Arrays.stream(ruleCall.rels)
+        .collect(Collectors.toList()));
+  }
+
+  @Override public void relDiscarded(RelDiscardedEvent event) {
+  }
+
+  @Override public void relEquivalenceFound(RelEquivalenceEvent event) {
+  }
+
+  public VolcanoRuleMatchVisualizer getVisualizer() {
+    return visualizer;
+  }

Review comment:
       Never called. Remove?

##########
File path: core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleMatchVisualizer.java
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.volcano;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.tools.visualizer.InputExcludedRelWriter;
+import org.apache.calcite.tools.visualizer.VisualizerNodeInfo;
+import org.apache.calcite.tools.visualizer.VisualizerRuleMatchInfo;
+import org.apache.calcite.tools.visualizer.VolcanoRuleMatchVisualizerListener;
+
+import org.apache.commons.io.IOUtils;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Charsets;
+
+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.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * This is tool to visualize the rule match process of the VolcanoPlanner.
+ *
+ *
+ * <p>To use the visualizer, add a listener before the VolcanoPlanner optimization phase.
+ * Then writes the output to a file after the optimization ends.
+ *
+ * <pre>
+ * // create the visualizer. This attaches a listener to VolcanoPlanner
+ * VolcanoRuleMatchVisualizer visualizer = VolcanoRuleMatchVisualizer.createAndAttach(planner);
+ *
+ * volcanoPlanner.findBestExpr();
+ *
+ * // writes the output to files
+ * visualizer.writeToFile(outputDirectory, "");
+ * </pre>
+ */
+public class VolcanoRuleMatchVisualizer {

Review comment:
       I am wondering if we can somehow unify this class with the `VolcanoRuleMatchVisualizerListener`? Actually, the main thing that I would like to avoid if possible is to have state dependent on the `VolcanoPlanner`. I see that we need the `VolcanoPlanner` to obtain some information about sets, subsets, etc., so I am wondering if we could obtain this information from `RelOptListener#relEquivalenceFound` API instead. The goal is to make this a bit more generic and dependent on the `RelOptListener` API instead of `VolcanoPlanner`.

##########
File path: core/src/main/resources/volcano-viz/volcano-viz-data.js
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.
+ */
+
+// sample generated visualizer data used by the visualizer JavaScript code
+var data = {
+
+	allNodes: {
+		"node1": {
+			label: "TableScan-1", // node label being displayed
+			isSubset: false, // if the node is a RelSubset
+			explanation: "table=t1", // additional explanation of properties
+			finalCost: "100 cpu, 100io" // final cost (in string) of the node at the end of optimization
+		},
+		"node2": {
+			label: "Filter-2",
+			isSubset: false,
+			explanation: "condition=c",
+			finalCost: "200cpu, 100io"
+		},
+		"node3": {
+			label: "TableSink-3",
+			isSubset: false,
+			explanation: "table=t2",
+			finalCost: "20cpu, 20io"
+		},
+		"node4": {
+			label: "IndexTableScan-4",
+			isSubset: false,
+			explanation: "table=t1, condition=c",
+			finalCost: "10cpu, 10io"
+		},
+	},

Review comment:
       A more realistic example from an actual run would be more easy to understand. The query doesn't have to be complicated just reflect actual classes/rules in Calcite.

##########
File path: core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleMatchVisualizer.java
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.volcano;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.tools.visualizer.InputExcludedRelWriter;
+import org.apache.calcite.tools.visualizer.VisualizerNodeInfo;
+import org.apache.calcite.tools.visualizer.VisualizerRuleMatchInfo;
+import org.apache.calcite.tools.visualizer.VolcanoRuleMatchVisualizerListener;
+
+import org.apache.commons.io.IOUtils;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Charsets;
+
+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.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ * This is tool to visualize the rule match process of the VolcanoPlanner.
+ *
+ *
+ * <p>To use the visualizer, add a listener before the VolcanoPlanner optimization phase.
+ * Then writes the output to a file after the optimization ends.
+ *
+ * <pre>
+ * // create the visualizer. This attaches a listener to VolcanoPlanner
+ * VolcanoRuleMatchVisualizer visualizer = VolcanoRuleMatchVisualizer.createAndAttach(planner);
+ *
+ * volcanoPlanner.findBestExpr();
+ *
+ * // writes the output to files
+ * visualizer.writeToFile(outputDirectory, "");
+ * </pre>
+ */
+public class VolcanoRuleMatchVisualizer {
+
+  public static VolcanoRuleMatchVisualizer createAndAttach(VolcanoPlanner volcanoPlanner) {
+    VolcanoRuleMatchVisualizer vis = new VolcanoRuleMatchVisualizer(volcanoPlanner);
+    // add listener outside of the constructor to make checker-framework happy

Review comment:
       I agree that being able to enable/disable easily the visualizer in Calcite without writing additional code is definitely useful. We could either associate it with a (new or existing) system property (e.g., `calcite.debug`) or maybe even better with a logger level. The advantage of associating this with a logger instead of a property is that it will not require a restart of the application to take effect.

##########
File path: core/src/main/java/org/apache/calcite/tools/visualizer/InputExcludedRelWriter.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.tools.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;
+
+/**
+ * This RelWriter is indented to be used for getting a digest of a relNode,
+ *  excluding the field of the relNode's inputs.
+ * The result digest of the RelNode only contains its own properties.
+ * <p>
+ *
+ * <pre>
+ * InputExcludedRelWriter relWriter = new InputExcludedRelWriter();
+ * rel.explain(relWriter);
+ * String digest = relWriter.toString();
+ * </pre>
+ *
+ */
+public class InputExcludedRelWriter implements RelWriter {

Review comment:
       Reduce visibility?

##########
File path: core/src/main/resources/volcano-viz/volcano-viz-data.js
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.
+ */
+
+// sample generated visualizer data used by the visualizer JavaScript code
+var data = {
+
+	allNodes: {
+		"node1": {
+			label: "TableScan-1", // node label being displayed
+			isSubset: false, // if the node is a RelSubset
+			explanation: "table=t1", // additional explanation of properties
+			finalCost: "100 cpu, 100io" // final cost (in string) of the node at the end of optimization
+		},
+		"node2": {
+			label: "Filter-2",
+			isSubset: false,
+			explanation: "condition=c",
+			finalCost: "200cpu, 100io"
+		},
+		"node3": {
+			label: "TableSink-3",
+			isSubset: false,
+			explanation: "table=t2",
+			finalCost: "20cpu, 20io"
+		},
+		"node4": {
+			label: "IndexTableScan-4",
+			isSubset: false,
+			explanation: "table=t1, condition=c",
+			finalCost: "10cpu, 10io"
+		},
+	},

Review comment:
       Moreover, I think it would be useful to add a few unit tests where the visualizer runs and generates these kind of files and compares with some reference files. The unit tests and their output would be the perfect example to understand the results of the visualizer.

##########
File path: core/src/main/java/org/apache/calcite/tools/visualizer/VisualizerNodeInfo.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.tools.visualizer;
+
+import org.apache.calcite.plan.RelOptCost;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.text.DecimalFormat;
+import java.text.MessageFormat;
+import java.util.Locale;
+
+/**
+ * VisualizerNodeInfo is a helper data class for Volcano Visualizer.
+ * This class will be serialized to JSON and read by the Javascript file.
+ */
+public class VisualizerNodeInfo {

Review comment:
       Reduce visibility (if possible)?




-- 
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: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org