asfgit closed pull request #1426: DRILL-6671: Multi level lateral unnest join 
is throwing an exception …
URL: https://github.com/apache/drill/pull/1426
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
index 987e65c2435..2f7fdce1839 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.planner.fragment;
 
+import java.util.ArrayDeque;
+import java.util.Deque;
 import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -116,29 +118,28 @@ public PhysicalOperator visitLateralJoin(LateralJoinPOP 
op, IndexedFragmentNode
 
     children.add(op.getLeft().accept(this, iNode));
     children.add(op.getRight().accept(this, iNode));
-    UnnestPOP unnestInLeftInput = iNode.getUnnest();
+    UnnestPOP unnestForThisLateral = iNode.getUnnest();
 
     PhysicalOperator newOp = op.getNewWithChildren(children);
     newOp.setCost(op.getCost());
     newOp.setOperatorId(Short.MAX_VALUE & op.getOperatorId());
 
-    ((LateralJoinPOP)newOp).setUnnestForLateralJoin(unnestInLeftInput);
-
+    ((LateralJoinPOP) newOp).setUnnestForLateralJoin(unnestForThisLateral);
     return newOp;
   }
 
   @Override
   public PhysicalOperator visitUnnest(UnnestPOP unnest, IndexedFragmentNode 
value) throws ExecutionSetupException {
     PhysicalOperator newOp = visitOp(unnest, value);
-    value.addUnnest((UnnestPOP)newOp);
+    value.addUnnest((UnnestPOP) newOp);
     return newOp;
   }
 
   public static class IndexedFragmentNode{
-    final Wrapper info;
-    final int minorFragmentId;
+    private final Wrapper info;
+    private final int minorFragmentId;
 
-    UnnestPOP unnest = null;
+    private final Deque<UnnestPOP> unnest = new ArrayDeque<>();
 
     public IndexedFragmentNode(int minorFragmentId, Wrapper info) {
       super();
@@ -163,11 +164,11 @@ public void addAllocation(PhysicalOperator pop) {
     }
 
     public void addUnnest(UnnestPOP unnest) {
-      this.unnest = unnest;
+      this.unnest.addFirst(unnest);
     }
 
     public UnnestPOP getUnnest() {
-      return this.unnest;
+      return this.unnest.removeFirst();
     }
 
   }
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnnestPrel.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnnestPrel.java
index 274f27a2e0b..a3449150918 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnnestPrel.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnnestPrel.java
@@ -25,6 +25,7 @@
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexFieldAccess;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
@@ -85,6 +86,15 @@ public boolean needsFinalColumnReordering() {
     return LateralJoinPrel.class;
   }
 
+  @Override
+  public RelNode accept(RexShuttle shuttle) {
+    RexNode ref = shuttle.apply(this.ref);
+    if (this.ref == ref) {
+      return this;
+    }
+    return new UnnestPrel(getCluster(), traitSet, rowType, ref);
+  }
+
   @Override
   public Prel prepareForLateralUnnestPipeline(List<RelNode> children) {
     RelDataTypeFactory typeFactory = this.getCluster().getTypeFactory();
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/LateralUnnestRowIDVisitor.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/LateralUnnestRowIDVisitor.java
index dc4af5b08ff..7734d90aa16 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/LateralUnnestRowIDVisitor.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/LateralUnnestRowIDVisitor.java
@@ -19,11 +19,18 @@
 
 import com.google.common.collect.Lists;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.rules.ProjectCorrelateTransposeRule;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.drill.exec.planner.physical.LateralJoinPrel;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.UnnestPrel;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 /**
  * LateralUnnestRowIDVisitor traverses the physical plan and modifies all the 
operators in the
@@ -59,16 +66,55 @@ public Prel visitPrel(Prel prel, Boolean isRightOfLateral) 
throws RuntimeExcepti
   }
 
   @Override
-  public Prel visitLateral(LateralJoinPrel prel, Boolean value) throws 
RuntimeException {
+  public Prel visitLateral(LateralJoinPrel prel, Boolean isRightOfLateral) 
throws RuntimeException {
     List<RelNode> children = Lists.newArrayList();
-    children.add(((Prel)prel.getInput(0)).accept(this, value));
+    children.add(((Prel) prel.getInput(0)).accept(this, isRightOfLateral));
     children.add(((Prel) prel.getInput(1)).accept(this, true));
 
-    return (Prel) prel.copy(prel.getTraitSet(), children);
+    if (!isRightOfLateral) {
+      return (Prel) prel.copy(prel.getTraitSet(), children);
+    } else {
+      //Adjust the column numbering due to an additional column 
"$drill_implicit_field$" is added to the inputs.
+      Map<Integer, Integer> requiredColsMap = new HashMap<>();
+      for (Integer corrColIndex : prel.getRequiredColumns()) {
+        requiredColsMap.put(corrColIndex, corrColIndex + 1);
+      }
+      ImmutableBitSet requiredColumns = prel.getRequiredColumns().shift(1);
+
+      CorrelationId corrId = prel.getCluster().createCorrel();
+      RexCorrelVariable updatedCorrel =
+              (RexCorrelVariable) prel.getCluster().getRexBuilder().makeCorrel(
+                      children.get(0).getRowType(),
+                      corrId);
+      RelNode rightChild = children.get(1).accept(
+              new CorrelateVarReplacer(
+                      new 
ProjectCorrelateTransposeRule.RexFieldAccessReplacer(prel.getCorrelationId(),
+                              updatedCorrel, 
prel.getCluster().getRexBuilder(), requiredColsMap)));
+      return (Prel) prel.copy(prel.getTraitSet(), children.get(0), rightChild,
+              corrId, requiredColumns, prel.getJoinType());
+    }
   }
 
   @Override
-  public Prel visitUnnest(UnnestPrel prel, Boolean value) throws 
RuntimeException {
+  public Prel visitUnnest(UnnestPrel prel, Boolean isRightOfLateral) throws 
RuntimeException {
     return prel.prepareForLateralUnnestPipeline(null);
   }
+
+  /**
+   * Visitor for RelNodes which applies specified {@link RexShuttle} visitor
+   * for every node in the tree.
+   */
+  public static class CorrelateVarReplacer extends 
ProjectCorrelateTransposeRule.RelNodesExprsHandler {
+    protected final RexShuttle rexVisitor;
+
+    public CorrelateVarReplacer(RexShuttle rexVisitor) {
+      super(rexVisitor);
+      this.rexVisitor = rexVisitor;
+    }
+
+    @Override
+    public RelNode visit(RelNode other) {
+      return super.visit(other.accept(rexVisitor));
+    }
+  }
 }
diff --git 
a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestE2EUnnestAndLateral.java
 
b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestE2EUnnestAndLateral.java
index cc9c14a2084..65807bc152d 100644
--- 
a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestE2EUnnestAndLateral.java
+++ 
b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestE2EUnnestAndLateral.java
@@ -58,40 +58,40 @@ public static void setupTestFiles() throws Exception {
 
   @Test
   public void testLateral_WithLimitInSubQuery() throws Exception {
-    String Sql = "SELECT customer.c_name, customer.c_address, orders.o_id, 
orders.o_amount " +
+    String sql = "SELECT customer.c_name, customer.c_address, orders.o_id, 
orders.o_amount " +
       "FROM cp.`lateraljoin/nested-customer.parquet` customer, LATERAL " +
       "(SELECT t.ord.o_id as o_id, t.ord.o_amount as o_amount FROM 
UNNEST(customer.orders) t(ord) LIMIT 1) orders";
-    runAndLog(Sql);
+    runAndLog(sql);
   }
 
   @Test
   public void testLateral_WithFilterInSubQuery() throws Exception {
-    String Sql = "SELECT customer.c_name, customer.c_address, orders.o_id, 
orders.o_amount " +
+    String sql = "SELECT customer.c_name, customer.c_address, orders.o_id, 
orders.o_amount " +
       "FROM cp.`lateraljoin/nested-customer.parquet` customer, LATERAL " +
       "(SELECT t.ord.o_id as o_id, t.ord.o_amount as o_amount FROM 
UNNEST(customer.orders) t(ord) WHERE t.ord.o_amount > 10) orders";
-    runAndLog(Sql);
+    runAndLog(sql);
   }
 
   @Test
   public void testLateral_WithFilterAndLimitInSubQuery() throws Exception {
-    String Sql = "SELECT customer.c_name, customer.c_address, orders.o_id, 
orders.o_amount " +
+    String sql = "SELECT customer.c_name, customer.c_address, orders.o_id, 
orders.o_amount " +
       "FROM cp.`lateraljoin/nested-customer.parquet` customer, LATERAL " +
       "(SELECT t.ord.o_id as o_id, t.ord.o_amount as o_amount FROM 
UNNEST(customer.orders) t(ord) WHERE t.ord.o_amount > 10 LIMIT 1) orders";
-    runAndLog(Sql);
+    runAndLog(sql);
   }
 
   @Test
   public void testLateral_WithTopNInSubQuery() throws Exception {
     runAndLog("alter session set `planner.enable_topn`=false");
 
-    String Sql = "SELECT customer.c_name, orders.o_id, orders.o_amount " +
+    String sql = "SELECT customer.c_name, orders.o_id, orders.o_amount " +
       "FROM cp.`lateraljoin/nested-customer.parquet` customer, LATERAL " +
       "(SELECT t.ord.o_id as o_id, t.ord.o_amount as o_amount FROM 
UNNEST(customer.orders) t(ord) ORDER BY " +
       "o_amount DESC LIMIT 1) orders";
 
     try {
       testBuilder()
-         .sqlQuery(Sql)
+         .sqlQuery(sql)
          .unOrdered()
          .baselineColumns("c_name", "o_id", "o_amount")
          .baselineValues("customer1", 3.0,  294.5)
@@ -113,14 +113,14 @@ public void testLateral_WithSortAndLimitInSubQuery() 
throws Exception {
 
     runAndLog("alter session set `planner.enable_topn`=false");
 
-    String Sql = "SELECT customer.c_name, orders.o_id, orders.o_amount " +
+    String sql = "SELECT customer.c_name, orders.o_id, orders.o_amount " +
       "FROM cp.`lateraljoin/nested-customer.parquet` customer, LATERAL " +
       "(SELECT t.ord.o_id as o_id, t.ord.o_amount as o_amount FROM 
UNNEST(customer.orders) t(ord) ORDER BY " +
       "o_amount DESC LIMIT 1) orders";
 
     try {
       testBuilder()
-        .sqlQuery(Sql)
+        .sqlQuery(sql)
         .unOrdered()
         .baselineColumns("c_name", "o_id", "o_amount")
         .baselineValues("customer1", 3.0,  294.5)
@@ -135,13 +135,13 @@ public void testLateral_WithSortAndLimitInSubQuery() 
throws Exception {
 
   @Test
   public void testLateral_WithSortInSubQuery() throws Exception {
-    String Sql = "SELECT customer.c_name, orders.o_id, orders.o_amount " +
+    String sql = "SELECT customer.c_name, orders.o_id, orders.o_amount " +
       "FROM cp.`lateraljoin/nested-customer.parquet` customer, LATERAL " +
       "(SELECT t.ord.o_id as o_id, t.ord.o_amount as o_amount FROM 
UNNEST(customer.orders) t(ord) ORDER BY " +
       "o_amount DESC) orders WHERE customer.c_id = 1.0";
 
     testBuilder()
-      .sqlQuery(Sql)
+      .sqlQuery(sql)
       .ordered()
       .baselineColumns("c_name", "o_id", "o_amount")
       .baselineValues("customer1", 3.0,  294.5)
@@ -152,28 +152,61 @@ public void testLateral_WithSortInSubQuery() throws 
Exception {
 
   @Test
   public void testOuterApply_WithFilterAndLimitInSubQuery() throws Exception {
-    String Sql = "SELECT customer.c_name, customer.c_address, orders.o_id, 
orders.o_amount " +
+    String sql = "SELECT customer.c_name, customer.c_address, orders.o_id, 
orders.o_amount " +
       "FROM cp.`lateraljoin/nested-customer.parquet` customer OUTER APPLY " +
       "(SELECT t.ord.o_id as o_id , t.ord.o_amount as o_amount FROM 
UNNEST(customer.orders) t(ord) WHERE t.ord.o_amount > 10 LIMIT 1) orders";
-    runAndLog(Sql);
+    runAndLog(sql);
   }
 
   @Test
   public void testLeftLateral_WithFilterAndLimitInSubQuery() throws Exception {
-    String Sql = "SELECT customer.c_name, customer.c_address, orders.o_id, 
orders.o_amount " +
+    String sql = "SELECT customer.c_name, customer.c_address, orders.o_id, 
orders.o_amount " +
       "FROM cp.`lateraljoin/nested-customer.parquet` customer LEFT JOIN 
LATERAL " +
       "(SELECT t.ord.o_id as o_id, t.ord.o_amount as o_amount FROM 
UNNEST(customer.orders) t(ord) WHERE t.ord.o_amount > 10 LIMIT 1) orders ON 
TRUE";
-    runAndLog(Sql);
+    runAndLog(sql);
   }
 
   @Test
   public void testMultiUnnestAtSameLevel() throws Exception {
-    String Sql = "EXPLAIN PLAN FOR SELECT customer.c_name, customer.c_address, 
U1.order_id, U1.order_amt," +
-      " U1.itemName, U1.itemNum" + " FROM 
cp.`lateraljoin/nested-customer.parquet` customer, LATERAL" +
-      " (SELECT t.ord.o_id AS order_id, t.ord.o_amount AS order_amt, 
U2.item_name AS itemName, U2.item_num AS " +
-        "itemNum FROM UNNEST(customer.orders) t(ord) , LATERAL" +
-      " (SELECT t1.ord.i_name AS item_name, t1.ord.i_number AS item_num FROM 
UNNEST(t.ord) AS t1(ord)) AS U2) AS U1";
-    runAndLog(Sql);
+    String sql = "EXPLAIN PLAN FOR SELECT customer.c_name, customer.c_address, 
U1.order_id, U1.order_amt," +
+            " U1.itemName, U1.itemNum" + " FROM 
cp.`lateraljoin/nested-customer.parquet` customer, LATERAL" +
+            " (SELECT t.ord.o_id AS order_id, t.ord.o_amount AS order_amt, 
U2.item_name AS itemName, U2.item_num AS " +
+            "itemNum FROM UNNEST(customer.orders) t(ord) , LATERAL" +
+            " (SELECT t1.ord.i_name AS item_name, t1.ord.i_number AS item_num 
FROM UNNEST(t.ord) AS t1(ord)) AS U2) AS U1";
+    runAndLog(sql);
+  }
+
+  @Test
+  public void testMultiUnnestAtSameLevelExec() throws Exception {
+    String sql = "SELECT customer.c_name, customer.c_address, U1.order_id, 
U1.order_amt," +
+      " U1.itemName, U1.itemNum FROM cp.`lateraljoin/nested-customer.parquet` 
customer, LATERAL" +
+      " (SELECT dt.order_id, dt.order_amt, U2.item_name AS itemName, 
U2.item_num AS itemNum from" +
+            "(select t.ord.items as items, t.ord.o_id AS order_id, 
t.ord.o_amount AS order_amt FROM UNNEST(customer.orders) t(ord)) dt , LATERAL" +
+      " (SELECT t1.items.i_name AS item_name, t1.items.i_number AS item_num 
FROM UNNEST(dt.items) AS t1(items)) AS U2) AS U1";
+    String baseline = "SELECT customer.c_name, customer.c_address, 
U1.order_id, U1.order_amount as order_amt, U2.item_name as itemName, 
U2.item_num as itemNum" +
+            " FROM cp.`lateraljoin/nested-customer.parquet` customer, LATERAL 
" +
+            "(SELECT t.ord.items as items, t.ord.o_id as order_id, 
t.ord.o_amount as order_amount from UNNEST(customer.orders) t(ord)) U1, 
LATERAL" +
+            "(SELECT t1.items.i_name as item_name, t1.items.i_number as 
item_num from UNNEST(U1.items) t1(items)) U2";
+    testBuilder()
+            .unOrdered()
+            .sqlQuery(sql)
+            .sqlBaselineQuery(baseline)
+            .go();
+  }
+
+  @Test
+  public void testMultiUnnestAtSameLevelExecExplicitResult() throws Exception {
+    String sql = "SELECT customer.c_name, customer.c_address, U1.order_id, 
U1.order_amt," +
+            " U1.itemName, U1.itemNum FROM 
cp.`lateraljoin/nested-customer.parquet` customer, LATERAL" +
+            " (SELECT dt.order_id, dt.order_amt, U2.item_name AS itemName, 
U2.item_num AS itemNum from" +
+            "(select t.ord.items as items, t.ord.o_id AS order_id, 
t.ord.o_amount AS order_amt FROM UNNEST(customer.orders) t(ord)) dt , LATERAL" +
+            " (SELECT t1.items.i_name AS item_name, t1.items.i_number AS 
item_num FROM UNNEST(dt.items) AS t1(items)) AS U2) AS U1 order by 1,2,3,4,5,6 
limit 1";
+    testBuilder()
+            .unOrdered()
+            .sqlQuery(sql)
+            .baselineColumns("c_name", "c_address", "order_id", "order_amt", 
"itemName", "itemNum")
+            .baselineValues("customer1","bay area, CA",1.0,4.5,"cheese",9.0)
+            .go();
   }
 
   @Test
