You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by jn...@apache.org on 2017/04/03 05:12:01 UTC

[1/9] drill git commit: DRILL-5394: Optimize query planning for MapR-DB tables by caching row counts

Repository: drill
Updated Branches:
  refs/heads/master 841ead401 -> d3718a62e


DRILL-5394: Optimize query planning for MapR-DB tables by caching row counts

close #802


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/e03d7f9b
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/e03d7f9b
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/e03d7f9b

Branch: refs/heads/master
Commit: e03d7f9b1ac510ccee909b42653ee9b9ca3d69b1
Parents: 841ead4
Author: Padma Penumarthy <pp...@yahoo.com>
Authored: Thu Mar 16 11:40:13 2017 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Fri Mar 31 15:31:40 2017 -0700

----------------------------------------------------------------------
 .../store/mapr/db/MapRDBPushFilterIntoScan.java |  4 +++-
 .../mapr/db/binary/BinaryTableGroupScan.java    | 20 ++++++++++++++++++--
 2 files changed, 21 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/e03d7f9b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
----------------------------------------------------------------------
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
index 6a286a8..b363262 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
@@ -184,8 +184,10 @@ public abstract class MapRDBPushFilterIntoScan extends StoragePluginOptimizerRul
       return; //no filter pushdown ==> No transformation.
     }
 
+    // Pass tableStats from old groupScan so we do not go and fetch stats (an expensive operation) again from MapR DB client.
     final BinaryTableGroupScan newGroupsScan = new BinaryTableGroupScan(groupScan.getUserName(), groupScan.getStoragePlugin(),
-                                                              groupScan.getFormatPlugin(), newScanSpec, groupScan.getColumns());
+                                                                        groupScan.getFormatPlugin(), newScanSpec, groupScan.getColumns(),
+                                                                        groupScan.getTableStats());
     newGroupsScan.setFilterPushedDown(true);
 
     final ScanPrel newScanPrel = ScanPrel.create(scan, filter.getTraitSet(), newGroupsScan, scan.getRowType());

http://git-wip-us.apache.org/repos/asf/drill/blob/e03d7f9b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
----------------------------------------------------------------------
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
index 4eaeee7..c298456 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
@@ -89,6 +89,15 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
     init();
   }
 
