You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2016/07/23 04:58:41 UTC

[1/6] drill git commit: DRILL-4746: Verification Failures (Decimal values) in drill's regression tests.

Repository: drill
Updated Branches:
  refs/heads/master 1f3c74dad -> 5a7d4c398


DRILL-4746: Verification Failures (Decimal values) in drill's regression tests.

This closes #545


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

Branch: refs/heads/master
Commit: 5ca2340a0a83412aa8fc8b077b72eca5f55e4226
Parents: 1f3c74d
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Wed Jul 13 13:44:27 2016 +0300
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Jul 22 21:28:07 2016 -0700

----------------------------------------------------------------------
 .../store/easy/text/compliant/TextInput.java    | 50 ++++++--------------
 1 file changed, 14 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/5ca2340a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/TextInput.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/TextInput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/TextInput.java
index d8b1672..1bb2d30 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/TextInput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/TextInput.java
@@ -17,22 +17,6 @@
  */
 package org.apache.drill.exec.store.easy.text.compliant;
 
-/*******************************************************************************
- * Copyright 2014 uniVocity Software Pty Ltd
- *
- * Licensed 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.
- ******************************************************************************/
-
 import io.netty.buffer.DrillBuf;
 import io.netty.util.internal.PlatformDependent;
 
@@ -188,7 +172,7 @@ final class TextInput {
    * read some more bytes from the stream.  Uses the zero copy interface if available.  Otherwise, does byte copy.
    * @throws IOException
    */
-  private final void read() throws IOException {
+  private void read() throws IOException {
     if(bufferReadable){
 
       if(remByte != -1){
@@ -219,7 +203,7 @@ final class TextInput {
    * Read more data into the buffer.  Will also manage split end conditions.
    * @throws IOException
    */
-  private final void updateBuffer() throws IOException {
+  private void updateBuffer() throws IOException {
     streamPos = seekable.getPos();
     underlyingBuffer.clear();
 
@@ -249,30 +233,24 @@ final class TextInput {
    * the split boundary.
    */
   private void updateLengthBasedOnConstraint() {
-    // find the next line separator:
     final long max = bStart + length;
-
-    for (long m = this.bStart + (endPos - streamPos); m < max; m++) {
-      long mTemp = m - 1;
+    for(long m = bStart + (endPos - streamPos); m < max; m++) {
       for (int i = 0; i < lineSeparator.length; i++) {
-        mTemp++;
-        if (PlatformDependent.getByte(mTemp) == lineSeparator[i]) {
-          if (mTemp < max) {
-            continue;
-          } else {
-            // remnant bytes
-            // the last N characters of the read were a remnant bytes. We'll hold off on dealing with these bytes until the next read.
-            remByte = i;
-            length -= (i + 1);
+        long mPlus = m + i;
+        if (mPlus < max) {
+          // we found a line separator and don't need to consult the next byte.
+          if (lineSeparator[i] == PlatformDependent.getByte(mPlus) && i == lineSeparator.length - 1) {
+            length = (int) (mPlus - bStart) + 1;
+            endFound = true;
             return;
           }
+        } else {
+          // the last N characters of the read were remnant bytes. We'll hold off on dealing with these bytes until the next read.
+          remByte = i;
+          length = length - i;
+          return;
         }
-        break;
       }
-      // we found line delimiter
-      length = (int) (mTemp - bStart);
-      endFound = true;
-      break;
     }
   }
 


[5/6] drill git commit: DRILL-4690: remove dependency introduced in DRILL-4690 for jdbc-all

Posted by pa...@apache.org.
DRILL-4690: remove dependency introduced in DRILL-4690 for jdbc-all


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

Branch: refs/heads/master
Commit: 81772d3b89457e83ef76f7fb3beb44773646644b
Parents: 855873e
Author: Parth Chandra <pa...@apache.org>
Authored: Fri Jul 22 16:47:48 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Jul 22 21:28:10 2016 -0700

----------------------------------------------------------------------
 exec/jdbc-all/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/81772d3b/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index 3d5725e..fe6e5cd 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -312,6 +312,7 @@
               <exclude>org.hamcrest:hamcrest-core</exclude>
               <exclude>org.mockito:mockito-core</exclude>
               <exclude>org.objenesis:objenesis</exclude>
+              <exclude>org.eclipse.jetty:*</exclude>
             </excludes>
           </artifactSet>
           <relocations>


[2/6] drill git commit: DRILL-4695: Log error thrown out of drillbit.run before close.

Posted by pa...@apache.org.
DRILL-4695: Log error thrown out of drillbit.run before close.

This closes #537


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

Branch: refs/heads/master
Commit: 4d9413415afcd254bdca2674cfb0eaa59d47e806
Parents: f36fec9
Author: chunhui-shi <cs...@maprtech.com>
Authored: Wed Jun 29 10:37:45 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Jul 22 21:28:08 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/drill/exec/server/Drillbit.java        | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4d941341/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index d0f4125..55a2b05 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -290,6 +290,7 @@ public class Drillbit implements AutoCloseable {
     try {
       bit.run();
     } catch (final Exception e) {
+      logger.error("Failure during initial startup of Drillbit.", e);
       bit.close();
       throw new DrillbitStartupException("Failure during initial startup of Drillbit.", e);
     }


[3/6] drill git commit: DRILL-4673: Implement "DROP TABLE IF EXISTS" for drill to prevent FAILED status on command return - implement DROP TABLE IF EXISTS and DROP VIEW IF EXISTS; - added unit test for DROP TABLE IF EXISTS; - added unit test for DROP VIE

Posted by pa...@apache.org.
DRILL-4673: Implement "DROP TABLE IF EXISTS" for drill to prevent FAILED status on command return - implement DROP TABLE IF EXISTS and DROP VIEW IF EXISTS; - added unit test for DROP TABLE IF EXISTS; - added unit test for DROP VIEW IF EXISTS; - added unit test for "IF" hive UDF.

This closes #541


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

Branch: refs/heads/master
Commit: f36fec9750fedd00a063f26b9998f9a994d025ad
Parents: 5ca2340
Author: Vitalii Diravka <vi...@gmail.com>
Authored: Fri May 13 19:24:31 2016 +0000
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Jul 22 21:28:08 2016 -0700

----------------------------------------------------------------------
 .../drill/exec/fn/hive/TestInbuiltHiveUDFs.java | 12 ++++-
 exec/java-exec/src/main/codegen/data/Parser.tdd |  3 +-
 .../src/main/codegen/includes/parserImpls.ftl   | 16 +++---
 .../planner/sql/handlers/DropTableHandler.java  | 22 +++++++--
 .../exec/planner/sql/handlers/ViewHandler.java  | 33 ++++++++-----
 .../sql/parser/CompoundIdentifierConverter.java |  4 +-
 .../exec/planner/sql/parser/SqlDropTable.java   | 27 +++++++++--
 .../exec/planner/sql/parser/SqlDropView.java    | 30 +++++++++---
 .../java/org/apache/drill/TestDropTable.java    | 51 ++++++++++++++++++++
 .../drill/exec/sql/TestBaseViewSupport.java     | 44 +++++++++++++++++
 .../apache/drill/exec/sql/TestViewSupport.java  | 31 ++++++++++++
 11 files changed, 233 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/f36fec97/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
index 93c9374..9e13844 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
@@ -23,7 +23,6 @@ import org.apache.drill.TestBuilder;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.hive.HiveTestBase;
-import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.junit.Test;
 
 import java.util.List;
@@ -84,4 +83,15 @@ public class TestInbuiltHiveUDFs extends HiveTestBase {
             "first_name","Bh","last_name","Venkata","position","Store"))
         .go();
   }
+
+  @Test // DRILL-3272
+  public void testIf() throws Exception {
+    testBuilder()
+        .sqlQuery("select `if`(1999 > 2000, 'latest', 'old') Period from hive.kv limit 1")
+        .ordered()
+        .baselineColumns("Period")
+        .baselineValues("old")
+        .go();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/f36fec97/exec/java-exec/src/main/codegen/data/Parser.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/Parser.tdd b/exec/java-exec/src/main/codegen/data/Parser.tdd
index 84b3dce..ce3ee4c 100644
--- a/exec/java-exec/src/main/codegen/data/Parser.tdd
+++ b/exec/java-exec/src/main/codegen/data/Parser.tdd
@@ -37,7 +37,8 @@
     "FILES",
     "REFRESH",
     "METADATA",
-    "DATABASE"
+    "DATABASE",
+    "IF"
   ]
 
   # List of methods for parsing custom SQL statements.

http://git-wip-us.apache.org/repos/asf/drill/blob/f36fec97/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
index 3a9424b..9901098 100644
--- a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
+++ b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
@@ -196,18 +196,20 @@ SqlNode SqlCreateOrReplaceView() :
 }
 
 /**
- * Parses a drop view statement.
- * DROP VIEW view_name;
+ * Parses a drop view or drop view if exists statement.
+ * DROP VIEW [IF EXISTS] view_name;
  */
 SqlNode SqlDropView() :
 {
     SqlParserPos pos;
+    boolean viewExistenceCheck = false;
 }
 {
     <DROP> { pos = getPos(); }
     <VIEW>
+    [ <IF> <EXISTS> { viewExistenceCheck = true; } ]
     {
-        return new SqlDropView(pos, CompoundIdentifier());
+        return new SqlDropView(pos, CompoundIdentifier(), viewExistenceCheck);
     }
 }
 
@@ -242,18 +244,20 @@ SqlNode SqlCreateTable() :
 }
 
 /**
- * Parses a drop table statement.
- * DROP TABLE table_name;
+ * Parses a drop table or drop table if exists statement.
+ * DROP TABLE [IF EXISTS] table_name;
  */
 SqlNode SqlDropTable() :
 {
     SqlParserPos pos;
+    boolean tableExistenceCheck = false;
 }
 {
     <DROP> { pos = getPos(); }
     <TABLE>
+    [ <IF> <EXISTS> { tableExistenceCheck = true; } ]
     {
-        return new SqlDropTable(pos, CompoundIdentifier());
+        return new SqlDropTable(pos, CompoundIdentifier(), tableExistenceCheck);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/f36fec97/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java
index 7684cb3..517c183 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java
@@ -19,7 +19,9 @@ package org.apache.drill.exec.planner.sql.handlers;
 
 import java.io.IOException;
 
+import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.tools.RelConversionException;
@@ -41,10 +43,12 @@ public class DropTableHandler extends DefaultSqlHandler {
   }
 
   /**
-   * Function resolves the schema and invokes the drop method. Raises an exception if the schema is
-   * immutable.
-   * @param sqlNode - Table name identifier
-   * @return - Single row indicating drop succeeded, raise exception otherwise
+   * Function resolves the schema and invokes the drop method
+   * (while IF EXISTS statement is used function invokes the drop method only if table exists).
+   * Raises an exception if the schema is immutable.
+   * @param sqlNode - SqlDropTable (SQL parse tree of drop table [if exists] query)
+   * @return - Single row indicating drop succeeded or table is not found while IF EXISTS statement is used,
+   * raise exception otherwise
    * @throws ValidationException
    * @throws RelConversionException
    * @throws IOException
@@ -62,13 +66,21 @@ public class DropTableHandler extends DefaultSqlHandler {
       drillSchema = SchemaUtilites.resolveToMutableDrillSchema(defaultSchema, dropTableNode.getSchema());
     }
 
-    String tableName = ((SqlDropTable) sqlNode).getName();
+    String tableName = dropTableNode.getName();
     if (drillSchema == null) {
       throw UserException.validationError()
           .message("Invalid table_name [%s]", tableName)
           .build(logger);
     }
 
+    if (dropTableNode.checkTableExistence()) {
+      final Table tableToDrop = SqlHandlerUtil.getTableFromSchema(drillSchema, tableName);
+      if (tableToDrop == null || tableToDrop.getJdbcTableType() != Schema.TableType.TABLE) {
+        return DirectPlan.createDirectPlan(context, true,
+            String.format("Table [%s] not found", tableName));
+      }
+    }
+
     drillSchema.dropTable(tableName);
 
     return DirectPlan.createDirectPlan(context, true,

http://git-wip-us.apache.org/repos/asf/drill/blob/f36fec97/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
index 6024fa5..b8396e6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
@@ -106,7 +106,7 @@ public abstract class ViewHandler extends DefaultSqlHandler {
     }
   }
 
-  /** Handler for Drop View DDL command. */
+  /** Handler for Drop View [If Exists] DDL command. */
   public static class DropView extends ViewHandler {
     public DropView(SqlHandlerConfig config) {
       super(config);
@@ -115,27 +115,34 @@ public abstract class ViewHandler extends DefaultSqlHandler {
     @Override
     public PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, RelConversionException, IOException, ForemanSetupException {
       SqlDropView dropView = unwrap(sqlNode, SqlDropView.class);
-      final String viewToDrop = dropView.getName();
+      final String viewName = dropView.getName();
       final AbstractSchema drillSchema =
           SchemaUtilites.resolveToMutableDrillSchema(context.getNewDefaultSchema(), dropView.getSchemaPath());
 
       final String schemaPath = drillSchema.getFullSchemaName();
 
-      final Table existingTable = SqlHandlerUtil.getTableFromSchema(drillSchema, viewToDrop);
-      if (existingTable != null && existingTable.getJdbcTableType() != Schema.TableType.VIEW) {
-        throw UserException.validationError()
-            .message("[%s] is not a VIEW in schema [%s]", viewToDrop, schemaPath)
-            .build(logger);
-      } else if (existingTable == null) {
-        throw UserException.validationError()
-            .message("Unknown view [%s] in schema [%s].", viewToDrop, schemaPath)
-            .build(logger);
+      final Table viewToDrop = SqlHandlerUtil.getTableFromSchema(drillSchema, viewName);
+      if (dropView.checkViewExistence()) {
+        if (viewToDrop == null || viewToDrop.getJdbcTableType() != Schema.TableType.VIEW){
+          return DirectPlan.createDirectPlan(context, true,
+              String.format("View [%s] not found in schema [%s].", viewName, schemaPath));
+        }
+      } else {
+        if (viewToDrop != null && viewToDrop.getJdbcTableType() != Schema.TableType.VIEW) {
+          throw UserException.validationError()
+              .message("[%s] is not a VIEW in schema [%s]", viewName, schemaPath)
+              .build(logger);
+        } else if (viewToDrop == null) {
+          throw UserException.validationError()
+              .message("Unknown view [%s] in schema [%s].", viewName, schemaPath)
+              .build(logger);
+        }
       }
 
-      drillSchema.dropView(viewToDrop);
+      drillSchema.dropView(viewName);
 
       return DirectPlan.createDirectPlan(context, true,
-          String.format("View [%s] deleted successfully from schema [%s].", viewToDrop, schemaPath));
+          String.format("View [%s] deleted successfully from schema [%s].", viewName, schemaPath));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/f36fec97/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
index a125628..fa0d319 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
@@ -164,13 +164,13 @@ public class CompoundIdentifierConverter extends SqlShuttle {
     rules.put(SqlCreateTable.class, R(D, D, D, E));
     rules.put(SqlCreateView.class, R(D, E, E, D));
     rules.put(SqlDescribeTable.class, R(D, D, E));
-    rules.put(SqlDropView.class, R(D));
+    rules.put(SqlDropView.class, R(D, D));
     rules.put(SqlShowFiles.class, R(D));
     rules.put(SqlShowSchemas.class, R(D, D));
     rules.put(SqlUseSchema.class, R(D));
     rules.put(SqlJoin.class, R(D, D, D, D, D, E));
     rules.put(SqlOrderBy.class, R(D, E, D, D));
-    rules.put(SqlDropTable.class, R(D));
+    rules.put(SqlDropTable.class, R(D, D));
     rules.put(SqlRefreshMetadata.class, R(D));
     rules.put(SqlSetOption.class, R(D, D, D));
     rules.put(SqlDescribeSchema.class, R(D));

http://git-wip-us.apache.org/repos/asf/drill/blob/f36fec97/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
index de272a1..c5e9ce3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
@@ -17,9 +17,9 @@
  */
 package org.apache.drill.exec.planner.sql.parser;
 
-import java.util.Collections;
 import java.util.List;
 
+import com.google.common.collect.Lists;
 import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
 import org.apache.drill.exec.planner.sql.handlers.DropTableHandler;
 import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
@@ -39,15 +39,21 @@ public class SqlDropTable extends DrillSqlCall {
   public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_TABLE", SqlKind.OTHER) {
     @Override
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
-      return new SqlDropTable(pos, (SqlIdentifier) operands[0]);
+      return new SqlDropTable(pos, (SqlIdentifier) operands[0], (SqlLiteral) operands[1]);
     }
   };
 
   private SqlIdentifier tableName;
+  private boolean tableExistenceCheck;
 
-  public SqlDropTable(SqlParserPos pos, SqlIdentifier tableName) {
+  public SqlDropTable(SqlParserPos pos, SqlIdentifier tableName, SqlLiteral tableExistenceCheck) {
+    this(pos, tableName, tableExistenceCheck.booleanValue());
+  }
+
+  public SqlDropTable(SqlParserPos pos, SqlIdentifier tableName, boolean tableExistenceCheck) {
     super(pos);
     this.tableName = tableName;
+    this.tableExistenceCheck = tableExistenceCheck;
   }
 
   @Override
@@ -57,13 +63,22 @@ public class SqlDropTable extends DrillSqlCall {
 
   @Override
   public List<SqlNode> getOperandList() {
-    return Collections.singletonList((SqlNode) tableName);
+    final List<SqlNode> ops =
+        ImmutableList.of(
+            tableName,
+            SqlLiteral.createBoolean(tableExistenceCheck, SqlParserPos.ZERO)
+        );
+    return ops;
   }
 
   @Override
   public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
     writer.keyword("DROP");
     writer.keyword("TABLE");
+    if (tableExistenceCheck) {
+      writer.keyword("IF");
+      writer.keyword("EXISTS");
+    }
     tableName.unparse(writer, leftPrec, rightPrec);
   }
 
@@ -92,4 +107,8 @@ public class SqlDropTable extends DrillSqlCall {
     return tableName;
   }
 
+  public boolean checkTableExistence() {
+    return tableExistenceCheck;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/f36fec97/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java
index 0e775b8..6fa6dbe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java
@@ -17,16 +17,11 @@
  */
 package org.apache.drill.exec.planner.sql.parser;
 
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.calcite.tools.Planner;
-
-import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
 import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
 import org.apache.drill.exec.planner.sql.handlers.ViewHandler.DropView;
-import org.apache.calcite.plan.hep.HepPlanner;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
@@ -43,15 +38,21 @@ public class SqlDropView extends DrillSqlCall {
   public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_VIEW", SqlKind.OTHER) {
     @Override
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
-      return new SqlDropView(pos, (SqlIdentifier) operands[0]);
+      return new SqlDropView(pos, (SqlIdentifier) operands[0], (SqlLiteral) operands[1]);
     }
   };
 
   private SqlIdentifier viewName;
+  private boolean viewExistenceCheck;
+
+  public SqlDropView(SqlParserPos pos, SqlIdentifier viewName, SqlLiteral viewExistenceCheck) {
+    this(pos, viewName, viewExistenceCheck.booleanValue());
+  }
 
-  public SqlDropView(SqlParserPos pos, SqlIdentifier viewName) {
+  public SqlDropView(SqlParserPos pos, SqlIdentifier viewName, boolean viewExistenceCheck) {
     super(pos);
     this.viewName = viewName;
+    this.viewExistenceCheck = viewExistenceCheck;
   }
 
   @Override
@@ -61,13 +62,22 @@ public class SqlDropView extends DrillSqlCall {
 
   @Override
   public List<SqlNode> getOperandList() {
-    return Collections.singletonList((SqlNode)viewName);
+    final List<SqlNode> ops =
+        ImmutableList.of(
+            viewName,
+            SqlLiteral.createBoolean(viewExistenceCheck, SqlParserPos.ZERO)
+        );
+    return ops;
   }
 
   @Override
   public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
     writer.keyword("DROP");
     writer.keyword("VIEW");
+    if (viewExistenceCheck) {
+      writer.keyword("IF");
+      writer.keyword("EXISTS");
+    }
     viewName.unparse(writer, leftPrec, rightPrec);
   }
 
@@ -92,4 +102,8 @@ public class SqlDropView extends DrillSqlCall {
     return viewName.names.get(viewName.names.size() - 1);
   }
 
+  public boolean checkViewExistence() {
+    return viewExistenceCheck;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/f36fec97/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java b/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java
index 4f8fe1a..e9a38b0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java
@@ -26,7 +26,10 @@ import org.junit.Assert;
 public class TestDropTable extends PlanTestBase {
 
   private static final String CREATE_SIMPLE_TABLE = "create table %s as select 1 from cp.`employee.json`";
+  private static final String CREATE_SIMPLE_VIEW = "create view %s as select 1 from cp.`employee.json`";
   private static final String DROP_TABLE = "drop table %s";
+  private static final String DROP_TABLE_IF_EXISTS = "drop table if exists %s";
+  private static final String DROP_VIEW_IF_EXISTS = "drop view if exists %s";
   private static final String BACK_TICK = "`";
 
   @Test
@@ -171,4 +174,52 @@ public class TestDropTable extends PlanTestBase {
 
     Assert.assertTrue("Dropping table on immutable schema failed", dropFailed);
   }
+
+  @Test // DRILL-4673
+  public void testDropTableIfExistsWhileTableExists() throws Exception {
+    final String existentTableName = "test_table";
+    test("use dfs_test.tmp");
+
+    // successful dropping of existent table
+    test(String.format(CREATE_SIMPLE_TABLE, existentTableName));
+    testBuilder()
+        .sqlQuery(String.format(DROP_TABLE_IF_EXISTS, existentTableName))
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(true, String.format("Table [%s] dropped", existentTableName))
+        .go();
+  }
+
+  @Test // DRILL-4673
+  public void testDropTableIfExistsWhileTableDoesNotExist() throws Exception {
+    final String nonExistentTableName = "test_table";
+    test("use dfs_test.tmp");
+
+    // dropping of non existent table without error
+    testBuilder()
+        .sqlQuery(String.format(DROP_TABLE_IF_EXISTS, nonExistentTableName))
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(true, String.format("Table [%s] not found", nonExistentTableName))
+        .go();
+  }
+
+  @Test // DRILL-4673
+  public void testDropTableIfExistsWhileItIsAView() throws Exception {
+    final String viewName = "test_view";
+    try{
+      test("use dfs_test.tmp");
+
+      // dropping of non existent table without error if the view with such name is existed
+      test(String.format(CREATE_SIMPLE_VIEW, viewName));
+      testBuilder()
+          .sqlQuery(String.format(DROP_TABLE_IF_EXISTS, viewName))
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true, String.format("Table [%s] not found", viewName))
+          .go();
+    } finally {
+      test(String.format(DROP_VIEW_IF_EXISTS, viewName));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/f36fec97/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
index a8f5bbb..43d8d57 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
@@ -105,6 +105,50 @@ public class TestBaseViewSupport extends BaseTestQuery {
   }
 
   /**
+   * Drop view if exists with given parameters.
+   *
+   * Current schema "dfs_test"
+   * DROP VIEW IF EXISTS tmp.viewName
+   *
+   * For the above DROP VIEW IF EXISTS query, function parameters values are:
+   *  viewSchema = "tmp"
+   *  "viewName" = "viewName"
+   *  "finalSchema" = "dfs_test.tmp"
+   *  "ifViewExists" = null
+   *
+   * @param viewSchema
+   * @param viewName
+   * @param finalSchema
+   * @param ifViewExists Helps to check query result depending from the existing of the view.
+   * @throws Exception
+   */
+  protected static void dropViewIfExistsHelper(final String viewSchema, final String viewName, final String finalSchema, Boolean ifViewExists) throws
+      Exception{
+    String viewFullName = "`" + viewName + "`";
+    if (!Strings.isNullOrEmpty(viewSchema)) {
+      viewFullName = viewSchema + "." + viewFullName;
+    }
+    if (ifViewExists == null) {
+      // ifViewExists == null: we do not know whether the table exists. Just drop it if exists or skip dropping if doesn't exist
+      test(String.format("DROP VIEW IF EXISTS %s", viewFullName));
+    } else if (ifViewExists) {
+      testBuilder()
+          .sqlQuery(String.format("DROP VIEW IF EXISTS %s", viewFullName))
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true, String.format("View [%s] deleted successfully from schema [%s].", viewName, finalSchema))
+          .go();
+    } else {
+      testBuilder()
+          .sqlQuery(String.format("DROP VIEW IF EXISTS %s", viewFullName))
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true, String.format("View [%s] not found in schema [%s].", viewName, finalSchema))
+          .go();
+    }
+  }
+
+  /**
    * Execute the given query and check against the given baseline.
    *
    * @param query

http://git-wip-us.apache.org/repos/asf/drill/blob/f36fec97/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
index 955da48..92f5c91 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
@@ -591,4 +591,35 @@ public class TestViewSupport extends TestBaseViewSupport {
       FileUtils.deleteQuietly(tblPath);
     }
   }
+
+  @Test // DRILL-4673
+  public void dropViewIfExistsWhenViewExists() throws Exception {
+    final String existentViewName = generateViewName();
+
+    // successful dropping of existent view
+    createViewHelper(TEMP_SCHEMA, existentViewName, TEMP_SCHEMA, null,
+        "SELECT c_custkey, c_nationkey from cp.`tpch/customer.parquet`");
+    dropViewIfExistsHelper(TEMP_SCHEMA, existentViewName, TEMP_SCHEMA, true);
+  }
+
+  @Test // DRILL-4673
+  public void dropViewIfExistsWhenViewDoesNotExist() throws Exception {
+    final String nonExistentViewName = generateViewName();
+
+    // dropping of non existent view without error
+    dropViewIfExistsHelper(TEMP_SCHEMA, nonExistentViewName, TEMP_SCHEMA, false);
+  }
+
+  @Test // DRILL-4673
+  public void dropViewIfExistsWhenItIsATable() throws Exception {
+    final String tableName = "table_name";
+    try{
+      // dropping of non existent view without error if the table with such name is existed
+      test(String.format("CREATE TABLE %s.%s as SELECT region_id, sales_city FROM cp.`region.json`",
+          TEMP_SCHEMA, tableName));
+      dropViewIfExistsHelper(TEMP_SCHEMA, tableName, TEMP_SCHEMA, false);
+    } finally {
+      test(String.format("DROP TABLE IF EXISTS %s.%s ", TEMP_SCHEMA, tableName));
+    }
+  }
 }


[6/6] drill git commit: DRILL-4499: Remove 16 unused classes

Posted by pa...@apache.org.
DRILL-4499: Remove 16 unused classes

This closes #426


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

Branch: refs/heads/master
Commit: 5a7d4c3983747a778e6a29d3450dd18871e98f2c
Parents: 81772d3
Author: Sudheesh Katkam <sk...@maprtech.com>
Authored: Thu Jul 21 17:13:59 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Jul 22 21:28:10 2016 -0700

----------------------------------------------------------------------
 .../apache/drill/common/DrillCloseables.java    | 55 -------------------
 .../drill/exec/expr/annotations/MethodMap.java  | 35 ------------
 .../apache/drill/exec/expr/fn/FunctionBody.java | 29 ----------
 .../org/apache/drill/exec/ops/Multitimer.java   | 42 ---------------
 .../org/apache/drill/exec/ops/QuerySetup.java   | 26 ---------
 .../exec/rpc/control/AvailabilityListener.java  | 22 --------
 .../drill/exec/rpc/control/ControlCommand.java  | 28 ----------
 .../drill/exec/rpc/control/SendProgress.java    | 22 --------
 .../drill/exec/rpc/data/SendProgress.java       | 22 --------
 .../apache/drill/exec/rpc/user/DrillUser.java   | 36 -------------
 .../apache/drill/exec/store/RecordRecorder.java | 36 -------------
 .../drill/exec/store/schedule/PartialWork.java  | 43 ---------------
 .../org/apache/drill/exec/util/AtomicState.java | 56 --------------------
 .../drill/exec/work/RecordOutputStream.java     | 22 --------
 .../apache/drill/exec/work/ResourceRequest.java | 30 -----------
 .../apache/drill/exec/work/RootNodeDriver.java  | 25 ---------
 16 files changed, 529 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/common/src/main/java/org/apache/drill/common/DrillCloseables.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/DrillCloseables.java b/common/src/main/java/org/apache/drill/common/DrillCloseables.java
deleted file mode 100644
index 289066b..0000000
--- a/common/src/main/java/org/apache/drill/common/DrillCloseables.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.common;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/**
- * Provides additional functionality to Guava's Closeables.
- */
-public class DrillCloseables {
-  /**
-   * Constructor. Prevents construction for class of static utilities.
-   */
-  private DrillCloseables() {
-  }
-
-  /**
-   * Close() a {@see java.io.Closeable} without throwing a (checked)
-   * {@see java.io.IOException}. This wraps the close() call with a
-   * try-catch that will rethrow an IOException wrapped with a
-   * {@see java.lang.RuntimeException}, providing a way to call close()
-   * without having to do the try-catch everywhere or propagate the IOException.
-   *
-   * <p>Guava has deprecated {@see com.google.common.io.Closeables.closeQuietly()}
-   * as described in
-   * {@link https://code.google.com/p/guava-libraries/issues/detail?id=1118}.
-   *
-   * @param closeable the Closeable to close
-   * @throws RuntimeException if an IOException occurs; the IOException is
-   *   wrapped by the RuntimeException
-   */
-  public static void closeNoChecked(final Closeable closeable) {
-    try {
-      closeable.close();
-    } catch(final IOException e) {
-      throw new RuntimeException("IOException while closing", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/MethodMap.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/MethodMap.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/MethodMap.java
deleted file mode 100644
index 775f6a3..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/MethodMap.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.expr.annotations;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * Marker annotation to determine which fields should be included as parameters for the function.
- */
-@Retention(RetentionPolicy.RUNTIME)
-@Target({ElementType.METHOD})
-public @interface MethodMap {
-
-
-  String parentMethod();
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionBody.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionBody.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionBody.java
deleted file mode 100644
index df110ed..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionBody.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.expr.fn;
-
-public class FunctionBody {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FunctionBody.class);
-
-
-  public static enum BodyType{
-    SETUP, EVAL_INNER, EVAL_OUTER,
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/ops/Multitimer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/Multitimer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/Multitimer.java
deleted file mode 100644
index 7e6ae8e..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/Multitimer.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.ops;
-
-import org.slf4j.Logger;
-
-public class Multitimer<T extends Enum<T>> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Multitimer.class);
-
-  private final long start;
-  private final long[] times;
-  private final Class<T> clazz;
-
-  public Multitimer(Class<T> clazz){
-    this.times = new long[clazz.getEnumConstants().length];
-    this.start = System.nanoTime();
-    this.clazz = clazz;
-  }
-
-  public void mark(T timer){
-    times[timer.ordinal()] = System.nanoTime();
-  }
-
-  public void log(Logger logger){
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QuerySetup.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QuerySetup.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QuerySetup.java
deleted file mode 100644
index ef73867..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QuerySetup.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.ops;
-
-public enum QuerySetup {
-  START_SQL_PARSING,
-  START_SQL_VALIDATION,
-  START_SQL_TO_REL,
-  START_OPTIQ_REL_TO_DRILL_LOGICAL,
-  START_DRILL_LOGICAL_TO_PHYSICAL;
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/AvailabilityListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/AvailabilityListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/AvailabilityListener.java
deleted file mode 100644
index 4f817c6..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/AvailabilityListener.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.rpc.control;
-
-public interface AvailabilityListener {
-  public void isAvailable(ControlConnection connection);
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlCommand.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlCommand.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlCommand.java
deleted file mode 100644
index 52d4289..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlCommand.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.rpc.control;
-
-import org.apache.drill.exec.rpc.RpcConnectionHandler;
-
-import com.google.protobuf.MessageLite;
-
-public interface ControlCommand<T extends MessageLite> extends RpcConnectionHandler<ControlConnection>{
-
-  public abstract void connectionAvailable(ControlConnection connection);
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/SendProgress.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/SendProgress.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/SendProgress.java
deleted file mode 100644
index f270eee..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/SendProgress.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.rpc.control;
-
-public class SendProgress {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SendProgress.class);
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/SendProgress.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/SendProgress.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/SendProgress.java
deleted file mode 100644
index 490013f..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/SendProgress.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.rpc.data;
-
-public class SendProgress {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SendProgress.class);
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/DrillUser.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/DrillUser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/DrillUser.java
deleted file mode 100644
index c344ce1..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/DrillUser.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.rpc.user;
-
-import java.io.IOException;
-
-import org.apache.hadoop.security.UserGroupInformation;
-
-public class DrillUser {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillUser.class);
-
-  private UserGroupInformation hadoopUser;
-
-  public DrillUser(String userName) throws IOException {
-    this.hadoopUser = UserGroupInformation.createProxyUser(userName, UserGroupInformation.getCurrentUser());
-  }
-
-  public UserGroupInformation getHadoopUser(){
-    return hadoopUser;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordRecorder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordRecorder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordRecorder.java
deleted file mode 100644
index ffaff8f..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordRecorder.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store;
-
-import java.io.IOException;
-
-import org.apache.drill.exec.record.RecordBatch;
-
-public interface RecordRecorder {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordRecorder.class);
-
-  public void setup() throws IOException;
-
-  /**
-   *
-   * @param batch
-   * @return
-   */
-  public boolean record(RecordBatch batch);
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/PartialWork.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/PartialWork.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/PartialWork.java
deleted file mode 100644
index 8080747..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/PartialWork.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.schedule;
-
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-
-public class PartialWork {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartialWork.class);
-
-  private final long length;
-  private final DrillbitEndpoint[] locations;
-
-  public PartialWork(long length, DrillbitEndpoint[] locations) {
-    super();
-    this.length = length;
-    this.locations = locations;
-  }
-
-  public long getLength() {
-    return length;
-  }
-  public DrillbitEndpoint[] getLocations() {
-    return locations;
-  }
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
deleted file mode 100644
index ec498d4..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.util;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-import com.google.protobuf.Internal.EnumLite;
-
-/**
- * Simple wrapper class around AtomicInteger which allows management of a State value extending EnumLite.
- * @param <T> The type of EnumLite to use for state.
- */
-public abstract class AtomicState<T extends EnumLite> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AtomicState.class);
-
-  private final AtomicInteger state = new AtomicInteger();
-
-  /**
-   * Constructor that defines initial T state.
-   * @param initial
-   */
-  public AtomicState(T initial){
-    state.set(initial.getNumber());
-  }
-
-  protected abstract T getStateFromNumber(int i);
-
-  /**
-   * Does an atomic conditional update from one state to another.
-   * @param oldState The expected current state.
-   * @param newState The desired new state.
-   * @return Whether or not the update was successful.
-   */
-  public boolean updateState(T oldState, T newState){
-    return state.compareAndSet(oldState.getNumber(), newState.getNumber());
-  }
-
-  public T getState(){
-    return getStateFromNumber(state.get());
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/work/RecordOutputStream.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/RecordOutputStream.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/RecordOutputStream.java
deleted file mode 100644
index 25d1786..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/RecordOutputStream.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.work;
-
-public class RecordOutputStream {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordOutputStream.class);
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java
deleted file mode 100644
index 5b7b623..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.work;
-
-public class ResourceRequest {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ResourceRequest.class);
-
-  public long memoryMin;
-  public long memoryDesired;
-
-
-  public static class ResourceAllocation {
-    public long memory;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5a7d4c39/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java
deleted file mode 100644
index b56477b..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.work;
-
-public interface RootNodeDriver {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootNodeDriver.class);
-
-  public boolean doNext();
-
-}


[4/6] drill git commit: DRILL-4690: initial support for CORS

Posted by pa...@apache.org.
DRILL-4690: initial support for CORS

Added CrossOriginFilter to WebServer based on option HTTP_ENABLE_CORS
Fixed issues related to style
Restricted headers, added run of filterChain
Filter from org.eclipse.jetty.servlets
Enabled configuration, jetty version 9.1.5, restrict filtered paths
CORS by default disabled, reduced size of dependencies (reset maxsize)

This closes #507


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

Branch: refs/heads/master
Commit: 855873ef30bae0af402d32ee4cf44a65f97fb1db
Parents: 4d94134
Author: Wojciech Nowak <ma...@pythonic.ninja>
Authored: Sat May 21 18:00:37 2016 +0200
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Jul 22 21:28:09 2016 -0700

----------------------------------------------------------------------
 .../src/resources/drill-override-example.conf   |  9 ++++-
 exec/java-exec/pom.xml                          | 35 ++++++++++++++++++++
 .../org/apache/drill/exec/ExecConstants.java    |  5 +++
 .../drill/exec/server/rest/WebServer.java       | 21 ++++++++++++
 .../src/main/resources/drill-module.conf        |  9 ++++-
 5 files changed, 77 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/855873ef/distribution/src/resources/drill-override-example.conf
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-override-example.conf b/distribution/src/resources/drill-override-example.conf
index 9e29a5f..52949db 100644
--- a/distribution/src/resources/drill-override-example.conf
+++ b/distribution/src/resources/drill-override-example.conf
@@ -84,7 +84,14 @@ drill.exec: {
     enabled: true,
     ssl_enabled: false,
     port: 8047
-    session_max_idle_secs: 3600 # Default value 1hr
+    session_max_idle_secs: 3600, # Default value 1hr
+    cors: {
+      enabled: false,
+      allowedOrigins: ["null"],
+      allowedMethods: ["GET", "POST", "HEAD", "OPTIONS"],
+      allowedHeaders: ["X-Requested-With", "Content-Type", "Accept", "Origin"],
+      credentials: true
+    }
   },
   functions: ["org.apache.drill.expr.fn.impl"],
   network: {

http://git-wip-us.apache.org/repos/asf/drill/blob/855873ef/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 9de4720..892467d 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -104,6 +104,41 @@
       <version>9.1.5.v20140505</version>
     </dependency>
     <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlets</artifactId>
+      <version>9.1.5.v20140505</version>
+      <exclusions>
+        <exclusion>
+          <artifactId>jetty-continuation</artifactId>
+          <groupId>org.eclipse.jetty</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jetty-http</artifactId>
+          <groupId>org.eclipse.jetty</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jetty-util</artifactId>
+          <groupId>org.eclipse.jetty</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>javax.servlet-api</artifactId>
+          <groupId>javax.servlet</groupId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-io</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-jmx</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty.toolchain</groupId>
+          <artifactId>jetty-test-helper</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
       <groupId>org.glassfish.jersey.containers</groupId>
       <artifactId>jersey-container-jetty-servlet</artifactId>
       <version>2.8</version>

http://git-wip-us.apache.org/repos/asf/drill/blob/855873ef/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 0d7e0d0..64931a2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -76,6 +76,11 @@ public interface ExecConstants {
   String HTTP_ENABLE = "drill.exec.http.enabled";
   String HTTP_PORT = "drill.exec.http.port";
   String HTTP_ENABLE_SSL = "drill.exec.http.ssl_enabled";
+  String HTTP_CORS_ENABLED = "drill.exec.http.cors.enabled";
+  String HTTP_CORS_ALLOWED_ORIGINS = "drill.exec.http.cors.allowedOrigins";
+  String HTTP_CORS_ALLOWED_METHODS = "drill.exec.http.cors.allowedMethods";
+  String HTTP_CORS_ALLOWED_HEADERS = "drill.exec.http.cors.allowedHeaders";
+  String HTTP_CORS_CREDENTIALS = "drill.exec.http.cors.credentials";
   String HTTP_SESSION_MAX_IDLE_SECS = "drill.exec.http.session_max_idle_secs";
   String HTTP_KEYSTORE_PATH = "javax.net.ssl.keyStore";
   String HTTP_KEYSTORE_PASSWORD = "javax.net.ssl.keyStorePassword";

http://git-wip-us.apache.org/repos/asf/drill/blob/855873ef/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
index 5ea781b..0b19dce 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
@@ -23,6 +23,7 @@ import com.codahale.metrics.servlets.ThreadDumpServlet;
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableSet;
 import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.server.rest.auth.DrillRestLoginService;
@@ -53,13 +54,16 @@ import org.eclipse.jetty.server.handler.ErrorHandler;
 import org.eclipse.jetty.server.session.HashSessionManager;
 import org.eclipse.jetty.server.session.SessionHandler;
 import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.FilterHolder;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlets.CrossOriginFilter;
 import org.eclipse.jetty.util.resource.Resource;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
 import org.glassfish.jersey.servlet.ServletContainer;
 import org.joda.time.DateTime;
 
+import javax.servlet.DispatcherType;
 import javax.servlet.http.HttpSession;
 import javax.servlet.http.HttpSessionEvent;
 import javax.servlet.http.HttpSessionListener;
@@ -71,6 +75,7 @@ import java.security.SecureRandom;
 import java.security.cert.X509Certificate;
 import java.util.Collections;
 import java.util.Date;
+import java.util.EnumSet;
 import java.util.Set;
 
 import static org.apache.drill.exec.server.rest.auth.DrillUserPrincipal.ADMIN_ROLE;
@@ -159,6 +164,22 @@ public class WebServer implements AutoCloseable {
       servletContextHandler.setSessionHandler(createSessionHandler(servletContextHandler.getSecurityHandler()));
     }
 
+    if (config.getBoolean(ExecConstants.HTTP_CORS_ENABLED)) {
+      FilterHolder holder = new FilterHolder(CrossOriginFilter.class);
+      holder.setInitParameter(CrossOriginFilter.ALLOWED_ORIGINS_PARAM,
+              StringUtils.join(config.getStringList(ExecConstants.HTTP_CORS_ALLOWED_ORIGINS), ","));
+      holder.setInitParameter(CrossOriginFilter.ALLOWED_METHODS_PARAM,
+              StringUtils.join(config.getStringList(ExecConstants.HTTP_CORS_ALLOWED_METHODS), ","));
+      holder.setInitParameter(CrossOriginFilter.ALLOWED_HEADERS_PARAM,
+              StringUtils.join(config.getStringList(ExecConstants.HTTP_CORS_ALLOWED_HEADERS), ","));
+      holder.setInitParameter(CrossOriginFilter.ALLOW_CREDENTIALS_PARAM,
+              String.valueOf(config.getBoolean(ExecConstants.HTTP_CORS_CREDENTIALS)));
+
+      for (String path: new String[] { "*.json", "/storage/*/enable/*", "/status*" }) {
+        servletContextHandler.addFilter(holder, path, EnumSet.of(DispatcherType.REQUEST));
+      }
+    }
+
     embeddedJetty.start();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/855873ef/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index b1b9b46..d8f4759b 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -111,7 +111,14 @@ drill.exec: {
     enabled: true,
     ssl_enabled: false,
     port: 8047
-    session_max_idle_secs: 3600 # Default value 1hr
+    session_max_idle_secs: 3600, # Default value 1hr
+    cors: {
+      enabled: false,
+      allowedOrigins: ["null"],
+      allowedMethods: ["GET", "POST", "HEAD", "OPTIONS"],
+      allowedHeaders: ["X-Requested-With", "Content-Type", "Accept", "Origin"],
+      credentials: true
+    }
   },
   network: {
     start: 35000