@@ -265,9 +298,9 @@ public void testSingleUnnestCol() throws Exception {
 
   @Test
   public void testNestedUnnest() throws Exception {
-    String Sql = "select * from (select customer.orders as orders from 
cp.`lateraljoin/nested-customer.parquet` customer ) t1," +
+    String sql = "select * from (select customer.orders as orders from 
cp.`lateraljoin/nested-customer.parquet` customer ) t1," +
         " lateral ( select t.ord.items as items from unnest(t1.orders) t(ord) 
) t2, unnest(t2.items) t3(item) ";
-    runAndLog(Sql);
+    runAndLog(sql);
   }
 
   
/***********************************************************************************************
diff --git 
a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestLateralPlans.java
 
b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestLateralPlans.java
index 9cb5b6dc12e..ef52f8e5cae 100644
--- 
a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestLateralPlans.java
+++ 
b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestLateralPlans.java
@@ -51,15 +51,15 @@ public static void enableUnnestLateral() throws Exception {
 
   @Test
   public void testLateralSql() throws Exception {
-    String Sql = "select t.c_name, t2.ord.o_shop as o_shop from 
cp.`lateraljoin/nested-customer.json` t," +
+    String sql = "select t.c_name, t2.ord.o_shop as o_shop from 
cp.`lateraljoin/nested-customer.json` t," +
         " unnest(t.orders) t2(ord) limit 1";
 
-    PlanTestBase.testPlanMatchingPatterns(Sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"},
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"},
       new String[]{});
 
     testBuilder()
         .unOrdered()
-        .sqlQuery(Sql)
+        .sqlQuery(sql)
         .baselineColumns("c_name", "o_shop")
         .baselineValues("customer1", "Meno Park 1st")
         .go();
@@ -70,10 +70,10 @@ public void testExplainLateralSql() throws Exception {
     String explainSql = "explain plan without implementation for select 
t.c_name, t2.ord.o_shop as o_shop from cp.`lateraljoin/nested-customer.json` 
t," +
         " unnest(t.orders) t2(ord) limit 1";
 
-    String Sql = "select t.c_name, t2.ord.o_shop as o_shop from 
cp.`lateraljoin/nested-customer.json` t," +
+    String sql = "select t.c_name, t2.ord.o_shop as o_shop from 
cp.`lateraljoin/nested-customer.json` t," +
       " unnest(t.orders) t2(ord) limit 1";
 
-    PlanTestBase.testPlanMatchingPatterns(Sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"},
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"},
       new String[]{});
 
     test(explainSql);
@@ -82,18 +82,18 @@ public void testExplainLateralSql() throws Exception {
   @Test
   public void testFilterPushCorrelate() throws Exception {
     test("alter session set `planner.slice_target`=1");
-    String query = "select t.c_name, t2.ord.o_shop as o_shop from 
cp.`lateraljoin/nested-customer.json` t,"
+    String sql = "select t.c_name, t2.ord.o_shop as o_shop from 
cp.`lateraljoin/nested-customer.json` t,"
         + " unnest(t.orders) t2(ord) where t.c_name='customer1' AND 
t2.ord.o_shop='Meno Park 1st' ";
 
-    PlanTestBase.testPlanMatchingPatterns(query, new 
String[]{"LateralJoin(.*[\n\r])+.*Filter(.*[\n\r])+.*Scan(.*[\n\r])+.*Filter"},
+    PlanTestBase.testPlanMatchingPatterns(sql, new 
String[]{"LateralJoin(.*[\n\r])+.*Filter(.*[\n\r])+.*Scan(.*[\n\r])+.*Filter"},
         new String[]{});
 
-    PlanTestBase.testPlanMatchingPatterns(query, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"},
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"},
       new String[]{});
 
     testBuilder()
         .unOrdered()
-        .sqlQuery(query)
+        .sqlQuery(sql)
         .baselineColumns("c_name", "o_shop")
         .baselineValues("customer1", "Meno Park 1st")
         .go();
@@ -101,15 +101,15 @@ public void testFilterPushCorrelate() throws Exception {
 
   @Test
   public void testLateralSqlPlainCol() throws Exception {
-    String Sql = "select t.c_name, t2.phone as c_phone from 
cp.`lateraljoin/nested-customer.json` t,"
+    String sql = "select t.c_name, t2.phone as c_phone from 
cp.`lateraljoin/nested-customer.json` t,"
         + " unnest(t.c_phone) t2(phone) limit 1";
 
-    PlanTestBase.testPlanMatchingPatterns(Sql, new String[]{"column excluded 
from output: =\\[\\`c_phone\\`\\]"},
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{"column excluded 
from output: =\\[\\`c_phone\\`\\]"},
       new String[]{});
 
     testBuilder()
         .unOrdered()
-        .sqlQuery(Sql)
+        .sqlQuery(sql)
         .baselineColumns("c_name", "c_phone")
         .baselineValues("customer1", "6505200001")
         .go();
@@ -117,14 +117,14 @@ public void testLateralSqlPlainCol() throws Exception {
 
   @Test
   public void testLateralSqlStar() throws Exception {
-    String Sql = "select * from cp.`lateraljoin/nested-customer.json` t, 
unnest(t.orders) Orders(ord) limit 0";
+    String sql = "select * from cp.`lateraljoin/nested-customer.json` t, 
unnest(t.orders) Orders(ord) limit 0";
 
-    PlanTestBase.testPlanMatchingPatterns(Sql, new String[]{},
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{},
       new String[]{"column excluded from output: =\\[\\`orders\\`\\]"});
 
     testBuilder()
         .unOrdered()
-        .sqlQuery(Sql)
+        .sqlQuery(sql)
         .baselineColumns("c_name", "c_id", "c_phone", "orders", "c_address", 
"ord")
         .expectsEmptyResultSet()
         .go();
@@ -147,14 +147,14 @@ public void testLateralSqlStar2() throws Exception {
 
   @Test
   public void testLateralSqlStar3() throws Exception {
-    String Sql = "select Orders.*, c.* from 
cp.`lateraljoin/nested-customer.json` c, unnest(c.orders) Orders(ord) limit 0";
+    String sql = "select Orders.*, c.* from 
cp.`lateraljoin/nested-customer.json` c, unnest(c.orders) Orders(ord) limit 0";
 
-    PlanTestBase.testPlanMatchingPatterns(Sql, new String[]{},
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{},
       new String[]{"column excluded from output: =\\[\\`orders\\`\\]"});
 
     testBuilder()
         .unOrdered()
-        .sqlQuery(Sql)
+        .sqlQuery(sql)
         .baselineColumns("ord","c_name", "c_id", "c_phone", "orders", 
"c_address")
         .expectsEmptyResultSet()
         .go();
@@ -162,13 +162,13 @@ public void testLateralSqlStar3() throws Exception {
 
   @Test
   public void testLateralSqlStar4() throws Exception {
-    String Sql = "select Orders.* from cp.`lateraljoin/nested-customer.json` 
c, unnest(c.orders) Orders(ord) limit 0";
+    String sql = "select Orders.* from cp.`lateraljoin/nested-customer.json` 
c, unnest(c.orders) Orders(ord) limit 0";
 
-    PlanTestBase.testPlanMatchingPatterns(Sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"}, new String[]{});
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"}, new String[]{});
 
     testBuilder()
         .unOrdered()
-        .sqlQuery(Sql)
+        .sqlQuery(sql)
         .baselineColumns("ord")
         .expectsEmptyResultSet()
         .go();
@@ -176,17 +176,17 @@ public void testLateralSqlStar4() throws Exception {
 
   @Test
   public void testLateralSqlWithAS() throws Exception {
-    String Sql = "select t.c_name, t2.orders from 
cp.`lateraljoin/nested-customer.parquet` t,"
+    String sql = "select t.c_name, t2.orders from 
cp.`lateraljoin/nested-customer.parquet` t,"
         + " unnest(t.orders) as t2(orders)";
     String baselineQuery = "select t.c_name, t2.orders from 
cp.`lateraljoin/nested-customer.parquet` t inner join" +
         " (select c_name, flatten(orders) from cp" +
         ".`lateraljoin/nested-customer.parquet` ) as t2(name, orders) on 
t.c_name = t2.name";
 
-    PlanTestBase.testPlanMatchingPatterns(Sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"}, new String[]{});
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"}, new String[]{});
 
     testBuilder()
         .unOrdered()
-        .sqlQuery(Sql)
+        .sqlQuery(sql)
         .sqlBaselineQuery(baselineQuery)
         .go();
 
@@ -194,25 +194,25 @@ public void testLateralSqlWithAS() throws Exception {
 
   @Test
   public void testMultiUnnestLateralAtSameLevel() throws Exception {
-    String Sql = "select t.c_name, t2.orders, t3.orders from 
cp.`lateraljoin/nested-customer.parquet` t," +
+    String sql = "select t.c_name, t2.orders, t3.orders from 
cp.`lateraljoin/nested-customer.parquet` t," +
         " LATERAL ( select t2.orders from unnest(t.orders) as t2(orders)) as 
t2, LATERAL " +
         "(select t3.orders from unnest(t.orders) as t3(orders)) as t3";
     String baselineQuery = "select t.c_name, t2.orders, t3.orders from 
cp.`lateraljoin/nested-customer.parquet` t inner join" +
         " (select c_name, flatten(orders) from 
cp.`lateraljoin/nested-customer.parquet` ) as t2 (name, orders) on t.c_name = 
t2.name " +
         " inner join (select c_name, flatten(orders) from 
cp.`lateraljoin/nested-customer.parquet` ) as t3(name, orders) on t.c_name = 
t3.name";
 
-    PlanTestBase.testPlanMatchingPatterns(Sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"}, new String[]{});
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"}, new String[]{});
 
     testBuilder()
         .unOrdered()
-        .sqlQuery(Sql)
+        .sqlQuery(sql)
         .sqlBaselineQuery(baselineQuery)
         .go();
   }
 
   @Test
   public void testSubQuerySql() throws Exception {
-    String Sql = "select t.c_name, d1.items as items0 , t3.items as items1 
from cp.`lateraljoin/nested-customer.parquet` t," +
+    String sql = "select t.c_name, d1.items as items0 , t3.items as items1 
from cp.`lateraljoin/nested-customer.parquet` t," +
         " lateral (select t2.ord.items as items from unnest(t.orders) t2(ord)) 
d1," +
         " unnest(d1.items) t3(items)";
 
@@ -220,18 +220,18 @@ public void testSubQuerySql() throws Exception {
         " inner join (select c_name, f, flatten(t1.f.items) from (select 
c_name, flatten(orders) as f from cp.`lateraljoin/nested-customer.parquet`) as 
t1 ) " +
         "t3(name, orders, items) on t.c_name = t3.name ";
 
-    PlanTestBase.testPlanMatchingPatterns(Sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"}, new String[]{"column excluded from output: 
=\\[\\`items\\`\\]"});
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"}, new String[]{"column excluded from output: 
=\\[\\`items\\`\\]"});
 
     testBuilder()
         .unOrdered()
-        .sqlQuery(Sql)
+        .sqlQuery(sql)
         .sqlBaselineQuery(baselineQuery)
         .go();
   }
 
   @Test
   public void testUnnestWithFilter() throws Exception {
-    String Sql = "select t.c_name, d1.items as items0, t3.items as items1 from 
cp.`lateraljoin/nested-customer.parquet` t," +
+    String sql = "select t.c_name, d1.items as items0, t3.items as items1 from 
cp.`lateraljoin/nested-customer.parquet` t," +
         " lateral (select t2.ord.items as items from unnest(t.orders) t2(ord)) 
d1," +
         " unnest(d1.items) t3(items) where t.c_id > 1";
 
@@ -239,18 +239,18 @@ public void testUnnestWithFilter() throws Exception {
         " inner join (select c_name, f, flatten(t1.f.items) from (select 
c_name, flatten(orders) as f from cp.`lateraljoin/nested-customer.parquet`) as 
t1 ) " +
         "t3(name, orders, items) on t.c_name = t3.name where t.c_id > 1";
 
-    PlanTestBase.testPlanMatchingPatterns(Sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"}, new String[]{"column excluded from output: 
=\\[\\`items\\`\\]"});
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]"}, new String[]{"column excluded from output: 
=\\[\\`items\\`\\]"});
 
     testBuilder()
         .unOrdered()
-        .sqlQuery(Sql)
+        .sqlQuery(sql)
         .sqlBaselineQuery(baselineQuery)
         .go();
   }
 
   @Test
   public void testUnnestWithAggInSubquery() throws Exception {
-    String Sql = "select t.c_name, sum(t4.items) from 
cp.`lateraljoin/nested-customer.parquet` t," +
+    String sql = "select t.c_name, sum(t4.items) from 
cp.`lateraljoin/nested-customer.parquet` t," +
         " lateral (select t2.ord.items as items from unnest(t.orders) t2(ord)) 
d1," +
         " lateral (select sum(t3.items.i_number) from unnest(d1.items) 
t3(items)) t4(items) where t.c_id > 1 group by t.c_name";
 
@@ -267,14 +267,14 @@ public void testUnnestWithAggInSubquery() throws 
Exception {
           .testBuilder()
           .ordered()
           .sqlBaselineQuery(baselineQuery)
-          .sqlQuery(Sql)
+          .sqlQuery(sql)
           .go();
     }
   }
 
   @Test
   public void testUnnestWithAggOnOuterTable() throws Exception {
-    String Sql = "select avg(d2.inum) from 
cp.`lateraljoin/nested-customer.parquet` t," +
+    String sql = "select avg(d2.inum) from 
cp.`lateraljoin/nested-customer.parquet` t," +
         " lateral (select t2.ord.items as items from unnest(t.orders) t2(ord)) 
d1," +
         " lateral (select t3.items.i_number as inum from unnest(d1.items) 
t3(items)) d2 where t.c_id > 1 group by t.c_id";
 
@@ -282,11 +282,11 @@ public void testUnnestWithAggOnOuterTable() throws 
Exception {
         " inner join (select c_name, f, flatten(t1.f.items) from (select 
c_name, flatten(orders) as f from cp.`lateraljoin/nested-customer.parquet`) as 
t1 ) " +
         "t3(name, orders, items) on t.c_name = t3.name where t.c_id > 1 group 
by t.c_id";
 
-    PlanTestBase.testPlanMatchingPatterns(Sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]", "column excluded from output: 
=\\[\\`items\\`\\]"}, new String[]{});
+    PlanTestBase.testPlanMatchingPatterns(sql, new String[]{"column excluded 
from output: =\\[\\`orders\\`\\]", "column excluded from output: 
=\\[\\`items\\`\\]"}, new String[]{});
 
     testBuilder()
         .unOrdered()
-        .sqlQuery(Sql)
+        .sqlQuery(sql)
         .sqlBaselineQuery(baselineQuery)
         .go();
 
@@ -294,7 +294,7 @@ public void testUnnestWithAggOnOuterTable() throws 
Exception {
 
   @Test
   public void testUnnestTableAndColumnAlias() throws Exception {
-    String Sql = "select t.c_name from cp.`lateraljoin/nested-customer.json` 
t, unnest(t.orders) ";
+    String sql = "select t.c_name from cp.`lateraljoin/nested-customer.json` 
t, unnest(t.orders) ";
     ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
         .setOptionDefault(ExecConstants.ENABLE_UNNEST_LATERAL_KEY, true);
 
@@ -302,7 +302,7 @@ public void testUnnestTableAndColumnAlias() throws 
Exception {
          ClientFixture client = cluster.clientFixture()) {
       client
           .queryBuilder()
-          .sql(Sql)
+          .sql(sql)
           .run();
     } catch (UserRemoteException ex) {
       assertTrue(ex.getMessage().contains("Alias table and column name are 
required for UNNEST"));
@@ -311,7 +311,7 @@ public void testUnnestTableAndColumnAlias() throws 
Exception {
 
   @Test
   public void testUnnestColumnAlias() throws Exception {
-    String Sql = "select t.c_name, t2.orders from 
cp.`lateraljoin/nested-customer.json` t, unnest(t.orders) t2";
+    String sql = "select t.c_name, t2.orders from 
cp.`lateraljoin/nested-customer.json` t, unnest(t.orders) t2";
     ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
         .setOptionDefault(ExecConstants.ENABLE_UNNEST_LATERAL_KEY, true);
 
@@ -319,7 +319,7 @@ public void testUnnestColumnAlias() throws Exception {
          ClientFixture client = cluster.clientFixture()) {
       client
           .queryBuilder()
-          .sql(Sql)
+          .sql(sql)
           .run();
     } catch (UserRemoteException ex) {
       assertTrue(ex.getMessage().contains("Alias table and column name are 
required for UNNEST"));
@@ -333,7 +333,7 @@ public void testUnnestColumnAlias() throws Exception {
 
   @Test
   public void testNoExchangeWithAggWithoutGrpBy() throws Exception {
-    String Sql = "select d1.totalprice from dfs.`lateraljoin/multipleFiles` 
t," +
+    String sql = "select d1.totalprice from dfs.`lateraljoin/multipleFiles` 
t," +
             " lateral ( select sum(t2.ord.o_totalprice) as totalprice from 
unnest(t.c_orders) t2(ord)) d1";
     ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
             .setOptionDefault(ExecConstants.ENABLE_UNNEST_LATERAL_KEY, true)
@@ -341,7 +341,7 @@ public void testNoExchangeWithAggWithoutGrpBy() throws 
Exception {
 
     try (ClusterFixture cluster = builder.build();
          ClientFixture client = cluster.clientFixture()) {
-      String explain = client.queryBuilder().sql(Sql).explainText();
+      String explain = client.queryBuilder().sql(sql).explainText();
       String rightChild = getRightChildOfLateral(explain);
       assertFalse(rightChild.contains("Exchange"));
     }
@@ -349,7 +349,7 @@ public void testNoExchangeWithAggWithoutGrpBy() throws 
Exception {
 
   @Test
   public void testNoExchangeWithStreamAggWithGrpBy() throws Exception {
-    String Sql = "select d1.totalprice from dfs.`lateraljoin/multipleFiles` 
t," +
+    String sql = "select d1.totalprice from dfs.`lateraljoin/multipleFiles` 
t," +
             " lateral ( select sum(t2.ord.o_totalprice) as totalprice from 
unnest(t.c_orders) t2(ord) group by t2.ord.o_orderkey) d1";
 
     ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
@@ -360,7 +360,7 @@ public void testNoExchangeWithStreamAggWithGrpBy() throws 
Exception {
 
     try (ClusterFixture cluster = builder.build();
          ClientFixture client = cluster.clientFixture()) {
-      String explain = client.queryBuilder().sql(Sql).explainText();
+      String explain = client.queryBuilder().sql(sql).explainText();
       String rightChild = getRightChildOfLateral(explain);
       assertFalse(rightChild.contains("Exchange"));
     }
@@ -368,7 +368,7 @@ public void testNoExchangeWithStreamAggWithGrpBy() throws 
Exception {
 
   @Test
   public void testNoExchangeWithHashAggWithGrpBy() throws Exception {
-    String Sql = "select d1.totalprice from dfs.`lateraljoin/multipleFiles` 
t," +
+    String sql = "select d1.totalprice from dfs.`lateraljoin/multipleFiles` 
t," +
             " lateral ( select sum(t2.ord.o_totalprice) as totalprice from 
unnest(t.c_orders) t2(ord) group by t2.ord.o_orderkey) d1";
     ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
             .setOptionDefault(ExecConstants.ENABLE_UNNEST_LATERAL_KEY, true)
@@ -378,7 +378,7 @@ public void testNoExchangeWithHashAggWithGrpBy() throws 
Exception {
 
     try (ClusterFixture cluster = builder.build();
          ClientFixture client = cluster.clientFixture()) {
-      String explain = client.queryBuilder().sql(Sql).explainText();
+      String explain = client.queryBuilder().sql(sql).explainText();
       String rightChild = getRightChildOfLateral(explain);
       assertFalse(rightChild.contains("Exchange"));
     }
@@ -402,7 +402,7 @@ public void testNoExchangeWithOrderByWithoutLimit() throws 
Exception {
 
   @Test
   public void testNoExchangeWithOrderByLimit() throws Exception {
-    String Sql = "select d1.totalprice from dfs.`lateraljoin/multipleFiles` 
t," +
+    String sql = "select d1.totalprice from dfs.`lateraljoin/multipleFiles` 
t," +
             " lateral ( select t2.ord.o_totalprice as totalprice from 
unnest(t.c_orders) t2(ord) order by t2.ord.o_orderkey limit 10) d1";
     ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
             .setOptionDefault(ExecConstants.ENABLE_UNNEST_LATERAL_KEY, true)
@@ -410,7 +410,7 @@ public void testNoExchangeWithOrderByLimit() throws 
Exception {
 
     try (ClusterFixture cluster = builder.build();
          ClientFixture client = cluster.clientFixture()) {
-      String explain = client.queryBuilder().sql(Sql).explainText();
+      String explain = client.queryBuilder().sql(sql).explainText();
       String rightChild = getRightChildOfLateral(explain);
       assertFalse(rightChild.contains("Exchange"));
     }
@@ -419,7 +419,7 @@ public void testNoExchangeWithOrderByLimit() throws 
Exception {
 
   @Test
   public void testNoExchangeWithLateralsDownStreamJoin() throws Exception {
-    String Sql = "select d1.totalprice from dfs.`lateraljoin/multipleFiles` t, 
dfs.`lateraljoin/multipleFiles` t2, " +
+    String sql = "select d1.totalprice from dfs.`lateraljoin/multipleFiles` t, 
dfs.`lateraljoin/multipleFiles` t2, " +
             " lateral ( select t2.ord.o_totalprice as totalprice from 
unnest(t.c_orders) t2(ord) order by t2.ord.o_orderkey limit 10) d1" +
             " where t.c_name = t2.c_name";
     ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
@@ -428,7 +428,7 @@ public void testNoExchangeWithLateralsDownStreamJoin() 
throws Exception {
 
     try (ClusterFixture cluster = builder.build();
          ClientFixture client = cluster.clientFixture()) {
-      String explain = client.queryBuilder().sql(Sql).explainText();
+      String explain = client.queryBuilder().sql(sql).explainText();
       String rightChild = getRightChildOfLateral(explain);
       assertFalse(rightChild.contains("Exchange"));
     }
@@ -436,7 +436,7 @@ public void testNoExchangeWithLateralsDownStreamJoin() 
throws Exception {
 
   @Test
   public void testNoExchangeWithLateralsDownStreamUnion() throws Exception {
-    String Sql = "select t.c_name from dfs.`lateraljoin/multipleFiles` t union 
all " +
+    String sql = "select t.c_name from dfs.`lateraljoin/multipleFiles` t union 
all " +
             " select t.c_name from dfs.`lateraljoin/multipleFiles` t, " +
                     " lateral ( select t2.ord.o_totalprice as totalprice from 
unnest(t.c_orders) t2(ord) order by t2.ord.o_orderkey limit 10) d1";
     ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
@@ -445,7 +445,7 @@ public void testNoExchangeWithLateralsDownStreamUnion() 
throws Exception {
 
     try (ClusterFixture cluster = builder.build();
          ClientFixture client = cluster.clientFixture()) {
-      String explain = client.queryBuilder().sql(Sql).explainText();
+      String explain = client.queryBuilder().sql(sql).explainText();
       String rightChild = getRightChildOfLateral(explain);
       assertFalse(rightChild.contains("Exchange"));
     }
@@ -453,7 +453,7 @@ public void testNoExchangeWithLateralsDownStreamUnion() 
throws Exception {
 
   @Test
   public void testNoExchangeWithLateralsDownStreamAgg() throws Exception {
-    String Sql = "select sum(d1.totalprice) from 
dfs.`lateraljoin/multipleFiles` t, " +
+    String sql = "select sum(d1.totalprice) from 
dfs.`lateraljoin/multipleFiles` t, " +
             " lateral ( select t2.ord.o_totalprice as totalprice from 
unnest(t.c_orders) t2(ord) order by t2.ord.o_orderkey limit 10) d1 group by 
t.c_custkey";
     ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
             .setOptionDefault(ExecConstants.ENABLE_UNNEST_LATERAL_KEY, true)
@@ -463,7 +463,7 @@ public void testNoExchangeWithLateralsDownStreamAgg() 
throws Exception {
 
     try (ClusterFixture cluster = builder.build();
          ClientFixture client = cluster.clientFixture()) {
-      String explain = client.queryBuilder().sql(Sql).explainText();
+      String explain = client.queryBuilder().sql(sql).explainText();
       String rightChild = getRightChildOfLateral(explain);
       assertFalse(rightChild.contains("Exchange"));
     }
@@ -480,14 +480,14 @@ private String getRightChildOfLateral(String explain) 
throws Exception {
   //The following test is for testing the explain plan contains relation 
between lateral and corresponding unnest.
   @Test
   public void testLateralAndUnnestExplainPlan() throws Exception {
-    String Sql = "select c.* from cp.`lateraljoin/nested-customer.json` c, 
unnest(c.orders) Orders(ord)";
+    String sql = "select c.* from cp.`lateraljoin/nested-customer.json` c, 
unnest(c.orders) Orders(ord)";
     ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
             .setOptionDefault(ExecConstants.ENABLE_UNNEST_LATERAL_KEY, true)
             .setOptionDefault(ExecConstants.SLICE_TARGET, 1);
 
     try (ClusterFixture cluster = builder.build();
          ClientFixture client = cluster.clientFixture()) {
-      String explain = client.queryBuilder().sql(Sql).explainText();
+      String explain = client.queryBuilder().sql(sql).explainText();
       String srcOp = explain.substring(explain.indexOf("srcOp"));
       assertTrue(srcOp != null && srcOp.length() > 0);
       String correlateFragmentPattern = srcOp.substring(srcOp.indexOf("=")+1, 
srcOp.indexOf("]"));
@@ -499,7 +499,7 @@ public void testLateralAndUnnestExplainPlan() throws 
Exception {
 
   @Test
   public void testUnnestTopN() throws Exception {
-    String query =
+    String sql =
         "select customer.c_custkey," +
                 "customer.c_name," +
                 "t.o.o_orderkey," +
@@ -517,7 +517,7 @@ public void testUnnestTopN() throws Exception {
     try (ClusterFixture cluster = builder.build();
          ClientFixture client = cluster.clientFixture()) {
       String plan = client.queryBuilder()
-          .sql(query)
+          .sql(sql)
           .explainText();
 
       assertThat("Query plan doesn't contain TopN operator",
@@ -529,7 +529,7 @@ public void testUnnestTopN() throws Exception {
 
   @Test
   public void testMultiUnnestQuery() throws Exception {
-    String Sql = "SELECT t5.l_quantity FROM dfs.`lateraljoin/multipleFiles` t, 
" +
+    String sql = "SELECT t5.l_quantity FROM dfs.`lateraljoin/multipleFiles` t, 
" +
             "LATERAL (SELECT t2.ordrs.o_lineitems FROM UNNEST(t.c_orders) 
t2(ordrs)) t3(lineitems), " +
             "LATERAL (SELECT t4.lineitems.l_quantity FROM UNNEST(t3.lineitems) 
t4(lineitems)) t5(l_quantity) order by 1";
 
@@ -545,7 +545,7 @@ public void testMultiUnnestQuery() throws Exception {
       client.testBuilder()
               .ordered()
               .sqlBaselineQuery(baselineQuery)
-              .sqlQuery(Sql)
+              .sqlQuery(sql)
               .go();
     }
   }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to