+  public BinaryTableGroupScan(String userName, FileSystemPlugin storagePlugin,
+                              MapRDBFormatPlugin formatPlugin, HBaseScanSpec scanSpec,
+                              List<SchemaPath> columns, MapRDBTableStats tableStats) {
+    super(storagePlugin, formatPlugin, columns, userName);
+    this.hbaseScanSpec = scanSpec;
+    this.tableStats = tableStats;
+    init();
+  }
+
   /**
    * Private constructor, used for cloning.
    * @param that The HBaseGroupScan to clone
@@ -115,8 +124,10 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
     try (Admin admin = formatPlugin.getConnection().getAdmin();
          RegionLocator locator = formatPlugin.getConnection().getRegionLocator(tableName)) {
       hTableDesc = admin.getTableDescriptor(tableName);
-      tableStats = new MapRDBTableStats(getHBaseConf(), hbaseScanSpec.getTableName());
-
+      // Fetch tableStats only once and cache it.
+      if (tableStats == null) {
+        tableStats = new MapRDBTableStats(getHBaseConf(), hbaseScanSpec.getTableName());
+      }
       boolean foundStartRegion = false;
       regionsToScan = new TreeMap<TabletFragmentInfo, String>();
       List<HRegionLocation> regionLocations = locator.getAllRegionLocations();
@@ -197,6 +208,11 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
     return getHBaseScanSpec().getTableName();
   }
 
+  @JsonIgnore
+  public MapRDBTableStats getTableStats() {
+    return tableStats;
+  }
+
   @Override
   public String toString() {
     return "BinaryTableGroupScan [ScanSpec="


[7/9] drill git commit: DRILL-4971: Query encounters system error, when there aren't eval subexpressions of any function in boolean and/or expressions

Posted by jn...@apache.org.
DRILL-4971: Query encounters system error, when there aren't eval subexpressions of any function in boolean and/or expressions

- New evaluated blocks for boolean operators should be with braces always, since they use labels.

close #792


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/e0121d5f
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/e0121d5f
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/e0121d5f

Branch: refs/heads/master
Commit: e0121d5f953a9f4ecd71d74e97ddaa1a169a5102
Parents: 4c7e3a1
Author: Vitalii Diravka <vi...@gmail.com>
Authored: Fri Mar 17 11:41:46 2017 +0000
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Fri Mar 31 15:32:44 2017 -0700

----------------------------------------------------------------------
 .../apache/drill/exec/expr/ClassGenerator.java  | 20 +++++++++++++
 .../drill/exec/expr/EvaluationVisitor.java      | 11 ++++---
 .../java/org/apache/drill/TestBugFixes.java     | 31 ++++++++++++++++++++
 3 files changed, 58 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/e0121d5f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
index 0b6adaa..5310334 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
@@ -186,6 +186,26 @@ public class ClassGenerator<T>{
     return getEvalBlock().label(prefix + labelIndex ++);
   }
 
+  /**
+   * Creates an inner braced and indented block
+   * @param type type of the created block
+   * @return a newly created inner block
+   */
+  private JBlock createInnerBlock(BlockType type) {
+    final JBlock currBlock = getBlock(type);
+    final JBlock innerBlock = new JBlock();
+    currBlock.add(innerBlock);
+    return innerBlock;
+  }
+
+  /**
+   * Creates an inner braced and indented block for evaluation of the expression.
+   * @return a newly created inner eval block
+   */
+  protected JBlock createInnerEvalBlock() {
+    return createInnerBlock(BlockType.EVAL);
+  }
+
   public JVar declareVectorValueSetupAndMember(String batchName, TypedFieldId fieldId) {
     return declareVectorValueSetupAndMember(DirectExpression.direct(batchName), fieldId);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/e0121d5f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
index 53bd8b8..75b83c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -144,6 +144,9 @@ public class EvaluationVisitor {
     previousExpressions = mapStack.pop();
   }
 
+  /**
+   * Get a HoldingContainer for the expression if it had been already evaluated
+   */
   private HoldingContainer getPrevious(LogicalExpression expression, MappingSet mappingSet) {
     HoldingContainer previous = previousExpressions.get(new ExpressionHolder(expression, mappingSet));
     if (previous != null) {
@@ -671,8 +674,8 @@ public class EvaluationVisitor {
       HoldingContainer out = generator.declare(op.getMajorType());
 
       JLabel label = generator.getEvalBlockLabel("AndOP");
-      JBlock eval = generator.getEvalBlock().block();  // enter into nested block
-      generator.nestEvalBlock(eval);
+      JBlock eval = generator.createInnerEvalBlock();
+      generator.nestEvalBlock(eval);  // enter into nested block
 
       HoldingContainer arg = null;
 
@@ -733,7 +736,7 @@ public class EvaluationVisitor {
       HoldingContainer out = generator.declare(op.getMajorType());
 
       JLabel label = generator.getEvalBlockLabel("OrOP");
-      JBlock eval = generator.getEvalBlock().block();
+      JBlock eval = generator.createInnerEvalBlock();
       generator.nestEvalBlock(eval);   // enter into nested block.
 
       HoldingContainer arg = null;

http://git-wip-us.apache.org/repos/asf/drill/blob/e0121d5f/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java b/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
index 526a7e3..27df710 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
@@ -241,4 +241,35 @@ public class TestBugFixes extends BaseTestQuery {
     query.append("(CAST('1951-05-16' AS DATE))) tbl(dt)");
     test(query.toString());
   }
+
+  @Test // DRILL-4971
+  public void testVisitBooleanOrWithoutFunctionsEvaluation() throws Exception {
+    String query = "SELECT\n" +
+        "CASE WHEN employee_id IN (1) THEN 1 ELSE 0 END `first`\n" +
+        ", CASE WHEN employee_id IN (2) THEN 1 ELSE 0 END `second`\n" +
+        ", CASE WHEN employee_id IN (1, 2) THEN 1 ELSE 0 END `any`\n" +
+        "FROM cp.`employee.json` ORDER BY employee_id limit 2";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("first", "second", "any")
+        .baselineValues(1, 0, 1)
+        .baselineValues(0, 1, 1)
+        .go();
+  }
+
+  @Test // DRILL-4971
+  public void testVisitBooleanAndWithoutFunctionsEvaluation() throws Exception {
+    String query = "SELECT employee_id FROM cp.`employee.json` WHERE\n" +
+        "((employee_id > 1 AND employee_id < 3) OR (employee_id > 9 AND employee_id < 11))\n" +
+        "AND (employee_id > 1 AND employee_id < 3)";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("employee_id")
+        .baselineValues((long) 2)
+        .go();
+  }
 }


[6/9] drill git commit: DRILL-5297: when the generated plan mismatches, PlanTest print the generated plan along with expected pattern

Posted by jn...@apache.org.
DRILL-5297: when the generated plan mismatches, PlanTest print the generated plan along with expected pattern

close #798


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/4c7e3a19
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/4c7e3a19
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/4c7e3a19

Branch: refs/heads/master
Commit: 4c7e3a194ca8055980420f201f8df4ea0f9e4cfc
Parents: 152c87a
Author: chunhui-shi <cs...@maprtech.com>
Authored: Fri Mar 24 18:40:15 2017 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Fri Mar 31 15:32:34 2017 -0700

----------------------------------------------------------------------
 exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java | 4 ++--
 .../java/org/apache/drill/TestPlanVerificationUtilities.java    | 5 +++--
 2 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4c7e3a19/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
index 5d1cdbf..e422a77 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
@@ -86,7 +86,7 @@ public class PlanTestBase extends BaseTestQuery {
       for (final String s : expectedPatterns) {
         final Pattern p = Pattern.compile(s);
         final Matcher m = p.matcher(plan);
-        assertTrue(EXPECTED_NOT_FOUND + s, m.find());
+        assertTrue(EXPECTED_NOT_FOUND + s +"\n" + plan, m.find());
       }
     }
 
@@ -95,7 +95,7 @@ public class PlanTestBase extends BaseTestQuery {
       for (final String s : excludedPatterns) {
         final Pattern p = Pattern.compile(s);
         final Matcher m = p.matcher(plan);
-        assertFalse(UNEXPECTED_FOUND + s, m.find());
+        assertFalse(UNEXPECTED_FOUND + s +"\n" + plan, m.find());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/4c7e3a19/exec/java-exec/src/test/java/org/apache/drill/TestPlanVerificationUtilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestPlanVerificationUtilities.java b/exec/java-exec/src/test/java/org/apache/drill/TestPlanVerificationUtilities.java
index 1d8ebc7..4accb36 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestPlanVerificationUtilities.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestPlanVerificationUtilities.java
@@ -20,6 +20,7 @@ package org.apache.drill;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class TestPlanVerificationUtilities extends PlanTestBase {
 
@@ -41,14 +42,14 @@ public class TestPlanVerificationUtilities extends PlanTestBase {
       testPlanMatchingPatterns(query,
           new String[] {expectedPattern}, new String[] {expectedPattern});
     } catch (AssertionError ex) {
-      assertEquals(ex.getMessage(), UNEXPECTED_FOUND + expectedPattern);
+      assertTrue(ex.getMessage().contains(UNEXPECTED_FOUND));
     }
 
     try {
       testPlanMatchingPatterns(query,
           new String[] {excludedPattern}, new String[] {excludedPattern});
     } catch (AssertionError ex) {
-      assertEquals(ex.getMessage(), EXPECTED_NOT_FOUND + excludedPattern);
+      assertTrue(ex.getMessage().contains(EXPECTED_NOT_FOUND));
     }
   }
 }


[5/9] drill git commit: DRILL-5351: Minimize bounds checking in var len vectors for Parquet reader

Posted by jn...@apache.org.
DRILL-5351: Minimize bounds checking in var len vectors for Parquet reader

close #781


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/152c87aa
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/152c87aa
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/152c87aa

Branch: refs/heads/master
Commit: 152c87aa6cad84c8752b4a87967c7826cc90dbaa
Parents: 0ded1d0
Author: Parth Chandra <pc...@maprtech.com>
Authored: Fri Feb 10 17:40:25 2017 -0800
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Fri Mar 31 15:32:13 2017 -0700

----------------------------------------------------------------------
 .../codegen/templates/NullableValueVectors.java | 24 +++++--
 .../templates/VariableLengthVectors.java        | 72 +++++++++++++-------
 2 files changed, 65 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/152c87aa/exec/vector/src/main/codegen/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/NullableValueVectors.java b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
index b242728..fcbe79a 100644
--- a/exec/vector/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
@@ -509,7 +509,9 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       <#if type.major != "VarLen">
       throw new UnsupportedOperationException();
       <#else>
-      fillEmpties(index);
+      if (index > lastSet + 1) {
+        fillEmpties(index);
+      }
 
       bits.getMutator().setSafe(index, 1);
       values.getMutator().setSafe(index, value, start, length);
@@ -522,7 +524,9 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       <#if type.major != "VarLen">
       throw new UnsupportedOperationException();
       <#else>
-      fillEmpties(index);
+      if (index > lastSet + 1) {
+        fillEmpties(index);
+      }
 
       bits.getMutator().setSafe(index, 1);
       values.getMutator().setSafe(index, value, start, length);
@@ -587,7 +591,9 @@ public final class ${className} extends BaseDataValueVector implements <#if type
 
     public void setSafe(int index, int isSet<#list fields as field><#if field.include!true >, ${field.type} ${field.name}Field</#if></#list> ) {
       <#if type.major == "VarLen">
-      fillEmpties(index);
+      if (index > lastSet + 1) {
+        fillEmpties(index);
+      }
       </#if>
 
       bits.getMutator().setSafe(index, isSet);
@@ -600,7 +606,9 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     public void setSafe(int index, Nullable${minor.class}Holder value) {
 
       <#if type.major == "VarLen">
-      fillEmpties(index);
+      if (index > lastSet + 1) {
+        fillEmpties(index);
+      }
       </#if>
       bits.getMutator().setSafe(index, value.isSet);
       values.getMutator().setSafe(index, value);
@@ -611,7 +619,9 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     public void setSafe(int index, ${minor.class}Holder value) {
 
       <#if type.major == "VarLen">
-      fillEmpties(index);
+      if (index > lastSet + 1) {
+        fillEmpties(index);
+      }
       </#if>
       bits.getMutator().setSafe(index, 1);
       values.getMutator().setSafe(index, value);
@@ -622,7 +632,9 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     <#if !(type.major == "VarLen" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense" || minor.class == "Interval" || minor.class == "IntervalDay")>
       public void setSafe(int index, ${minor.javaType!type.javaType} value) {
         <#if type.major == "VarLen">
-        fillEmpties(index);
+        if (index > lastSet + 1) {
+          fillEmpties(index);
+        }
         </#if>
         bits.getMutator().setSafe(index, 1);
         values.getMutator().setSafe(index, value);

http://git-wip-us.apache.org/repos/asf/drill/blob/152c87aa/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
index bb1d4fb..e970271 100644
--- a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
+++ b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
@@ -507,11 +507,15 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       assert index >= 0;
 
       final int currentOffset = offsetVector.getAccessor().get(index);
-      while (data.capacity() < currentOffset + bytes.length) {
-        reAlloc();
-      }
       offsetVector.getMutator().setSafe(index + 1, currentOffset + bytes.length);
-      data.setBytes(currentOffset, bytes, 0, bytes.length);
+      try {
+        data.setBytes(currentOffset, bytes, 0, bytes.length);
+      } catch (IndexOutOfBoundsException e) {
+        while (data.capacity() < currentOffset + bytes.length) {
+          reAlloc();
+        }
+        data.setBytes(currentOffset, bytes, 0, bytes.length);
+      }
     }
 
     /**
@@ -533,12 +537,15 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       assert index >= 0;
 
       int currentOffset = offsetVector.getAccessor().get(index);
-
-      while (data.capacity() < currentOffset + length) {
-        reAlloc();
-      }
       offsetVector.getMutator().setSafe(index + 1, currentOffset + length);
-      data.setBytes(currentOffset, bytes, start, length);
+      try {
+        data.setBytes(currentOffset, bytes, start, length);
+      } catch (IndexOutOfBoundsException e) {
+        while (data.capacity() < currentOffset + length) {
+          reAlloc();
+        }
+        data.setBytes(currentOffset, bytes, start, length);
+      }
     }
 
     public void setSafe(int index, byte[] bytes, int start, int length) {
@@ -546,11 +553,15 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
 
       final int currentOffset = offsetVector.getAccessor().get(index);
 
-      while (data.capacity() < currentOffset + length) {
-        reAlloc();
-      }
       offsetVector.getMutator().setSafe(index + 1, currentOffset + length);
-      data.setBytes(currentOffset, bytes, start, length);
+      try {
+        data.setBytes(currentOffset, bytes, start, length);
+      } catch (IndexOutOfBoundsException e) {
+        while (data.capacity() < currentOffset + length) {
+          reAlloc();
+        }
+        data.setBytes(currentOffset, bytes, start, length);
+      }
     }
 
     @Override
@@ -567,12 +578,16 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       final int len = end - start;
       final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
 
-      while(data.capacity() < outputStart + len) {
-        reAlloc();
+      offsetVector.getMutator().setSafe( index+1,  outputStart + len);
+      try{
+        buffer.getBytes(start, data, outputStart, len);
+      } catch (IndexOutOfBoundsException e) {
+        while (data.capacity() < outputStart + len) {
+          reAlloc();
+        }
+        buffer.getBytes(start, data, outputStart, len);
       }
 
-      offsetVector.getMutator().setSafe( index+1,  outputStart + len);
-      buffer.getBytes(start, data, outputStart, len);
     }
 
     public void setSafe(int index, Nullable${minor.class}Holder holder){
@@ -584,11 +599,14 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
 
       int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
 
-      while(data.capacity() < outputStart + len) {
-        reAlloc();
+      try {
+        holder.buffer.getBytes(start, data, outputStart, len);
+      } catch (IndexOutOfBoundsException e) {
+        while (data.capacity() < outputStart + len) {
+          reAlloc();
+        }
+        holder.buffer.getBytes(start, data, outputStart, len);
       }
-
-      holder.buffer.getBytes(start, data, outputStart, len);
       offsetVector.getMutator().setSafe( index+1,  outputStart + len);
     }
 
@@ -598,11 +616,15 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       final int len = end - start;
       final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
 
-      while(data.capacity() < outputStart + len) {
-        reAlloc();
-      }
 
-      holder.buffer.getBytes(start, data, outputStart, len);
+      try {
+        holder.buffer.getBytes(start, data, outputStart, len);
+      } catch (IndexOutOfBoundsException e) {
+        while(data.capacity() < outputStart + len) {
+          reAlloc();
+        }
+        holder.buffer.getBytes(start, data, outputStart, len);
+      }
       offsetVector.getMutator().setSafe( index+1,  outputStart + len);
     }
 


[3/9] drill git commit: DRILL-5369: Add initializer for ServerMetaContext

Posted by jn...@apache.org.
DRILL-5369: Add initializer for ServerMetaContext

ServerMetaContext had no default constructor. The lack of it
might cause m_done to be set to true, same for other variables.

Add a default constructor to explicitly initialize its members.

close #791


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/b82177ed
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/b82177ed
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/b82177ed

Branch: refs/heads/master
Commit: b82177ed612edd0e33ac1d5500a7dd65953c8c1d
Parents: 70f85e5
Author: Laurent Goujon <la...@dremio.com>
Authored: Mon Mar 20 11:46:58 2017 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Fri Mar 31 15:31:57 2017 -0700

----------------------------------------------------------------------
 contrib/native/client/src/clientlib/drillClientImpl.cpp | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/b82177ed/contrib/native/client/src/clientlib/drillClientImpl.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.cpp b/contrib/native/client/src/clientlib/drillClientImpl.cpp
index 9d45f57..7e169e9 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.cpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.cpp
@@ -1799,6 +1799,8 @@ struct ServerMetaContext {
 	boost::mutex m_mutex;
 	boost::condition_variable m_cv;
 
+    ServerMetaContext(): m_done(false), m_status(QRY_SUCCESS), m_serverMeta(), m_mutex(), m_cv() {};
+
 	static status_t listener(void* ctx, const exec::user::ServerMeta* serverMeta, DrillClientError* err) {
 		ServerMetaContext* context = static_cast<ServerMetaContext*>(ctx);
 			if (err) {


[2/9] drill git commit: DRILL-5378: Put more information for schema change exception in hash join, hash agg, streaming agg and sort operator.

Posted by jn...@apache.org.
DRILL-5378: Put more information for schema change exception in hash join, hash agg, streaming agg and sort operator.

close #801


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/70f85e57
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/70f85e57
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/70f85e57

Branch: refs/heads/master
Commit: 70f85e57884fa129f05bec7d124d551f4938b2d9
Parents: e03d7f9
Author: Jinfeng Ni <jn...@apache.org>
Authored: Wed Mar 22 15:28:22 2017 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Fri Mar 31 15:31:50 2017 -0700

----------------------------------------------------------------------
 .../exec/exception/SchemaChangeException.java      | 14 ++++++++++++++
 .../exec/physical/impl/aggregate/HashAggBatch.java |  9 ++++++++-
 .../physical/impl/aggregate/StreamingAggBatch.java |  5 ++++-
 .../exec/physical/impl/join/HashJoinBatch.java     | 17 ++++++++++++++---
 .../physical/impl/join/HashJoinProbeTemplate.java  |  4 +++-
 .../physical/impl/xsort/ExternalSortBatch.java     |  4 +++-
 6 files changed, 46 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/70f85e57/exec/java-exec/src/main/java/org/apache/drill/exec/exception/SchemaChangeException.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/exception/SchemaChangeException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/exception/SchemaChangeException.java
index 64c469e..acb4913 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/exception/SchemaChangeException.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/exception/SchemaChangeException.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.exception;
 
 import org.apache.drill.common.exceptions.DrillException;
+import org.apache.drill.exec.record.BatchSchema;
 
 public class SchemaChangeException extends DrillException{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SchemaChangeException.class);
@@ -49,4 +50,17 @@ public class SchemaChangeException extends DrillException{
   public SchemaChangeException(String message, Throwable cause, Object...objects){
     super(String.format(message, objects), cause);
   }
+
+  public static SchemaChangeException schemaChanged(String message, BatchSchema priorSchema, BatchSchema newSchema) {
+    final String errorMsg = new StringBuilder()
+        .append(message)
+        .append("\n")
+        .append("Prior schema : \n")
+        .append(priorSchema.toString())
+        .append("\n")
+        .append("New schema : \n")
+        .append(newSchema.toString())
+        .toString();
+    return new SchemaChangeException(errorMsg);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/70f85e57/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
index 623c58b..dc913b1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
@@ -44,6 +44,7 @@ import org.apache.drill.exec.physical.impl.common.Comparator;
 import org.apache.drill.exec.physical.impl.common.HashTable;
 import org.apache.drill.exec.physical.impl.common.HashTableConfig;
 import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
@@ -67,6 +68,7 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
   private TypedFieldId[] groupByOutFieldIds;
   private TypedFieldId[] aggrOutFieldIds;      // field ids for the outgoing batch
   private final List<Comparator> comparators;
+  private BatchSchema incomingSchema;
 
   private final GeneratorMapping UPDATE_AGGR_INSIDE =
       GeneratorMapping.create("setupInterior" /* setup method */, "updateAggrValuesInternal" /* eval method */,
@@ -118,6 +120,7 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
         return;
     }
 
+    this.incomingSchema = incoming.getSchema();
     if (!createAggregator()) {
       state = BatchState.DONE;
     }
@@ -152,7 +155,11 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
       return aggregator.getOutcome();
     case UPDATE_AGGREGATOR:
       context.fail(UserException.unsupportedError()
-        .message("Hash aggregate does not support schema changes").build(logger));
+          .message(SchemaChangeException.schemaChanged(
+              "Hash aggregate does not support schema change",
+              incomingSchema,
+              incoming.getSchema()).getMessage())
+          .build(logger));
       close();
       killIncoming(false);
       return IterOutcome.STOP;

http://git-wip-us.apache.org/repos/asf/drill/blob/70f85e57/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
index 420851a..af41438 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
@@ -44,6 +44,7 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.StreamingAggregate;
 import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator.AggOutcome;
 import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
@@ -66,6 +67,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
   private boolean done = false;
   private boolean first = true;
   private int recordCount = 0;
+  private BatchSchema incomingSchema;
 
   /*
    * DRILL-2277, DRILL-2411: For straight aggregates without a group by clause we need to perform special handling when
@@ -111,6 +113,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
         return;
     }
 
+    this.incomingSchema = incoming.getSchema();
     if (!createAggregator()) {
       state = BatchState.DONE;
     }
@@ -188,7 +191,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
       return outcome;
     case UPDATE_AGGREGATOR:
       context.fail(UserException.unsupportedError()
-        .message("Streaming aggregate does not support schema changes")
+        .message(SchemaChangeException.schemaChanged("Streaming aggregate does not support schema changes", incomingSchema, incoming.getSchema()).getMessage())
         .build(logger));
       close();
       killIncoming(false);

http://git-wip-us.apache.org/repos/asf/drill/blob/70f85e57/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
index f1f81fb..e2c016b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
@@ -299,7 +299,12 @@ public class HashJoinBatch extends AbstractRecordBatch<HashJoinPOP> {
       leftExpr = null;
     } else {
       if (left.getSchema().getSelectionVectorMode() != BatchSchema.SelectionVectorMode.NONE) {
-        throw new SchemaChangeException("Hash join does not support probe batch with selection vectors");
+        final String errorMsg = new StringBuilder()
+            .append("Hash join does not support probe batch with selection vectors. ")
+            .append("Probe batch has selection mode = ")
+            .append(left.getSchema().getSelectionVectorMode())
+            .toString();
+        throw new SchemaChangeException(errorMsg);
       }
     }
 
@@ -340,12 +345,18 @@ public class HashJoinBatch extends AbstractRecordBatch<HashJoinPOP> {
           rightSchema = right.getSchema();
 
           if (rightSchema.getSelectionVectorMode() != BatchSchema.SelectionVectorMode.NONE) {
-            throw new SchemaChangeException("Hash join does not support build batch with selection vectors");
+            final String errorMsg = new StringBuilder()
+                .append("Hash join does not support build batch with selection vectors. ")
+                .append("Build batch has selection mode = ")
+                .append(left.getSchema().getSelectionVectorMode())
+                .toString();
+
+            throw new SchemaChangeException(errorMsg);
           }
           setupHashTable();
         } else {
           if (!rightSchema.equals(right.getSchema())) {
-            throw new SchemaChangeException("Hash join does not support schema changes");
+            throw SchemaChangeException.schemaChanged("Hash join does not support schema changes in build side.", rightSchema, right.getSchema());
           }
           hashTable.updateBatches();
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/70f85e57/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
index 5531bc7..3cdce2f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
@@ -138,7 +138,9 @@ public abstract class HashJoinProbeTemplate implements HashJoinProbe {
               doSetup(outgoingJoinBatch.getContext(), buildBatch, probeBatch, outgoingJoinBatch);
               hashTable.updateBatches();
             } else {
-              throw new SchemaChangeException("Hash join does not support schema changes");
+              throw SchemaChangeException.schemaChanged("Hash join does not support schema changes in probe side.",
+                  probeSchema,
+                  probeBatch.getSchema());
             }
           case OK:
             recordsToProcess = probeBatch.getRecordCount();

http://git-wip-us.apache.org/repos/asf/drill/blob/70f85e57/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index c009cc0..25f05b3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -346,7 +346,9 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
               if (unionTypeEnabled) {
                 this.schema = SchemaUtil.mergeSchemas(schema, incoming.getSchema());
               } else {
-                throw new SchemaChangeException("Schema changes not supported in External Sort. Please enable Union type");
+                throw SchemaChangeException.schemaChanged("Schema changes not supported in External Sort. Please enable Union type",
+                    schema,
+                    incoming.getSchema());
               }
             } else {
               schema = incoming.getSchema();


[9/9] drill git commit: DRILL-5373: Drill JDBC error in the process of connection via SQuirrel

Posted by jn...@apache.org.
DRILL-5373: Drill JDBC error in the process of connection via SQuirrel

 -  java.lang.NoClassDefFoundError: javax/validation/constraints/NotNull


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/d3718a62
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/d3718a62
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/d3718a62

Branch: refs/heads/master
Commit: d3718a62e2315a601615db8803cdfdcc3cedab82
Parents: b17019e
Author: Vitalii Diravka <vi...@gmail.com>
Authored: Thu Mar 16 13:45:36 2017 +0000
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Fri Mar 31 15:34:27 2017 -0700

----------------------------------------------------------------------
 exec/jdbc-all/pom.xml | 9 +++++++--
 exec/jdbc/pom.xml     | 5 +++++
 2 files changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/d3718a62/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index a395715..1b015e9 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -279,7 +279,6 @@
               <exclude>com.googlecode.json-simple:*</exclude>
               <exclude>dom4j:*</exclude>
               <exclude>org.hibernate:*</exclude>
-              <exclude>javax.validation:*</exclude>
               <exclude>antlr:*</exclude>
               <exclude>org.ow2.asm:*</exclude>
               <exclude>com.univocity:*</exclude>
@@ -391,7 +390,13 @@
                <exclude>**/*.SF</exclude>
                <exclude>**/*.RSA</exclude>
                <exclude>**/*.DSA</exclude>
-               <exclude>javax/**</exclude>
+               <exclude>javax/*</exclude>
+               <exclude>javax/activation/**</exclude>
+               <exclude>javax/annotation-api/**</exclude>
+               <exclude>javax/inject/**</exclude>
+               <exclude>javax/servlet-api/**</exclude>
+               <exclude>javax/json/**</exclude>
+               <exclude>javax/ws/**</exclude>
                <exclude>rest/**</exclude>
                <exclude>*.tokens</exclude>
                <exclude>codegen/**</exclude>

http://git-wip-us.apache.org/repos/asf/drill/blob/d3718a62/exec/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc/pom.xml b/exec/jdbc/pom.xml
index 9e4e411..cb0c517 100644
--- a/exec/jdbc/pom.xml
+++ b/exec/jdbc/pom.xml
@@ -95,6 +95,11 @@
       <groupId>xalan</groupId>
       <artifactId>xalan</artifactId>
     </dependency>
+    <dependency>
+      <groupId>javax.validation</groupId>
+      <artifactId>validation-api</artifactId>
+      <version>1.1.0.Final</version>
+    </dependency>
   </dependencies>
 
   <build>


[4/9] drill git commit: DRILL-5368: Fix memory leak issue in DrillClientImpl::processServerMetaResult

Posted by jn...@apache.org.
DRILL-5368: Fix memory leak issue in DrillClientImpl::processServerMetaResult

Fix a small memory leak by doing local allocation instead since the
object doesn't escape the function.

close #790


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/0ded1d08
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/0ded1d08
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/0ded1d08

Branch: refs/heads/master
Commit: 0ded1d08297cf56c8e38f98a8c1cf3708c3a584c
Parents: b82177e
Author: Laurent Goujon <la...@dremio.com>
Authored: Mon Mar 20 10:55:17 2017 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Fri Mar 31 15:32:06 2017 -0700

----------------------------------------------------------------------
 contrib/native/client/src/clientlib/drillClientImpl.cpp | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/0ded1d08/contrib/native/client/src/clientlib/drillClientImpl.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.cpp b/contrib/native/client/src/clientlib/drillClientImpl.cpp
index 7e169e9..30a354e 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.cpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.cpp
@@ -1364,15 +1364,15 @@ status_t DrillClientImpl::processServerMetaResult(AllocatedBufferPtr allocatedBu
     std::map<int,DrillClientQueryHandle*>::const_iterator it=this->m_queryHandles.find(msg.m_coord_id);
     if(it!=this->m_queryHandles.end()){
         DrillClientServerMetaHandle* pHandle=static_cast<DrillClientServerMetaHandle*>((*it).second);
-        exec::user::GetServerMetaResp* resp = new exec::user::GetServerMetaResp();
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE)  << "Received GetServerMetaResp result Handle " << msg.m_pbody.size() << std::endl;)
-        if (!(resp->ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size()))) {
+        exec::user::GetServerMetaResp resp;
+        if (!(resp.ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size()))) {
             return handleQryError(QRY_COMM_ERROR, "Cannot decode GetServerMetaResp results", pHandle);
         }
-        if (resp->status() != exec::user::OK) {
-            return handleQryError(QRY_FAILED, resp->error(), pHandle);
+        if (resp.status() != exec::user::OK) {
+            return handleQryError(QRY_FAILED, resp.error(), pHandle);
         }
-        pHandle->notifyListener(&(resp->server_meta()), NULL);
+        pHandle->notifyListener(&(resp.server_meta()), NULL);
         DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "GetServerMetaResp result " << std::endl;)
     }else{
         return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);


[8/9] drill git commit: DRILL-3510: Add ANSI_QUOTES option so that Drill's SQL Parser will recognize ANSI_SQL identifiers

Posted by jn...@apache.org.
DRILL-3510: Add ANSI_QUOTES option so that Drill's SQL Parser will recognize ANSI_SQL identifiers

- added supporing of quoting identifiers with DOUBLE_QUOTES or BRACKETS via setting new
  sys/sess EnumString option QUOTING_IDENTIFIERS;
- added possibility of setting QUOTING_IDENTIFIERS by the jdbc connection URL string;
- added relevant unit tests;

close #520


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/b17019e2
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/b17019e2
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/b17019e2

Branch: refs/heads/master
Commit: b17019e2c4c1d185e418c9b4ad2fec67817f7bc3
Parents: e0121d5
Author: Vitalii Diravka <vi...@gmail.com>
Authored: Fri May 20 20:11:33 2016 +0000
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Fri Mar 31 15:32:52 2017 -0700

----------------------------------------------------------------------
 .../drill/common/config/DrillProperties.java    |  5 +-
 .../apache/drill/exec/client/DrillClient.java   |  2 +-
 .../exec/planner/physical/PlannerSettings.java  | 26 +++++++-
 .../exec/planner/sql/DrillParserConfig.java     |  4 +-
 .../apache/drill/exec/rpc/user/UserSession.java | 30 +++++++--
 .../exec/server/options/OptionValidator.java    | 12 +++-
 .../drill/exec/server/options/OptionValue.java  |  2 +-
 .../server/options/SystemOptionManager.java     |  1 +
 .../exec/server/options/TypeValidators.java     | 12 +++-
 .../java/org/apache/drill/BaseTestQuery.java    |  6 +-
 .../impersonation/TestInboundImpersonation.java | 15 ++---
 .../exec/planner/sql/TestDrillSQLWorker.java    | 45 ++++++++++++-
 .../drill/jdbc/DrillDatabaseMetaData.java       | 16 +++--
 .../jdbc/impl/DrillDatabaseMetaDataImpl.java    |  6 +-
 .../apache/drill/jdbc/ConnectionFactory.java    |  3 +-
 .../apache/drill/jdbc/ConnectionInfoTest.java   | 70 ++++++++++++++++++++
 .../apache/drill/jdbc/DatabaseMetaDataTest.java | 11 +--
 .../org/apache/drill/jdbc/JdbcTestBase.java     |  9 ++-
 .../jdbc/MultiConnectionCachingFactory.java     |  4 +-
 .../jdbc/SingleConnectionCachingFactory.java    |  4 +-
 .../org/apache/drill/jdbc/test/JdbcAssert.java  |  4 +-
 21 files changed, 230 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/common/src/main/java/org/apache/drill/common/config/DrillProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/config/DrillProperties.java b/common/src/main/java/org/apache/drill/common/config/DrillProperties.java
index ba63349..c7e6e29 100644
--- a/common/src/main/java/org/apache/drill/common/config/DrillProperties.java
+++ b/common/src/main/java/org/apache/drill/common/config/DrillProperties.java
@@ -63,6 +63,8 @@ public final class DrillProperties extends Properties {
   // Subject's credentials set
   public static final String KERBEROS_FROM_SUBJECT = "from_subject";
 
+  public static final String QUOTING_IDENTIFIERS = "quoting_identifiers";
+
   // Although all properties from the application are sent to the server (from the client), the following
   // sets of properties are used by the client and server respectively. These are reserved words.
 
@@ -77,7 +79,8 @@ public final class DrillProperties extends Properties {
   public static final ImmutableSet<String> ACCEPTED_BY_SERVER = ImmutableSet.of(
       USER /** deprecated */, PASSWORD /** deprecated */,
       SCHEMA,
-      IMPERSONATION_TARGET
+      IMPERSONATION_TARGET,
+      QUOTING_IDENTIFIERS
   );
 
   private DrillProperties() {

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index 0ff6a5b..9fbbfdd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -527,7 +527,7 @@ public class DrillClient implements Closeable, ConnectionThrottle {
   /**
    * Returns the list of methods supported by the server based on its advertised information.
    *
-   * @return a immutable set of capabilities
+   * @return an immutable set of capabilities
    */
   public Set<ServerMethod> getSupportedMethods() {
     return client != null ? ServerMethod.getSupportedMethods(client.getSupportedMethods(), client.getServerInfos()) : null;

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
index b3dc7d6..da5bc41 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,13 +17,14 @@
  */
 package org.apache.drill.exec.planner.physical;
 
+import org.apache.calcite.avatica.util.Quoting;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValidator;
-import org.apache.drill.exec.server.options.TypeValidators;
 import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
+import org.apache.drill.exec.server.options.TypeValidators.EnumeratedStringValidator;
 import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
 import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
 import org.apache.drill.exec.server.options.TypeValidators.PositiveLongValidator;
@@ -105,6 +106,9 @@ public class PlannerSettings implements Context{
   public static final PositiveLongValidator PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD = new PositiveLongValidator(PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD_KEY,
       Long.MAX_VALUE, 10000);
 
+  public static final String QUOTING_IDENTIFIERS_KEY = "planner.parser.quoting_identifiers";
+  public static final EnumeratedStringValidator QUOTING_IDENTIFIERS = new EnumeratedStringValidator(
+      QUOTING_IDENTIFIERS_KEY, Quoting.BACK_TICK.string, Quoting.DOUBLE_QUOTE.string, Quoting.BRACKET.string);
 
   public OptionManager options = null;
   public FunctionImplementationRegistry functionImplementationRegistry = null;
@@ -262,6 +266,22 @@ public class PlannerSettings implements Context{
     return options.getOption(PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD);
   }
 
+  /**
+   * @return Quoting enum for current quoting identifiers character
+   */
+  public Quoting getQuotingIdentifiers() {
+    String quotingIdentifiersCharacter = options.getOption(QUOTING_IDENTIFIERS);
+    for (Quoting value : Quoting.values()) {
+      if (value.string.equals(quotingIdentifiersCharacter)) {
+        return value;
+      }
+    }
+    // this is never reached
+    throw UserException.validationError()
+        .message("Unknown quoting identifier character '%s'", quotingIdentifiersCharacter)
+        .build(logger);
+  }
+
   @Override
   public <T> T unwrap(Class<T> clazz) {
     if(clazz == PlannerSettings.class){

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserConfig.java
index 7e7b140..d8441ae 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserConfig.java
@@ -27,9 +27,11 @@ import org.apache.drill.exec.planner.sql.parser.impl.DrillParserWithCompoundIdCo
 public class DrillParserConfig implements SqlParser.Config {
 
   private final long identifierMaxLength;
+  private final Quoting quotingIdentifiers;
 
   public DrillParserConfig(PlannerSettings settings) {
     identifierMaxLength = settings.getIdentifierMaxLength();
+    quotingIdentifiers = settings.getQuotingIdentifiers();
   }
 
   @Override
@@ -49,7 +51,7 @@ public class DrillParserConfig implements SqlParser.Config {
 
   @Override
   public Quoting quoting() {
-    return Quoting.BACK_TICK;
+    return quotingIdentifiers;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
index 6d74c1d..fc647c8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
@@ -34,14 +34,17 @@ import org.apache.calcite.schema.Table;
 import org.apache.calcite.tools.ValidationException;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.planner.sql.SchemaUtilites;
 import org.apache.drill.exec.planner.sql.handlers.SqlHandlerUtil;
 import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
 import org.apache.drill.exec.proto.UserProtos.UserProperties;
 import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.server.options.SessionOptionManager;
 
 import com.google.common.collect.Maps;
+import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.StorageStrategy;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
@@ -52,11 +55,6 @@ import org.apache.hadoop.fs.Path;
 public class UserSession implements AutoCloseable {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserSession.class);
 
-  public static final String SCHEMA = "schema";
-  public static final String USER = "user";
-  public static final String PASSWORD = "password";
-  public static final String IMPERSONATION_TARGET = "impersonation_target";
-
   private boolean supportComplexTypes = false;
   private UserCredentials credentials;
   private DrillProperties properties;
@@ -125,6 +123,15 @@ public class UserSession implements AutoCloseable {
     }
 
     public UserSession build() {
+      if (userSession.properties.containsKey(DrillProperties.QUOTING_IDENTIFIERS)) {
+        if (userSession.sessionOptions != null) {
+          userSession.setSessionOption(PlannerSettings.QUOTING_IDENTIFIERS_KEY,
+              userSession.properties.getProperty(DrillProperties.QUOTING_IDENTIFIERS));
+        } else {
+          logger.warn("User property {} can't be installed as a server option without the session option manager",
+              DrillProperties.QUOTING_IDENTIFIERS);
+        }
+      }
       UserSession session = userSession;
       userSession = null;
       return session;
@@ -232,8 +239,17 @@ public class UserSession implements AutoCloseable {
     return SchemaUtilites.findSchema(rootSchema, defaultSchemaPath);
   }
 
-  public boolean setSessionOption(String name, String value) {
-    return true;
+  /**
+   * Set the option of a session level.
+   * Note: Option's kind is automatically detected if such option exists.
+   *
+   * @param name option name
+   * @param value option value
+   */
+  public void setSessionOption(String name, String value) {
+    OptionValue.Kind optionKind = SystemOptionManager.getValidator(name).getKind();
+    OptionValue optionValue = OptionValue.createOption(optionKind, OptionValue.OptionType.SESSION, name, value);
+    sessionOptions.setOption(optionValue);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
index 82f4ab9..5ab9644 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValidator.java
@@ -1,4 +1,4 @@
-/*******************************************************************************
+/*
  * 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
@@ -14,10 +14,11 @@
  * 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.drill.exec.server.options;
 
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.server.options.OptionValue.Kind;
 
 /**
  * Validates the values provided to Drill options.
@@ -98,4 +99,11 @@ public abstract class OptionValidator {
    */
   public abstract void validate(OptionValue value, OptionManager manager);
 
+  /**
+   * Gets the kind of this option value for this validator.
+   *
+   * @return kind of this option value
+   */
+  public abstract Kind getKind();
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValue.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValue.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValue.java
index 96fdf0c..1f572f9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValue.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionValue.java
@@ -200,6 +200,6 @@ public class OptionValue implements Comparable<OptionValue> {
 
   @Override
   public String toString() {
-    return "OptionValue [type=" + type + ", name=" + name + ", value=" + getValue() + "]";
+    return "OptionValue [ type=" + type + ", name=" + name + ", value=" + getValue() + " ]";
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index fa73e06..09c5259 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -91,6 +91,7 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       PlannerSettings.UNIONALL_DISTRIBUTE,
       PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING,
       PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD,
+      PlannerSettings.QUOTING_IDENTIFIERS,
       ExecConstants.CAST_TO_NULLABLE_NUMERIC_OPTION,
       ExecConstants.OUTPUT_FORMAT_VALIDATOR,
       ExecConstants.PARQUET_BLOCK_SIZE_VALIDATOR,

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
index b4074ab..de960f3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,9 +17,9 @@
  */
 package org.apache.drill.exec.server.options;
 
-import java.util.HashSet;
 import java.util.Set;
 
+import com.google.common.collect.Sets;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.server.options.OptionValue.Kind;
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
@@ -204,10 +204,11 @@ public class TypeValidators {
    * Validator that checks if the given value is included in a list of acceptable values. Case insensitive.
    */
   public static class EnumeratedStringValidator extends StringValidator {
-    private final Set<String> valuesSet = new HashSet<>();
+    private final Set<String> valuesSet = Sets.newLinkedHashSet();
 
     public EnumeratedStringValidator(String name, String def, String... values) {
       super(name, def);
+      valuesSet.add(def.toLowerCase());
       for (String value : values) {
         valuesSet.add(value.toLowerCase());
       }
@@ -258,5 +259,10 @@ public class TypeValidators {
             .build(logger);
       }
     }
+
+    @Override
+    public Kind getKind() {
+      return kind;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
index 19ed5e9..4401b9f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
@@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.drill.DrillTestWrapper.TestServices;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.config.DrillProperties;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.scanner.ClassPathScanner;
 import org.apache.drill.common.scanner.persistence.ScanResult;
@@ -51,7 +52,6 @@ import org.apache.drill.exec.rpc.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.AwaitableUserResultsListener;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
-import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.RemoteServiceSet;
@@ -237,9 +237,9 @@ public class BaseTestQuery extends ExecTest {
    */
   public static void updateClient(final String user, final String password) throws Exception {
     final Properties props = new Properties();
-    props.setProperty(UserSession.USER, user);
+    props.setProperty(DrillProperties.USER, user);
     if (password != null) {
-      props.setProperty(UserSession.PASSWORD, password);
+      props.setProperty(DrillProperties.PASSWORD, password);
     }
     updateClient(props);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java
index df33f2b..1997470 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -25,7 +25,6 @@ import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.dotdrill.DotDrillType;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl;
 import org.apache.drill.exec.store.dfs.WorkspaceConfig;
 import org.apache.drill.test.UserExceptionMatcher;
@@ -128,9 +127,9 @@ public class TestInboundImpersonation extends BaseTestImpersonation {
     // Connect as PROXY_NAME and query for IMPERSONATION_TARGET
     // data belongs to OWNER, however a view is shared with IMPERSONATION_TARGET
     final Properties connectionProps = new Properties();
-    connectionProps.setProperty(UserSession.USER, PROXY_NAME);
-    connectionProps.setProperty(UserSession.PASSWORD, PROXY_PASSWORD);
-    connectionProps.setProperty(UserSession.IMPERSONATION_TARGET, TARGET_NAME);
+    connectionProps.setProperty(DrillProperties.USER, PROXY_NAME);
+    connectionProps.setProperty(DrillProperties.PASSWORD, PROXY_PASSWORD);
+    connectionProps.setProperty(DrillProperties.IMPERSONATION_TARGET, TARGET_NAME);
     updateClient(connectionProps);
 
     testBuilder()
@@ -146,9 +145,9 @@ public class TestInboundImpersonation extends BaseTestImpersonation {
   public void unauthorizedTarget() throws Exception {
     final String unauthorizedTarget = org2Users[0];
     final Properties connectionProps = new Properties();
-    connectionProps.setProperty(UserSession.USER, PROXY_NAME);
-    connectionProps.setProperty(UserSession.PASSWORD, PROXY_PASSWORD);
-    connectionProps.setProperty(UserSession.IMPERSONATION_TARGET, unauthorizedTarget);
+    connectionProps.setProperty(DrillProperties.USER, PROXY_NAME);
+    connectionProps.setProperty(DrillProperties.PASSWORD, PROXY_PASSWORD);
+    connectionProps.setProperty(DrillProperties.IMPERSONATION_TARGET, unauthorizedTarget);
     updateClient(connectionProps); // throws up
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/TestDrillSQLWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/TestDrillSQLWorker.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/TestDrillSQLWorker.java
index 3d85e2e..3a6a7de 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/TestDrillSQLWorker.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/TestDrillSQLWorker.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -19,10 +19,13 @@ package org.apache.drill.exec.planner.sql;
 
 import static org.junit.Assert.assertEquals;
 
+import org.apache.calcite.avatica.util.Quoting;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.junit.Test;
 
-public class TestDrillSQLWorker {
+public class TestDrillSQLWorker extends BaseTestQuery {
 
   private void validateFormattedIs(String sql, SqlParserPos pos, String expected) {
     String formatted = SqlConverter.formatSQLParsingError(sql, pos);
@@ -48,4 +51,42 @@ public class TestDrillSQLWorker {
     validateFormattedIs(sql, new SqlParserPos(0, 10), sql);
     validateFormattedIs(sql, new SqlParserPos(100, 10), sql);
   }
+
+  @Test
+  public void testDoubleQuotesForQuotingIdentifiers() throws Exception {
+    try {
+      test("ALTER SESSION SET `%s` = '%s'", PlannerSettings.QUOTING_IDENTIFIERS_KEY,
+          Quoting.DOUBLE_QUOTE.string);
+      testBuilder()
+          .sqlQuery("select \"employee_id\", \"full_name\" from cp.\"employee.json\" limit 1")
+          .ordered()
+          .baselineColumns("employee_id", "full_name")
+          .baselineValues(1L, "Sheri Nowmer")
+          .go();
+
+      // Other quoting characters are not acceptable while particular one is chosen,
+      // since calcite doesn't support parsing sql statements with several quoting identifiers characters
+      errorMsgTestHelper("select `employee_id`, `full_name` from cp.`employee.json` limit 1", "Encountered: \"`\"");
+      // Mix of different quotes in the one SQL statement is not acceptable
+      errorMsgTestHelper("select \"employee_id\", \"full_name\" from cp.`employee.json` limit 1", "Encountered: \"`\"");
+    } finally {
+      test("ALTER SESSION RESET %s", PlannerSettings.QUOTING_IDENTIFIERS_KEY);
+    }
+  }
+
+  @Test
+  public void testBracketsForQuotingIdentifiers() throws Exception {
+    try {
+      test("ALTER SESSION SET `%s` = '%s'", PlannerSettings.QUOTING_IDENTIFIERS_KEY,
+          Quoting.BRACKET.string);
+      testBuilder()
+          .sqlQuery("select [employee_id], [full_name] from cp.[employee.json] limit 1")
+          .ordered()
+          .baselineColumns("employee_id", "full_name")
+          .baselineValues(1L, "Sheri Nowmer")
+          .go();
+    } finally {
+      test("ALTER SESSION RESET %s", PlannerSettings.QUOTING_IDENTIFIERS_KEY);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java
index 64908d0..81027c6 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillDatabaseMetaData.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -19,6 +19,7 @@ package org.apache.drill.jdbc;
 import java.sql.DatabaseMetaData;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import org.apache.calcite.avatica.util.Quoting;
 
 
 /**
@@ -80,13 +81,16 @@ public interface DrillDatabaseMetaData extends DatabaseMetaData {
   //  storesLowerCaseQuotedIdentifiers()
   //  storesMixedCaseQuotedIdentifiers()
 
-
-  // TODO(DRILL-3510):  Update when Drill accepts standard SQL's double quote.
   /**
    * <strong>Drill</strong>:
-   * Reports that the SQL identifier quoting character is the back-quote
-   * character ("{@code `}"; Unicode U+0060; "GRAVE ACCENT").
-   * @return "{@code `}"
+   * Reports current SQL identifier quoting character.
+   *  <li>{@link Quoting#BACK_TICK} - default back-quote character ("{@code `}"; Unicode U+0060; "GRAVE ACCENT") </li>
+   *  <li>{@link Quoting#DOUBLE_QUOTE} - double quote character ("{@code "}"; Unicode U+0022; 'QUOTATION MARK')</li>
+   *  <li>{@link Quoting#BRACKET} - brackets characters ("{@code [}"; Unicode U+005B; 'LEFT SQUARE BRACKET' and
+   *  "{@code ]}"; Unicode U+005D; 'RIGHT SQUARE BRACKET')</li>
+   *
+   * @return current SQL identifier quoting character. Note: 'LEFT SQUARE BRACKET' is returned,
+   *         when {@link Quoting#BRACKET} is set.
    */
   @Override
   String getIdentifierQuoteString() throws SQLException;

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
index 14ffe5f..b076385 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -28,6 +28,7 @@ import java.util.Set;
 import java.util.concurrent.ExecutionException;
 
 import org.apache.calcite.avatica.AvaticaDatabaseMetaData;
+import org.apache.calcite.avatica.util.Quoting;
 import org.apache.drill.common.Version;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -397,12 +398,11 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
     return getServerMeta().getQuotedIdentifierCasing() == IdentifierCasing.IC_STORES_MIXED;
   }
 
-  // TODO(DRILL-3510):  Update when Drill accepts standard SQL's double quote.
   @Override
   public String getIdentifierQuoteString() throws SQLException {
     throwIfClosed();
     if (!getServerMetaSupported()) {
-      return "`";
+      return Quoting.BACK_TICK.string;
     }
     return getServerMeta().getIdentifierQuoteString();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionFactory.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionFactory.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionFactory.java
index 09366de..07e021e 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionFactory.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionFactory.java
@@ -18,6 +18,7 @@
 package org.apache.drill.jdbc;
 
 import java.sql.Connection;
+import java.sql.SQLException;
 
 /**
  * A factory used to get open {@link Connection} instances.
@@ -31,5 +32,5 @@ public interface ConnectionFactory {
    * @param info the connection parameters
    * @throws Exception if factory fails to get a connection.
    */
-  Connection getConnection(ConnectionInfo info) throws Exception;
+  Connection getConnection(ConnectionInfo info) throws SQLException;
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionInfoTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionInfoTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionInfoTest.java
new file mode 100644
index 0000000..ac6d309
--- /dev/null
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionInfoTest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.drill.jdbc;
+
+import org.apache.calcite.avatica.util.Quoting;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.SQLException;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for Drill's Properties in the JDBC URL connection string
+ */
+public class ConnectionInfoTest extends JdbcTestBase {
+  private static Connection connection;
+  private static DatabaseMetaData dbmd;
+
+  @Test
+  public void testQuotingIdentifiersProperty() throws SQLException {
+    try {
+      // Test DoubleQuotes for the DrillProperty#QUOTING_IDENTIFIERS in connection URL
+      connection = connect("jdbc:drill:zk=local;quoting_identifiers='\"'");
+      dbmd = connection.getMetaData();
+      assertThat(dbmd.getIdentifierQuoteString(), equalTo(Quoting.DOUBLE_QUOTE.string));
+      reset();
+
+      // Test Brackets for the DrillProperty#QUOTING_IDENTIFIERS in connection URL
+      connection = connect("jdbc:drill:zk=local;quoting_identifiers=[");
+      dbmd = connection.getMetaData();
+      assertThat(dbmd.getIdentifierQuoteString(), equalTo(Quoting.BRACKET.string));
+    } finally {
+      reset();
+    }
+  }
+
+  @Test(expected = SQLException.class)
+  public void testIncorrectCharacterForQuotingIdentifiers() throws SQLException {
+    try {
+      connection = connect("jdbc:drill:zk=local;quoting_identifiers=&");
+    }
+    catch (SQLException e) {
+      // Check exception text message
+      assertThat(e.getMessage(), containsString("Option planner.parser.quoting_identifiers " +
+          "must be one of: [`, \", []"));
+      throw e;
+    } finally {
+      reset();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataTest.java
index 0ec5c8b..ec04ab1 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -28,6 +28,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
+import org.apache.calcite.avatica.util.Quoting;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.SQLException;
@@ -103,10 +104,12 @@ public class DatabaseMetaDataTest {
   //  storesMixedCaseQuotedIdentifiers()
 
 
-  // TODO(DRILL-3510):  Update when Drill accepts standard SQL's double quote.
+  // TODO(DRILL-5402): Update when server meta information will be updated during one session.
   @Test
-  public void testGetIdentifierQuoteStringSaysBackquote() throws SQLException {
-    assertThat( dbmd.getIdentifierQuoteString(), equalTo( "`" ) );
+  public void testGetIdentifierQuoteString() throws SQLException {
+    // If connection string hasn't "quoting_identifiers" property, this method will return current system
+    // "planner.parser.quoting_identifiers" option (back tick by default)
+    assertThat(dbmd.getIdentifierQuoteString(), equalTo(Quoting.BACK_TICK.string));
   }
 
 

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java
index b9e8392..52777c1 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -60,8 +60,7 @@ public class JdbcTestBase extends ExecTest {
   public static void setUpTestCase() {
     factory = new SingleConnectionCachingFactory(new ConnectionFactory() {
       @Override
-      public Connection getConnection(ConnectionInfo info) throws Exception {
-        Class.forName("org.apache.drill.jdbc.Driver");
+      public Connection getConnection(ConnectionInfo info) throws SQLException {
         return DriverManager.getConnection(info.getUrl(), info.getParamsAsProperties());
       }
     });
@@ -73,7 +72,7 @@ public class JdbcTestBase extends ExecTest {
    * @param url connection URL
    * @throws Exception if connection fails
    */
-  protected static Connection connect(String url) throws Exception {
+  protected static Connection connect(String url) throws SQLException {
     return connect(url, JdbcAssert.getDefaultProperties());
   }
 
@@ -84,7 +83,7 @@ public class JdbcTestBase extends ExecTest {
    * @param info connection info
    * @throws Exception if connection fails
    */
-  protected static Connection connect(String url, Properties info) throws Exception {
+  protected static Connection connect(String url, Properties info) throws SQLException {
     final Connection conn = factory.getConnection(new ConnectionInfo(url, info));
     changeSchemaIfSupplied(conn, info);
     return conn;

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/jdbc/src/test/java/org/apache/drill/jdbc/MultiConnectionCachingFactory.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/MultiConnectionCachingFactory.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/MultiConnectionCachingFactory.java
index 063b5a5..7a99ddf 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/MultiConnectionCachingFactory.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/MultiConnectionCachingFactory.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -45,7 +45,7 @@ public class MultiConnectionCachingFactory implements CachingConnectionFactory {
    * {@link java.sql.Connection#close()}. Consumer must call {#close} to close the cached connections.
    */
   @Override
-  public Connection getConnection(ConnectionInfo info) throws Exception {
+  public Connection getConnection(ConnectionInfo info) throws SQLException {
     Connection conn = cache.get(info);
     if (conn == null) {
       conn = delegate.getConnection(info);

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/jdbc/src/test/java/org/apache/drill/jdbc/SingleConnectionCachingFactory.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/SingleConnectionCachingFactory.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/SingleConnectionCachingFactory.java
index 6466137..f23fac8 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/SingleConnectionCachingFactory.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/SingleConnectionCachingFactory.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -45,7 +45,7 @@ public class SingleConnectionCachingFactory implements CachingConnectionFactory
    * </p>
    */
   @Override
-  public Connection getConnection(ConnectionInfo info) throws Exception {
+  public Connection getConnection(ConnectionInfo info) throws SQLException {
     if (connection == null) {
       connection = delegate.getConnection(info);
     } else {

http://git-wip-us.apache.org/repos/asf/drill/blob/b17019e2/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcAssert.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcAssert.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcAssert.java
index caa09b8..da66ac1 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcAssert.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcAssert.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -167,7 +167,7 @@ public class JdbcAssert {
       this.info = info;
       this.adapter = new ConnectionFactoryAdapter() {
         @Override
-        public Connection createConnection() throws Exception {
+        public Connection createConnection() throws SQLException {
           return factory.getConnection(new ConnectionInfo("jdbc:drill:zk=local", ModelAndSchema.this.info));
         }
       };