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/15 22:00:37 UTC

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

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



##########
File path: core/src/main/java/org/apache/calcite/plan/visualizer/package-info.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ *
+ *
+ *
+ * <p>  See {@link org.apache.calcite.plan.visualizer.RuleMatchVisualizer}.

Review comment:
       use `@see`

##########
File path: core/src/main/java/org/apache/calcite/plan/visualizer/StepInfo.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.List;
+import java.util.Map;
+
+/**
+ * All info for one step.
+ */
+class StepInfo {
+  // public, so that attributes are serialized to json
+  public final String id;
+  public final Map<String, Object> updates;
+  public final List<String> matchedRels;

Review comment:
       remove `public` modifiers, since class is package-protected.
   
   since list and map fields are not private, make them immutable copies.

##########
File path: core/src/main/java/org/apache/calcite/plan/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.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;
+
+/**
+ * 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>

Review comment:
       remove `<p>`

##########
File path: core/src/main/java/org/apache/calcite/plan/visualizer/StepInfo.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.List;
+import java.util.Map;
+
+/**
+ * All info for one step.
+ */

Review comment:
       'all the info for' is redundant. Classes always contain data. Instead, define a step.

##########
File path: core/src/test/java/org/apache/calcite/test/RuleMatchVisualizerTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.jetbrains.annotations.NotNull;
+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.
+   */
+  @NotNull private String renameMatches(final String str,

Review comment:
       don't use jetbrains `@NoNull ` annotation. 

##########
File path: core/src/main/java/org/apache/calcite/plan/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.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;
+
+/**
+ * This RelWriter is indented to be used for getting a digest of a relNode,

Review comment:
       Remove "This". Instead "An implement of RelWriter that..."
   
   "Indented" should be "intended"?

##########
File path: core/src/main/java/org/apache/calcite/plan/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.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;
+
+/**
+ * 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>

Review comment:
       preferred style is `<pre>{@code`. (see https://reflectoring.io/howto-format-code-snippets-in-javadoc/)

##########
File path: 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" : [ {

Review comment:
       why spaces after colons? can we use standard JSON formatting?




-- 
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