You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by sa...@apache.org on 2019/05/29 23:08:36 UTC

[drill] branch master updated (9f67e95 -> 2615d68)

This is an automated email from the ASF dual-hosted git repository.

sachouche pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git.


    from 9f67e95  DRILL-7181: Improve V3 text reader (row set) error messages
     new 5542cc9  DRILL-7276: Fixed an XSS vulnerability in Drill Web-UI query profile page
     new 7629664  DRILL-7206:using primitive int list at right or full join case
     new 20b58ab  DRILL-7204: Add proper validation when creating plugin
     new e8b921b  DRILL-7196: Queries are still runnable on disabled plugins
     new 2615d68  DRILL-4782 / DRILL-7139: Fix DATE_ADD and TO_TIME functions

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 common/src/test/resources/basic_physical.json      |  51 +++---
 common/src/test/resources/dsort-logical.json       |  48 ++---
 common/src/test/resources/jdbc_plan.json           |   4 +-
 common/src/test/resources/simple_plan.json         | 159 ++++++++---------
 common/src/test/resources/storage_engine_plan.json |   5 +-
 contrib/native/client/resources/simple_plan.json   | 126 ++++++-------
 .../exec/store/hbase/HBaseStoragePluginConfig.java |   2 +-
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../java/org/apache/drill/hbase/BaseHBaseTest.java |  30 ++--
 .../hbase/hbase_scan_screen_physical.json          |  31 ++--
 .../hbase_scan_screen_physical_column_select.json  |  33 ++--
 .../hbase_scan_screen_physical_family_select.json  |  33 ++--
 .../exec/store/hive/schema/HiveSchemaFactory.java  |   1 -
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../test/resources/bootstrap-storage-plugins.json  |   8 +-
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../drill/exec/store/mongo/MongoTestBase.java      |  71 ++------
 .../drill/exec/store/mongo/MongoTestConstants.java |  78 ++++-----
 .../drill/exec/store/mongo/MongoTestSuit.java      |  82 ++++-----
 .../exec/store/mongo/TestMongoFilterPushDown.java  |  34 ++--
 .../exec/store/mongo/TestMongoProjectPushDown.java |  36 ++--
 .../drill/exec/store/mongo/TestMongoQueries.java   |  79 +++++----
 .../drill/exec/store/mongo/TestTableGenerator.java |  10 +-
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../templates/CastStringTypesToInterval.java       |  29 +--
 .../IntervalIntervalArithmetic.java                |  22 ++-
 .../ToDateTypeFunctions.java                       |   2 +-
 .../org/apache/calcite/jdbc/DynamicRootSchema.java |  16 +-
 .../exec/physical/impl/common/HashPartition.java   |   3 +-
 .../exec/physical/impl/join/HashJoinHelper.java    |   5 +-
 .../physical/impl/join/HashJoinProbeTemplate.java  |   4 +-
 .../drill/exec/server/rest/StorageResources.java   |  13 +-
 .../exec/store/StoragePluginRegistryImpl.java      |  38 ++--
 .../main/resources/bootstrap-storage-plugins.json  | 194 ++++++++++-----------
 .../src/main/resources/rest/alertModals.ftl        |  11 +-
 .../src/main/resources/rest/confirmationModals.ftl |  54 ++++++
 .../src/main/resources/rest/static/js/graph.js     |   2 +-
 .../src/main/resources/rest/storage/list.ftl       |  21 ++-
 .../src/main/resources/rest/storage/update.ftl     |  29 ++-
 .../expr/fn/impl/TestIntervalDayFunctions.java     |  95 ++++++++++
 .../drill/exec/fn/impl/TestCastFunctions.java      |  69 +++++---
 .../exec/fn/impl/testing/TestDateConversions.java  |  12 ++
 .../drill/exec/store/store/TestDisabledPlugin.java |   6 +
 .../java/org/apache/drill/test/QueryBuilder.java   | 154 +++++++++++-----
 .../src/test/resources/agg/hashagg/q6.json         |  77 ++++----
 .../src/test/resources/agg/hashagg/q7_1.json       |  69 ++++----
 .../src/test/resources/agg/hashagg/q7_2.json       |  69 ++++----
 .../src/test/resources/agg/hashagg/q7_3.json       |  67 +++----
 .../src/test/resources/agg/hashagg/q8.json         |  61 +++----
 .../src/test/resources/agg/hashagg/q8_1.json       |  95 +++++-----
 .../src/test/resources/common/test_hashtable1.json |  64 ++++---
 .../test/resources/decimal/cast_decimal_float.json |  67 ++++---
 .../test/resources/decimal/cast_decimal_int.json   |  67 ++++---
 .../resources/decimal/cast_decimal_vardecimal.json | 106 ++++++-----
 .../test/resources/decimal/cast_float_decimal.json |  74 ++++----
 .../test/resources/decimal/cast_int_decimal.json   |  67 ++++---
 .../resources/decimal/cast_simple_decimal.json     |  77 ++++----
 .../resources/decimal/cast_vardecimal_decimal.json |  73 ++++----
 .../decimal/simple_decimal_arithmetic.json         | 102 ++++++-----
 .../resources/decimal/simple_decimal_math.json     |  69 ++++----
 .../resources/decimal/test_decimal_complex.json    |  95 +++++-----
 .../decimal/test_decimal_dense_sparse.json         | 129 ++++++--------
 .../decimal/test_decimal_sort_complex.json         |  98 +++++------
 .../decimal/test_decimal_sparse_dense_dense.json   |  40 ++---
 .../functions/cast/testCastVarCharNull.json        |  64 ++++---
 .../cast/test_cast_varchar_to_bigint.json          |  60 ++++---
 .../conv/conversionTestWithLogicalPlan.json        | 138 +++++++++------
 .../src/test/resources/functions/date/age.json     |  61 ++++---
 .../functions/date/date_difference_arithmetic.json |  57 +++---
 .../functions/date/date_interval_arithmetic.json   | 100 +++++++----
 .../functions/date/interval_arithmetic.json        |  93 ++++++----
 .../src/test/resources/functions/date/to_char.json |  57 +++---
 .../resources/functions/date/to_date_type.json     |  60 ++++---
 .../src/test/resources/functions/extractFrom.json  |  81 +++++----
 .../test/resources/functions/test_covariance.json  |  55 +++---
 .../resources/functions/test_logical_aggr.json     |  53 +++---
 .../resources/functions/test_stddev_variance.json  |  24 +--
 exec/java-exec/src/test/resources/hive/test.json   | 141 ++++++++-------
 .../src/test/resources/join/hashJoinExpr.json      |   2 +-
 .../src/test/resources/join/hash_join.json         | 109 ++++++------
 .../src/test/resources/join/hj_exchanges.json      |  19 +-
 .../src/test/resources/join/hj_exchanges1.json     |   3 +-
 .../resources/join/hj_multi_condition_join.json    | 123 +++++++------
 .../src/test/resources/join/mergeJoinExpr.json     |   2 +-
 .../resources/join/merge_inner_single_batch.json   |  81 +++++----
 .../test/resources/join/merge_join_nullkey.json    |   2 +-
 .../src/test/resources/join/merge_multi_batch.json |  82 +++++----
 .../test/resources/join/merge_single_batch.json    |  82 +++++----
 .../test/resources/join/mj_multi_condition.json    |   5 +-
 .../largefiles/merging_receiver_large_data.json    |  11 +-
 .../test/resources/lateraljoin/lateralplan1.json   |   9 +-
 .../src/test/resources/limit/limit_exchanges.json  |   9 +-
 .../src/test/resources/logical_group.json          | 133 +++++++-------
 .../src/test/resources/logical_json_scan.json      |  51 +++---
 .../src/test/resources/logical_order.json          |  85 ++++-----
 .../src/test/resources/logical_string_filter.json  |  77 ++++----
 .../test/resources/parquet/par_writer_test.json    |  42 ++---
 .../parquet/parquet_group_scan_union_screen.json   |  81 ++++-----
 .../test/resources/parquet/parquet_nullable.json   |  66 ++++---
 .../resources/parquet/parquet_nullable_varlen.json |  68 ++++----
 .../resources/parquet/parquet_scan_screen.json     |  70 ++++----
 .../parquet_scan_screen_read_entry_replace.json    |   3 +-
 .../parquet_scan_union_screen_physical.json        |  48 +++--
 .../parquet/parquet_selective_column_read.json     |  40 ++---
 .../src/test/resources/physical_filter.json        |  64 +++----
 .../src/test/resources/physical_join.json          | 182 +++++++++----------
 .../test/resources/physical_json_scan_test1.json   |  54 +++---
 .../test/resources/physical_order_varbinary.json   |  73 ++++----
 .../record/vector/test_all_date_literals.json      |  61 ++++---
 .../test/resources/record/vector/test_date.json    |  50 +++---
 .../resources/record/vector/test_date_add.json     |  53 +++---
 .../resources/record/vector/test_interval.json     |  64 +++----
 .../resources/record/vector/test_sort_date.json    |  91 +++++-----
 .../resources/record/vector/test_timestamp.json    |  50 +++---
 .../src/test/resources/scan_screen_logical.json    |  75 ++++----
 .../test/resources/sender/broadcast_exchange.json  |  92 +++++-----
 .../src/test/resources/sender/hash_exchange2.json  |  84 +++++----
 .../json/project_pushdown_json_physical_plan.json  |   4 +-
 .../src/test/resources/store/text/test.json        |  73 ++++----
 .../test/resources/writer/simple_csv_writer.json   | 158 +++++++++--------
 122 files changed, 3647 insertions(+), 3242 deletions(-)
 create mode 100644 exec/java-exec/src/main/resources/rest/confirmationModals.ftl
 create mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestIntervalDayFunctions.java


[drill] 05/05: DRILL-4782 / DRILL-7139: Fix DATE_ADD and TO_TIME functions

Posted by sa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sachouche pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 2615d68de4e44b1f03f5c047018548c06a7396b4
Author: Dmytriy Grinchenko <dm...@gmail.com>
AuthorDate: Mon May 6 09:01:29 2019 +0300

    DRILL-4782 / DRILL-7139: Fix DATE_ADD and TO_TIME functions
    
    - cast function for the day interval changed to round milliseconds to complete days
    - ToDateTypeFunctions#toTime now returning milliseconds of day
    - updated the way how DayInterval subtracts and adds, to follow the cast function logic
    
    UT core updates:
    
    - added vectorValue function to the queryBuilder to simplify retrieving value of the vector
    - refactored singleton query result functions at queryBuilder
---
 .../templates/CastStringTypesToInterval.java       |  29 ++--
 .../IntervalIntervalArithmetic.java                |  22 ++-
 .../ToDateTypeFunctions.java                       |   2 +-
 .../expr/fn/impl/TestIntervalDayFunctions.java     |  95 +++++++++++++
 .../drill/exec/fn/impl/TestCastFunctions.java      |  69 +++++----
 .../exec/fn/impl/testing/TestDateConversions.java  |  12 ++
 .../java/org/apache/drill/test/QueryBuilder.java   | 154 +++++++++++++++------
 7 files changed, 304 insertions(+), 79 deletions(-)

diff --git a/exec/java-exec/src/main/codegen/templates/CastStringTypesToInterval.java b/exec/java-exec/src/main/codegen/templates/CastStringTypesToInterval.java
index 196e860..7acfd69 100644
--- a/exec/java-exec/src/main/codegen/templates/CastStringTypesToInterval.java
+++ b/exec/java-exec/src/main/codegen/templates/CastStringTypesToInterval.java
@@ -86,24 +86,33 @@ public class CastEmptyString${type.from}To${type.to} implements DrillSimpleFunc
     org.joda.time.Period period = org.joda.time.Period.parse(input);
 
     <#if type.to == "Interval" || type.to == "NullableInterval">
-    out.months       = (period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) + period.getMonths();
+    out.months       = period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths + period.getMonths();
 
     out.days         = period.getDays();
 
-    out.milliseconds = (period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
-                       (period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
-                       (period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
-                       (period.getMillis());
+    out.milliseconds = period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis +
+                       period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis +
+                       period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis +
+                       period.getMillis();
 
     <#elseif type.to == "IntervalDay" || type.to == "NullableIntervalDay">
     out.days         = period.getDays();
 
-    out.milliseconds = (period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
-                       (period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
-                       (period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
-                       (period.getMillis());
+    long millis = period.getHours() * (long) org.apache.drill.exec.vector.DateUtilities.hoursToMillis +
+                  period.getMinutes() * (long) org.apache.drill.exec.vector.DateUtilities.minutesToMillis +
+                  period.getSeconds() * (long) org.apache.drill.exec.vector.DateUtilities.secondsToMillis +
+                  period.getMillis();
+
+    if (millis >= org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis) {
+      int daysFromMillis = (int) (millis / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
+      millis -= daysFromMillis * org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
+      out.days += daysFromMillis;
+    }
+
+    out.milliseconds = (int) millis;
+
     <#elseif type.to == "IntervalYear" || type.to == "NullableIntervalYear">
-    out.value = (period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) + period.getMonths();
+    out.value = period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths + period.getMonths();
     </#if>
   }
 }
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
index 7fecabb..ccd8e1f 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
@@ -65,7 +65,15 @@ public class ${intervaltype}Functions {
             out.value = left.value + right.value;
             <#elseif intervaltype == "IntervalDay">
             out.days = left.days + right.days;
-            out.milliseconds = left.milliseconds + right.milliseconds;
+            long millis = (long) left.milliseconds + right.milliseconds;
+
+            if (millis >= org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis) {
+              int daysFromMillis = (int) (millis / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
+
+              millis -= daysFromMillis * org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
+              out.days += daysFromMillis;
+            }
+            out.milliseconds = (int) millis;
             </#if>
         }
     }
@@ -90,6 +98,18 @@ public class ${intervaltype}Functions {
             <#elseif intervaltype == "IntervalDay">
             out.days = left.days - right.days;
             out.milliseconds = left.milliseconds - right.milliseconds;
+
+            int daysFromMillis = out.milliseconds/org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
+            if (daysFromMillis != 0) {
+              out.milliseconds -= org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis*daysFromMillis;
+              out.days -= Math.abs(daysFromMillis);
+            }
+
+            // if milliseconds are bellow zero, substract them from the days
+            if (out.milliseconds < 0 && out.days > 0) {
+              out.days  -= 1;
+              out.milliseconds =  org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis + out.milliseconds;
+            }
             </#if>
         }
     }
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
index 6ef196f..7f5784a 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
@@ -82,7 +82,7 @@ public class G${convert.from}To${convert.to} implements DrillSimpleFunc {
     <#elseif convert.to == "TimeStamp" || convert.to == "NullableTimeStamp">
     out.value = org.joda.time.DateTime.parse(input, format).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
     <#elseif convert.to == "Time" || convert.to == "NullableTime">
-    out.value = (int) ((format.parseDateTime(input)).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+    out.value = format.parseDateTime(input).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillisOfDay();
     </#if>
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestIntervalDayFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestIntervalDayFunctions.java
new file mode 100644
index 0000000..2bc29c5
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestIntervalDayFunctions.java
@@ -0,0 +1,95 @@
+/*
+ * 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.impl;
+
+import org.apache.drill.exec.vector.IntervalDayVector;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestIntervalDayFunctions extends ClusterTest {
+  @BeforeClass
+  public static void setUp() throws Exception {
+    startCluster(ClusterFixture.builder(dirTestWatcher));
+  }
+
+  @Test
+  public void testIntervalDaySubtractFunction() throws Exception {
+    QueryBuilder.VectorQueryReader<String,IntervalDayVector> vectorQueryReader =
+        (recordsCount, vector) -> vector.getAccessor().getAsStringBuilder(0).toString();
+
+    String result = queryBuilder()
+        .sql("select cast('P6D' as interval day) - cast('P5DT4M20S' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("0 days 23:55:40", result);
+
+    result = queryBuilder()
+        .sql("select cast('P4D' as interval day) - cast('P4DT4M' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("0 days 0:-4:00", result);
+
+    result = queryBuilder()
+        .sql("select cast('P4D' as interval day) - cast('PT4M' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("3 days 23:56:00", result);
+
+    result = queryBuilder()
+        .sql("select cast('P4D' as interval day) - cast('P5D' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("-1 day 0:00:00", result);
+
+    result = queryBuilder()
+        .sql("select cast('P4D' as interval day) - cast('P4DT23H59M59S' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("0 days -23:-59:-59", result);
+
+    result = queryBuilder()
+        .sql("select cast('P4D' as interval day) - cast('P5DT23H59S' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("-1 day -23:00:-59", result);
+
+    result = queryBuilder()
+        .sql("select cast('P2D' as interval day) - cast('P1DT4M' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("0 days 23:56:00", result);
+  }
+
+  @Test
+  public void testIntervalDayPlusFunction() throws Exception {
+    QueryBuilder.VectorQueryReader<String,IntervalDayVector> vectorQueryReader =
+        (recordsCount, vector) -> vector.getAccessor().getAsStringBuilder(0).toString();
+
+    String result = queryBuilder()
+        .sql("select cast('P1D' as interval day) + cast('P2DT1H' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("3 days 1:00:00", result);
+
+    result = queryBuilder()
+        .sql("select cast('PT12H' as interval day) + cast('PT12H' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("1 day 0:00:00", result);
+
+    result = queryBuilder()
+        .sql("select cast('PT11H' as interval day) + cast('PT12H59M60S' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("1 day 0:00:00", result);
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
index 73b4b94..fc06cb1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
@@ -30,15 +30,13 @@ import java.util.Set;
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.exceptions.UserRemoteException;
-import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
-import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.exec.vector.IntervalDayVector;
 import org.apache.drill.exec.vector.IntervalYearVector;
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.ClusterTest;
@@ -698,30 +696,53 @@ public class TestCastFunctions extends ClusterTest {
 
   @Test // DRILL-6783
   public void testCastVarCharIntervalYear() throws Exception {
-    String query = "select cast('P31M' as interval month) as i from cp.`employee.json` limit 10";
-    List<QueryDataBatch> result = queryBuilder().sql(query).results();
-    RecordBatchLoader loader = new RecordBatchLoader(cluster.drillbit().getContext().getAllocator());
-
-    QueryDataBatch b = result.get(0);
-    loader.load(b.getHeader().getDef(), b.getData());
-
-    IntervalYearVector vector = (IntervalYearVector) loader.getValueAccessorById(
-          IntervalYearVector.class,
-          loader.getValueVectorId(SchemaPath.getCompoundPath("i")).getFieldIds())
-        .getValueVector();
-
-    Set<String> resultSet = new HashSet<>();
-    for (int i = 0; i < loader.getRecordCount(); i++) {
-      String displayValue = vector.getAccessor().getAsStringBuilder(i).toString();
-      resultSet.add(displayValue);
-    }
+    Set<String> results = queryBuilder()
+        .sql("select cast('P31M' as interval month) as i from cp.`employee.json` limit 10")
+        .vectorValue(
+            "i",
+            IntervalYearVector.class,
+            (recordCount, vector) -> {
+              Set<String> r = new HashSet<>();
+              for (int i = 0; i < recordCount; i++) {
+                r.add(vector.getAccessor().getAsStringBuilder(i).toString());
+              }
+              return r;
+            }
+        );
 
     Assert.assertEquals(
-        "Casting literal string as INTERVAL should yield the same result for each row", 1, resultSet.size());
-    Assert.assertThat(resultSet, hasItem("2 years 7 months"));
+        "Casting literal string as INTERVAL should yield the same result for each row", 1, results.size());
+    Assert.assertThat(results, hasItem("2 years 7 months"));
+  }
 
-    b.release();
-    loader.clear();
+  @Test
+  public void testCastVarCharIntervalDay() throws Exception {
+    String result = queryBuilder()
+        .sql("select cast('PT1H' as interval minute) as i from (values(1))")
+        .vectorValue(
+            "i",
+            IntervalDayVector.class,
+            (recordsCount, vector) -> vector.getAccessor().getAsStringBuilder(0).toString()
+        );
+    Assert.assertEquals(result, "0 days 1:00:00");
+
+    result = queryBuilder()
+        .sql("select cast(concat('PT',107374,'M') as interval minute) as i from (values(1))")
+        .vectorValue(
+            "i",
+            IntervalDayVector.class,
+            (recordsCount, vector) -> vector.getAccessor().getAsStringBuilder(0).toString()
+        );
+    Assert.assertEquals(result, "74 days 13:34:00");
+
+    result = queryBuilder()
+        .sql("select cast(concat('PT',107375,'M') as interval minute) as i from (values(1))")
+        .vectorValue(
+            "i",
+            IntervalDayVector.class,
+            (recordsCount, vector) -> vector.getAccessor().getAsStringBuilder(0).toString()
+        );
+    Assert.assertEquals(result, "74 days 13:35:00");
   }
 
   @Test // DRILL-6959
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
index 363d326..c820408 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
@@ -108,6 +108,18 @@ public class TestDateConversions extends BaseTestQuery {
   }
 
   @Test
+  public void testToTimeWithDateTimePatternFormat() throws Exception {
+    mockUsDateFormatSymbols();
+
+    testBuilder()
+        .sqlQuery("select TO_TIME('2016-03-03 00:00', 'yyyy-MM-dd HH:mm') as `result` from (values(1))")
+        .unOrdered()
+        .baselineColumns("result")
+        .baselineValues(LocalTime.of(0,0,0))
+        .go();
+  }
+
+  @Test
   public void testPostgresTime() throws Exception {
     mockUsDateFormatSymbols();
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
index 30bff62..795cdae 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
@@ -51,6 +51,7 @@ import org.apache.drill.exec.rpc.user.UserResultsListener;
 import org.apache.drill.exec.util.VectorUtil;
 import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
 import org.apache.drill.test.BufferingQueryEventListener.QueryEvent;
 import org.apache.drill.test.ClientFixture.StatementParser;
 import org.apache.drill.test.rowSet.DirectRowSet;
@@ -58,6 +59,7 @@ import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSetReader;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.joda.time.Period;
 
 /**
  * Builder for a Drill query. Provides all types of query formats,
@@ -213,6 +215,23 @@ public class QueryBuilder {
     public QueryState finalState() { return finalState; }
   }
 
+  /**
+   * Scalar reader function interface for a set of reader methods
+   * @param <T> - reader returned type
+   */
+  private interface SingletonScalarReader<T> {
+    T read(ScalarReader reader);
+  }
+
+  /**
+   * VectorQueryReader function interface
+   * @param <V> - vector class
+   * @param <T> - result type
+   */
+  public interface VectorQueryReader<T, V> {
+    T read(int recordsCount, V vector);
+  }
+
   private final ClientFixture client;
   private QueryType queryType;
   private String queryText;
@@ -371,24 +390,92 @@ public class QueryBuilder {
   }
 
   /**
+   * Run the query which expect to return vector {@code V} representation
+   * of type {@code T} for the column {@code columnName}.
+   * <p>
+   * <pre>
+   * Example:
+   *
+   *  Set<String> results = queryBuilder()
+   *      .sql(query)
+   *      .vectorValue(
+   *        "columnName",
+   *        SomeVector.class,
+   *        (resultRecordCount, vector) -> {
+   *          Set<String> r = new HashSet<>();
+   *          for (int i = 0; i < resultRecordCount; i++) {
+   *            r.add(vector.getAccessor().getAsStringBuilder(i).toString());
+   *          }
+   *          return r;
+   *        }
+   *      );
+   * </pre>
+   * @param columnName name of the column to read
+   * @param vectorClass returned by the query vector class
+   * @param reader lambda to read the vector value representation
+   * @param <V> vector class
+   * @param <T> return type
+   * @return result produced by {@code reader} lambda or {@code null} if no records returned from the query
+   *
+   */
+  @SuppressWarnings("unchecked")
+  public <T, V> T vectorValue(String columnName, Class<V> vectorClass, VectorQueryReader<T, V> reader)
+      throws RpcException, SchemaChangeException {
+
+    List<QueryDataBatch> result = results();
+    RecordBatchLoader loader = new RecordBatchLoader(client.allocator());
+    QueryDataBatch queryDataBatch = null;
+
+    try {
+      queryDataBatch = result.get(0);
+      loader.load(queryDataBatch.getHeader().getDef(), queryDataBatch.getData());
+
+      V vector = (V) loader.getValueAccessorById(
+          vectorClass,
+          loader.getValueVectorId(SchemaPath.getCompoundPath(columnName)).getFieldIds())
+          .getValueVector();
+
+      return (loader.getRecordCount() > 0) ? reader.read(loader.getRecordCount(), vector) : null;
+    } finally {
+      if (queryDataBatch != null) {
+        queryDataBatch.release();
+      }
+      loader.clear();
+    }
+  }
+
+  /**
    * Run the query that is expected to return (at least) one row
-   * with the only (or first) column returning a long value.
-   * The long value cannot be null.
+   * with the only (or first) column returning a {@link T} value.
+   * The {@link T} value cannot be null.
    *
    * @return the value of the first column of the first row
    * @throws RpcException if anything goes wrong
    */
-
-  public long singletonLong() throws RpcException {
+  private <T> T singletonGeneric(SingletonScalarReader<T> scalarReader) throws RpcException {
     RowSet rowSet = rowSet();
     if (rowSet == null) {
       throw new IllegalStateException("No rows returned");
     }
-    RowSetReader reader = rowSet.reader();
-    reader.next();
-    long value = reader.scalar(0).getLong();
-    rowSet.clear();
-    return value;
+    try {
+      RowSetReader reader = rowSet.reader();
+      reader.next();
+      return scalarReader.read(reader.scalar(0));
+    } finally {
+      rowSet.clear();
+    }
+  }
+
+  /**
+   * Run the query that is expected to return (at least) one row
+   * with the only (or first) column returning a long value.
+   * The long value cannot be null.
+   *
+   * @return the value of the first column of the first row
+   * @throws RpcException if anything goes wrong
+   */
+  public long singletonLong() throws RpcException {
+    return singletonGeneric(ScalarReader::getLong);
   }
 
   /**
@@ -399,17 +486,8 @@ public class QueryBuilder {
    * @return the value of the first column of the first row
    * @throws RpcException if anything goes wrong
    */
-
   public double singletonDouble() throws RpcException {
-    RowSet rowSet = rowSet();
-    if (rowSet == null) {
-      throw new IllegalStateException("No rows returned");
-    }
-    RowSetReader reader = rowSet.reader();
-    reader.next();
-    double value = reader.scalar(0).getDouble();
-    rowSet.clear();
-    return value;
+    return singletonGeneric(ScalarReader::getDouble);
   }
 
   /**
@@ -420,17 +498,20 @@ public class QueryBuilder {
    * @return the value of the first column of the first row
    * @throws RpcException if anything goes wrong
    */
-
   public int singletonInt() throws RpcException {
-    RowSet rowSet = rowSet();
-    if (rowSet == null) {
-      throw new IllegalStateException("No rows returned");
-    }
-    RowSetReader reader = rowSet.reader();
-    reader.next();
-    int value = reader.scalar(0).getInt();
-    rowSet.clear();
-    return value;
+    return singletonGeneric(ScalarReader::getInt);
+  }
+
+  /**
+   * Run the query that is expected to return (at least) one row
+   * with the only (or first) column returning a {@link Period} value.
+   * The {@link Period} value cannot be null.
+   *
+   * @return the value of the first column of the first row
+   * @throws RpcException if anything goes wrong
+   */
+  public Period singletonPeriod() throws RpcException {
+    return singletonGeneric(ScalarReader::getPeriod);
   }
 
   /**
@@ -443,20 +524,7 @@ public class QueryBuilder {
    */
 
   public String singletonString() throws RpcException {
-    RowSet rowSet = rowSet();
-    if (rowSet == null) {
-      throw new IllegalStateException("No rows returned");
-    }
-    RowSetReader reader = rowSet.reader();
-    reader.next();
-    String value;
-    if (reader.scalar(0).isNull()) {
-      value = null;
-    } else {
-      value = reader.scalar(0).getString();
-    }
-    rowSet.clear();
-    return value;
+    return singletonGeneric(ScalarReader::getString);
   }
 
   /**


[drill] 01/05: DRILL-7276: Fixed an XSS vulnerability in Drill Web-UI query profile page

Posted by sa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sachouche pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 5542cc98f38003f028d97f95d257604c91723d70
Author: Anton Gozhiy <an...@gmail.com>
AuthorDate: Thu May 23 19:41:45 2019 +0300

    DRILL-7276: Fixed an XSS vulnerability in Drill Web-UI query profile page
---
 exec/java-exec/src/main/resources/rest/alertModals.ftl    | 11 ++++++++++-
 exec/java-exec/src/main/resources/rest/static/js/graph.js |  2 +-
 2 files changed, 11 insertions(+), 2 deletions(-)

diff --git a/exec/java-exec/src/main/resources/rest/alertModals.ftl b/exec/java-exec/src/main/resources/rest/alertModals.ftl
index 320d96d..17f0275 100644
--- a/exec/java-exec/src/main/resources/rest/alertModals.ftl
+++ b/exec/java-exec/src/main/resources/rest/alertModals.ftl
@@ -62,7 +62,7 @@
         var inputValuesKeys = Object.keys(inputValues);
         for (i=0; i<inputValuesKeys.length; ++i) {
             let currKey=inputValuesKeys[i];
-            updatedHtml=updatedHtml.replace(currKey, inputValues[currKey]);
+            updatedHtml=updatedHtml.replace(currKey, escapeHtml(inputValues[currKey]));
         }
         modalBody.innerHTML=updatedHtml;
       }
@@ -70,6 +70,15 @@
       $('#errorModal').modal('show');
     }
 
+    function escapeHtml(str) {
+        return str.replace(/&/g,'&amp;')
+            .replace(/</g,'&lt;')
+            .replace(/>/g,'&gt;')
+            .replace(/"/g,'&quot;')
+            .replace(/'/g,'&#x27;')
+            .replace(/\//g,'&#x2F;');
+    }
+
     //Map of error messages to populate the alert modal
     var errorMap = {
         "userNameMissing": {
diff --git a/exec/java-exec/src/main/resources/rest/static/js/graph.js b/exec/java-exec/src/main/resources/rest/static/js/graph.js
index 5801245..81042e8 100644
--- a/exec/java-exec/src/main/resources/rest/static/js/graph.js
+++ b/exec/java-exec/src/main/resources/rest/static/js/graph.js
@@ -390,7 +390,7 @@ $(window).on('load',(function () {
         // add json profile on click
         $('#full-json-profile-json').on('shown.bs.collapse', function (e) {
             if (jsonprofileshown) return;
-            $('#full-json-profile-json').html(JSON.stringify(globalconfig.profile, null, 4));
+            $('#full-json-profile-json').text(JSON.stringify(globalconfig.profile, null, 4)).html();
         });
 
         //builddomtable(d3.select("#timing-table")


[drill] 04/05: DRILL-7196: Queries are still runnable on disabled plugins

Posted by sa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sachouche pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit e8b921bae513f00ef0ede0d2787f311aea60cea0
Author: Dmytriy Grinchenko <dm...@gmail.com>
AuthorDate: Tue May 14 11:48:50 2019 +0300

    DRILL-7196: Queries are still runnable on disabled plugins
    
    - Storage client is not created anymore for disabled plugins
    - GET "/storage/{name}.json" endpoint now working with
      plugin configuration directly, without client instantination.
      It have increased UI responsitivity.
    - Hbase and mongo base test classes refactored to honor enabled
      plugin attribute
    - Fixed path contructor for mongo test datasets:
      Now it is cross-platform
    - Fixed test json files format which using plugin definitions
    - Code cleanup
---
 common/src/test/resources/basic_physical.json      |  51 +++---
 common/src/test/resources/dsort-logical.json       |  48 ++---
 common/src/test/resources/jdbc_plan.json           |   4 +-
 common/src/test/resources/simple_plan.json         | 159 ++++++++---------
 common/src/test/resources/storage_engine_plan.json |   5 +-
 contrib/native/client/resources/simple_plan.json   | 126 ++++++-------
 .../exec/store/hbase/HBaseStoragePluginConfig.java |   2 +-
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../java/org/apache/drill/hbase/BaseHBaseTest.java |  30 ++--
 .../hbase/hbase_scan_screen_physical.json          |  31 ++--
 .../hbase_scan_screen_physical_column_select.json  |  33 ++--
 .../hbase_scan_screen_physical_family_select.json  |  33 ++--
 .../exec/store/hive/schema/HiveSchemaFactory.java  |   1 -
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../test/resources/bootstrap-storage-plugins.json  |   8 +-
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../drill/exec/store/mongo/MongoTestBase.java      |  71 ++------
 .../drill/exec/store/mongo/MongoTestConstants.java |  78 ++++-----
 .../drill/exec/store/mongo/MongoTestSuit.java      |  82 ++++-----
 .../exec/store/mongo/TestMongoFilterPushDown.java  |  34 ++--
 .../exec/store/mongo/TestMongoProjectPushDown.java |  36 ++--
 .../drill/exec/store/mongo/TestMongoQueries.java   |  79 +++++----
 .../drill/exec/store/mongo/TestTableGenerator.java |  10 +-
 .../main/resources/bootstrap-storage-plugins.json  |   8 +-
 .../org/apache/calcite/jdbc/DynamicRootSchema.java |  16 +-
 .../drill/exec/server/rest/StorageResources.java   |   9 +-
 .../exec/store/StoragePluginRegistryImpl.java      |  38 ++--
 .../main/resources/bootstrap-storage-plugins.json  | 194 ++++++++++-----------
 .../drill/exec/store/store/TestDisabledPlugin.java |   6 +
 .../src/test/resources/agg/hashagg/q6.json         |  77 ++++----
 .../src/test/resources/agg/hashagg/q7_1.json       |  69 ++++----
 .../src/test/resources/agg/hashagg/q7_2.json       |  69 ++++----
 .../src/test/resources/agg/hashagg/q7_3.json       |  67 +++----
 .../src/test/resources/agg/hashagg/q8.json         |  61 +++----
 .../src/test/resources/agg/hashagg/q8_1.json       |  95 +++++-----
 .../src/test/resources/common/test_hashtable1.json |  64 ++++---
 .../test/resources/decimal/cast_decimal_float.json |  67 ++++---
 .../test/resources/decimal/cast_decimal_int.json   |  67 ++++---
 .../resources/decimal/cast_decimal_vardecimal.json | 106 ++++++-----
 .../test/resources/decimal/cast_float_decimal.json |  74 ++++----
 .../test/resources/decimal/cast_int_decimal.json   |  67 ++++---
 .../resources/decimal/cast_simple_decimal.json     |  77 ++++----
 .../resources/decimal/cast_vardecimal_decimal.json |  73 ++++----
 .../decimal/simple_decimal_arithmetic.json         | 102 ++++++-----
 .../resources/decimal/simple_decimal_math.json     |  69 ++++----
 .../resources/decimal/test_decimal_complex.json    |  95 +++++-----
 .../decimal/test_decimal_dense_sparse.json         | 129 ++++++--------
 .../decimal/test_decimal_sort_complex.json         |  98 +++++------
 .../decimal/test_decimal_sparse_dense_dense.json   |  40 ++---
 .../functions/cast/testCastVarCharNull.json        |  64 ++++---
 .../cast/test_cast_varchar_to_bigint.json          |  60 ++++---
 .../conv/conversionTestWithLogicalPlan.json        | 138 +++++++++------
 .../src/test/resources/functions/date/age.json     |  61 ++++---
 .../functions/date/date_difference_arithmetic.json |  57 +++---
 .../functions/date/date_interval_arithmetic.json   | 100 +++++++----
 .../functions/date/interval_arithmetic.json        |  93 ++++++----
 .../src/test/resources/functions/date/to_char.json |  57 +++---
 .../resources/functions/date/to_date_type.json     |  60 ++++---
 .../src/test/resources/functions/extractFrom.json  |  81 +++++----
 .../test/resources/functions/test_covariance.json  |  55 +++---
 .../resources/functions/test_logical_aggr.json     |  53 +++---
 .../resources/functions/test_stddev_variance.json  |  24 +--
 exec/java-exec/src/test/resources/hive/test.json   | 141 ++++++++-------
 .../src/test/resources/join/hashJoinExpr.json      |   2 +-
 .../src/test/resources/join/hash_join.json         | 109 ++++++------
 .../src/test/resources/join/hj_exchanges.json      |  19 +-
 .../src/test/resources/join/hj_exchanges1.json     |   3 +-
 .../resources/join/hj_multi_condition_join.json    | 123 +++++++------
 .../src/test/resources/join/mergeJoinExpr.json     |   2 +-
 .../resources/join/merge_inner_single_batch.json   |  81 +++++----
 .../test/resources/join/merge_join_nullkey.json    |   2 +-
 .../src/test/resources/join/merge_multi_batch.json |  82 +++++----
 .../test/resources/join/merge_single_batch.json    |  82 +++++----
 .../test/resources/join/mj_multi_condition.json    |   5 +-
 .../largefiles/merging_receiver_large_data.json    |  11 +-
 .../test/resources/lateraljoin/lateralplan1.json   |   9 +-
 .../src/test/resources/limit/limit_exchanges.json  |   9 +-
 .../src/test/resources/logical_group.json          | 133 +++++++-------
 .../src/test/resources/logical_json_scan.json      |  51 +++---
 .../src/test/resources/logical_order.json          |  85 ++++-----
 .../src/test/resources/logical_string_filter.json  |  77 ++++----
 .../test/resources/parquet/par_writer_test.json    |  42 ++---
 .../parquet/parquet_group_scan_union_screen.json   |  81 ++++-----
 .../test/resources/parquet/parquet_nullable.json   |  66 ++++---
 .../resources/parquet/parquet_nullable_varlen.json |  68 ++++----
 .../resources/parquet/parquet_scan_screen.json     |  70 ++++----
 .../parquet_scan_screen_read_entry_replace.json    |   3 +-
 .../parquet_scan_union_screen_physical.json        |  48 +++--
 .../parquet/parquet_selective_column_read.json     |  40 ++---
 .../src/test/resources/physical_filter.json        |  64 +++----
 .../src/test/resources/physical_join.json          | 182 +++++++++----------
 .../test/resources/physical_json_scan_test1.json   |  54 +++---
 .../test/resources/physical_order_varbinary.json   |  73 ++++----
 .../record/vector/test_all_date_literals.json      |  61 ++++---
 .../test/resources/record/vector/test_date.json    |  50 +++---
 .../resources/record/vector/test_date_add.json     |  53 +++---
 .../resources/record/vector/test_interval.json     |  64 +++----
 .../resources/record/vector/test_sort_date.json    |  91 +++++-----
 .../resources/record/vector/test_timestamp.json    |  50 +++---
 .../src/test/resources/scan_screen_logical.json    |  75 ++++----
 .../test/resources/sender/broadcast_exchange.json  |  92 +++++-----
 .../src/test/resources/sender/hash_exchange2.json  |  84 +++++----
 .../json/project_pushdown_json_physical_plan.json  |   4 +-
 .../src/test/resources/store/text/test.json        |  73 ++++----
 .../test/resources/writer/simple_csv_writer.json   | 158 +++++++++--------
 107 files changed, 3231 insertions(+), 3142 deletions(-)

diff --git a/common/src/test/resources/basic_physical.json b/common/src/test/resources/basic_physical.json
index 21e4c9a..675a034 100644
--- a/common/src/test/resources/basic_physical.json
+++ b/common/src/test/resources/basic_physical.json
@@ -1,40 +1,41 @@
 {
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
+    "head":{
+        "type":"APACHE_DRILL_PHYSICAL",
+        "version":"1",
+        "generator":{
+            "type":"manual"
         }
     },
-    storage:{
-        fs1:{
-            type:"mock"
+    "storage":{
+        "fs1":{
+            "type":"mock",
+            "enabled": true
         }
     },
-    graph:[
+    "graph":[
         {
-            @id:1,
-            pop:"scan",
-            storageengine:"fs1",
-            entries:[
+            "@id":1,
+            "pop":"scan",
+            "storageengine":"fs1",
+            "entries":[
                 {"test1"}
             ],
-            output:[
-                { "name":"key", mode: "VECTOR", type:"SINT32"},
-                { "name":"value", mode: "VECTOR", type:"SINT32"}
+            "output":[
+                { "name":"key", "mode": "VECTOR", "type":"SINT32"},
+                { "name":"value", "mode": "VECTOR", "type":"SINT32"}
             ]
         },
         {
-            @id:2,
-            child:1,
-            pop: "store",
-            mode: "SYSTEM_CHOICE",
-            storageengine: "fs1",
-            entries:[
+            "@id":2,
+            "child":1,
+            "pop": "store",
+            "mode": "SYSTEM_CHOICE",
+            "storageengine": "fs1",
+            "entries":[
                 {
-                    path:"/sort/sorted/${partition_number}.seq",
-                    key:"Text",
-                    type:"JAVA_SEQUENCE"
+                    "path":"/sort/sorted/${partition_number}.seq",
+                    "key":"Text",
+                    "type":"JAVA_SEQUENCE"
                 }
             ]
         }
diff --git a/common/src/test/resources/dsort-logical.json b/common/src/test/resources/dsort-logical.json
index 3e32ad6..4ebb46a 100644
--- a/common/src/test/resources/dsort-logical.json
+++ b/common/src/test/resources/dsort-logical.json
@@ -1,38 +1,38 @@
 {
-  head:{ type:"apache_drill_logical_plan", version:"1", generator:{ type:"manual", info:"na"}}},
-  storage:[ { type:"fs", name:"fs1", root:"file:///" }],
-  query:[ { op: "sequence", sequence: [
+  "head":{ "type":"apache_drill_logical_plan", "version":"1", "generator":{ "type":"manual", "info":"na"}},
+  "storage":[ { "type":"fs", "name":"fs1", "root":"file:///", "enabled":  true}],
+  "query":[ { "op": "sequence", "sequence": [
     {
-      op:"scan",
-      storageengine:"fs1",
-      ref: "data",
-      selection: {
-        path: "/sort/unsorted/*.seq",
-        type: "JAVA_SEQUENCE"
+      "op":"scan",
+      "storageengine":"fs1",
+      "ref": "data",
+      "selection": {
+        "path": "/sort/unsorted/*.seq",
+        "type": "JAVA_SEQUENCE"
       }
     },
     {
-      op: "order",
-      orderings: [
-        {order: "desc", expr: "data.key" }
+      "op": "order",
+      "orderings": [
+        {"order": "desc", "expr": "data.key" }
       ]
     },
     {
-      op: "project",
-      projections: [
-        { ref: "output.key", expr: "data.key" },
-        { ref: "output.value", expr: "data.value" }
+      "op": "project",
+      "projections": [
+        { "ref": "output.key", "expr": "data.key" },
+        { "ref": "output.value", "expr": "data.value" }
       ]
     },
     {
-      op: "store",
-      storageengine: "fs1",
-      target: {
-        path: "/sort/sorted/${partition}.seq",
-        type: "JAVA_SEQUENCE",
-        partition: {
-          type: "ORDERED",
-          exprs: ["key"]
+      "op": "store",
+      "storageengine": "fs1",
+      "target": {
+        "path": "/sort/sorted/${partition}.seq",
+        "type": "JAVA_SEQUENCE",
+        "partition": {
+          "type": "ORDERED",
+          "exprs": ["key"]
         }
       }
     }
diff --git a/common/src/test/resources/jdbc_plan.json b/common/src/test/resources/jdbc_plan.json
index 2d3b9d2..832dd15 100644
--- a/common/src/test/resources/jdbc_plan.json
+++ b/common/src/test/resources/jdbc_plan.json
@@ -4,8 +4,8 @@
         "generator":{"type":"manual","info":"na"}
     },
     "storage":{
-        "donuts-json":{"type":"classpath"},
-        "queue":{"type":"queue"}
+        "donuts-json":{"type":"classpath", "enabled":  true},
+        "queue":{"type":"queue", "enabled":  true}
     },
     "query":[
         {"op":"sequence",
diff --git a/common/src/test/resources/simple_plan.json b/common/src/test/resources/simple_plan.json
index 46ee6af..d5d8e5d 100644
--- a/common/src/test/resources/simple_plan.json
+++ b/common/src/test/resources/simple_plan.json
@@ -1,129 +1,132 @@
 {
-  head:{
-    type:"apache_drill_logical_plan",
-    version:"1",
-    generator:{
-      type:"manual",
-      info:"na"
+  "head":{
+    "type":"apache_drill_logical_plan",
+    "version":"1",
+    "generator":{
+      "type":"manual",
+      "info":"na"
     }
   },
-  storage:{
-    logs: {
-      type:"text",
-      file: "local://logs/*.log",
-      compress:"gzip",
-      line-delimiter:"\n",
-      record-maker:{
-        type:"first-row",
-        delimiter:","
-      }
+  "storage":[
+    {
+      "type":"text",
+      "file": "local://logs/*.log",
+      "compress":"gzip",
+      "line-delimiter":"\n",
+      "record-maker":{
+        "type":"first-row",
+        "delimiter":","
+      },
+      "enabled": true
     },
     {
-      type:"mongo",
-      name:"users",
-      connection:"mongodb://blue:red@localhost/users"
+      "type":"mongo",
+      "name":"users",
+      "connection":"mongodb://blue:red@localhost/users",
+      "enabled": true
     },
     {
-      type:"mysql",
-      name:"mysql",
-      connection:"jdbc:mysql://localhost/main"
+      "type":"mysql",
+      "name":"mysql",
+      "connection":"jdbc:mysql://localhost/main",
+      "enabled": true
     }
   ],
-  query:[
+  "query":[
     {
-      @id:"1",
-      op:"scan",
-      memo:"initial_scan",
-      storageengine:"local-logs",
-      selection: {}
+      "@id": 1 ,
+      "op":"scan",
+      "memo":"initial_scan",
+      "storageengine":"local-logs",
+      "selection": {}
     },
     {
-      @id:"2",
-      input:"1",
-      memo:"transform1",
-      op:"transform",
-      transforms:[
+      "@id": 2 ,
+      "input": 1 ,
+      "memo":"transform1",
+      "op":"transform",
+      "transforms":[
         {
-          ref:"userId",
-          expr:"regex_like('activity.cookie', \"persistent=([^;]*)\")"
+          "ref":"userId",
+          "expr":"regex_like('activity.cookie', \"persistent=([^;]*)\")"
         },
         {
-          ref:"session",
-          expr:"regex_like('activity.cookie', \"session=([^;]*)\")"
+          "ref":"session",
+          "expr":"regex_like('activity.cookie', \"session=([^;]*)\")"
         }
       ]
     },
     {
-      @id:"3",
-      input:"2",
-      memo:"transform2",
-      op:"transform",
-      transforms:[
+      "@id": 3 ,
+      "input": 2 ,
+      "memo":"transform2",
+      "op":"transform",
+      "transforms":[
         {
-          ref:"userId",
-          expr:"regex_like('activity.cookie', \"persistent=([^;]*)\")"
+          "ref":"userId",
+          "expr":"regex_like('activity.cookie', \"persistent=([^;]*)\")"
         },
         {
-          ref:"session",
-          expr:"regex_like('activity.cookie', \"session=([^;]*)\")"
+          "ref":"session",
+          "expr":"regex_like('activity.cookie', \"session=([^;]*)\")"
         }
       ]
     },
     {
-      @id:"7",
-      input:"3",
-      op:"sequence",
-      do:[
+      "@id": 7 ,
+      "input": 3 ,
+      "op":"sequence",
+      "do":[
         {
-          op:"transform",
-          memo:"seq_transform",
-          transforms:[
+          "op":"transform",
+          "memo":"seq_transform",
+          "transforms":[
             {
-              ref:"happy",
-              expr:"regex_like('ep2', \"dink\")"
+              "ref":"happy",
+              "expr":"regex_like('ep2', \"dink\")"
             }
           ]
         }
         ,
         {
-          op:"transform",
-          memo:"last_transform",
-          transforms:[
+          "op":"transform",
+          "memo":"last_transform",
+          "transforms":[
             {
-              ref:"abc",
-              expr:"123"
+              "ref":"abc",
+              "expr":"123"
             }
           ]
         }
       ]
     },
     {
-      @id:"10",
-      input:"3",
-      op:"transform",
-      memo:"t3",
-      transforms:[
+      "@id": 10,
+      "input": 3,
+      "op":"transform",
+      "memo":"t3",
+      "transforms":[
         {
-          ref:"happy",
-          expr:"regex_like('ep2', \"dink\")"
+          "ref":"happy",
+          "expr":"regex_like('ep2', \"dink\")"
         }
       ]
     },
     {
-      @id:12,
-      op:"join",
-      type: "inner",
-      left:"7",
-      right:"10",
-      conditions: [{relationship:"==", left: "1", right: "1" }]
+      "@id":12,
+      "op":"join",
+      "type": "inner",
+      "left":"7",
+      "right":"10",
+      "conditions": [{"relationship":"==", "left": "1", "right": "1" }]
     }
     ,
     {
-      input: 12,
-      op: "store",
-      memo: "output sink",
-      target: {
-        file: "console:///stdout"
+      "input": 12,
+      "op": "store",
+      "memo": "output sink",
+      "target": {
+        "file": "console:///stdout"
       }
     }
   ]
diff --git a/common/src/test/resources/storage_engine_plan.json b/common/src/test/resources/storage_engine_plan.json
index 6c9a6d6..c91d498 100644
--- a/common/src/test/resources/storage_engine_plan.json
+++ b/common/src/test/resources/storage_engine_plan.json
@@ -9,8 +9,9 @@
   },
   "storage" : {
       "mock-engine": {
-        type : "mock",
-        url : "http://www.apache.org/"
+        "type" : "mock",
+        "url" : "http://www.apache.org/",
+        "enabled": true
       }
   },
   "query" : [
diff --git a/contrib/native/client/resources/simple_plan.json b/contrib/native/client/resources/simple_plan.json
index 8fede75..f79e951 100644
--- a/contrib/native/client/resources/simple_plan.json
+++ b/contrib/native/client/resources/simple_plan.json
@@ -1,93 +1,93 @@
 {
-  head:{
-    type:"APACHE_DRILL_LOGICAL",
-    version:"1",
-    generator:{
-      type:"manual",
-      info:"na"
+  "head":{
+    "type":"APACHE_DRILL_LOGICAL",
+    "version":"1",
+    "generator":{
+      "type":"manual",
+      "info":"na"
     }
   },
-  storage:{
-    console: {type: "console"},
-    fs1: {type: "fs", root:"file:///"},
-    cp: {type: "classpath"}
+  "storage":{
+    "console": {"type": "console", "enabled":  true},
+    "fs1": {"type": "fs", "root":"file:///", "enabled":  true},
+    "cp": {"type": "classpath", "enabled":  true}
   },
-  query:[
+  "query":[
 
                 {
-                  @id:"1",
-                  op: "scan",
-                  memo: "initial_scan",
-                  ref: "donuts",
-                  storageengine: "cp",
-                  selection: {
-                    path: "/donuts.json",
-                    type: "JSON"
+                  "@id":"1",
+                  "op": "scan",
+                  "memo": "initial_scan",
+                  "ref": "donuts",
+                  "storageengine": "cp",
+                  "selection": {
+                    "path": "/donuts.json",
+                    "type": "JSON"
                   }
                 },
         {
-                  @id:"2",
-                  input:"1",
-                  op: "transform",
-                  transforms: [
-                    { ref: "quantity", expr: "donuts.sales"}
+                  "@id":"2",
+                  "input":"1",
+                  "op": "transform",
+                  "transforms": [
+                    { "ref": "quantity", "expr": "donuts.sales"}
                   ]
                 },
                 {
-                  @id:"3",
-                  input:"2",
-                  op: "filter",
-                  expr: "donuts.ppu < 1.00"
+                  "@id":"3",
+                  "input":"2",
+                  "op": "filter",
+                  "expr": "donuts.ppu < 1.00"
                 },
                 {
-                  @id:"4",
-                  input:"3",
-                  op: "segment",
-                  ref: "ppusegment",
-                  exprs: ["donuts.ppu"]
+                  "@id":"4",
+                  "input":"3",
+                  "op": "segment",
+                  "ref": "ppusegment",
+                  "exprs": ["donuts.ppu"]
                 },
                 {
-                  @id:"5",
-                  input:"4",
-                  op: "collapsingaggregate",
-                  within: "ppusegment",
-                  carryovers: ["donuts.ppu"],
-                  aggregations: [
-                    { ref: "donuts.typeCount",  expr: "count(1)" },
-                    { ref: "donuts.quantity",  expr: "sum(quantity)" },
-                    { ref: "donuts.sales",  expr: "sum(donuts.ppu * quantity)" }
+                  "@id":"5",
+                  "input":"4",
+                  "op": "collapsingaggregate",
+                  "within": "ppusegment",
+                  "carryovers": ["donuts.ppu"],
+                  "aggregations": [
+                    { "ref": "donuts.typeCount",  "expr": "count(1)" },
+                    { "ref": "donuts.quantity",  "expr": "sum(quantity)" },
+                    { "ref": "donuts.sales",  "expr": "sum(donuts.ppu * quantity)" }
                   ]
                 },
                 {
-                  @id:"6",
-                  input:"5",
-                  op: "order",
-                  orderings: [
-                    {order: "DESC", expr: "donuts.ppu" }
+                  "@id":"6",
+                  "input":"5",
+                  "op": "order",
+                  "orderings": [
+                    {"order": "DESC", "expr": "donuts.ppu" }
                   ]
                 },
                 {
-                  @id:"7",
-                  input:"6",
-                  op: "project",
-                  projections: [
-                    { ref: "output.output", expr: "donuts" }
+                  "@id":"7",
+                  "input":"6",
+                  "op": "project",
+                  "projections": [
+                    { "ref": "output.output", "expr": "donuts" }
                   ]
                 },
                {
-          @id:"8",
-                  input:"7",
-                  op: "limit",
-          first: 0,
-          last: 100
+          "@id":"8",
+                  "input":"7",
+                  "op": "limit",
+          "first": 0,
+          "last": 100
         },
                 {
-                  @id:"9",
-                  input:"8",
-                  op: "store",
-                  memo: "output sink",
-                  storageengine: "console",
-                  target: {pipe: "STD_OUT"}
+                  "@id":"9",
+                  "input":"8",
+                  "op": "store",
+                  "memo": "output sink",
+                  "storageengine": "console",
+                  "target": {"pipe": "STD_OUT"}
                 }
   ]
 }
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePluginConfig.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePluginConfig.java
index 7a27f79..946f6f5 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePluginConfig.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePluginConfig.java
@@ -106,7 +106,7 @@ public class HBaseStoragePluginConfig extends StoragePluginConfigBase implements
   }
 
   @JsonIgnore
-  public String getZookeeperport() {
+  public String getZookeeperPort() {
     return getHBaseConf().get(HBASE_ZOOKEEPER_PORT);
   }
 
diff --git a/contrib/storage-hbase/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-hbase/src/main/resources/bootstrap-storage-plugins.json
index 530a407..1b050ee 100644
--- a/contrib/storage-hbase/src/main/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-hbase/src/main/resources/bootstrap-storage-plugins.json
@@ -1,12 +1,12 @@
 {
   "storage":{
-    hbase : {
-      type:"hbase",
-      config : {
+    "hbase" : {
+      "type":"hbase",
+      "config" : {
         "hbase.zookeeper.quorum" : "localhost",
         "hbase.zookeeper.property.clientPort" : 2181
       },
-      enabled: false
+      "enabled": false
     }
   }
 }
diff --git a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java
index 33cd700..d2b3f16 100644
--- a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java
+++ b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java
@@ -41,28 +41,24 @@ public class BaseHBaseTest extends BaseTestQuery {
   public static final String HBASE_STORAGE_PLUGIN_NAME = "hbase";
 
   protected static Configuration conf = HBaseConfiguration.create();
-
   protected static HBaseStoragePlugin storagePlugin;
-
   protected static HBaseStoragePluginConfig storagePluginConfig;
 
   @BeforeClass
   public static void setupDefaultTestCluster() throws Exception {
-    /*
-     * Change the following to HBaseTestsSuite.configure(false, true)
-     * if you want to test against an externally running HBase cluster.
-     */
-    HBaseTestsSuite.configure(true /*manageHBaseCluster*/, true /*createTables*/);
+    boolean isManaged = Boolean.valueOf(System.getProperty("drill.hbase.tests.managed", "true"));
+    HBaseTestsSuite.configure(isManaged, true);
     HBaseTestsSuite.initCluster();
 
     BaseTestQuery.setupDefaultTestCluster();
 
-    final StoragePluginRegistry pluginRegistry = getDrillbitContext().getStorage();
-    storagePlugin = (HBaseStoragePlugin) pluginRegistry.getPlugin(HBASE_STORAGE_PLUGIN_NAME);
-    storagePluginConfig = storagePlugin.getConfig();
+    StoragePluginRegistry pluginRegistry = getDrillbitContext().getStorage();
+    storagePluginConfig = new HBaseStoragePluginConfig(null, false);
     storagePluginConfig.setEnabled(true);
     storagePluginConfig.setZookeeperPort(HBaseTestsSuite.getZookeeperPort());
-    pluginRegistry.createOrUpdate(HBASE_STORAGE_PLUGIN_NAME, storagePluginConfig, true);
+
+    storagePlugin = (HBaseStoragePlugin) pluginRegistry.createOrUpdate(HBASE_STORAGE_PLUGIN_NAME,
+        storagePluginConfig, true);
   }
 
   @AfterClass
@@ -72,11 +68,14 @@ public class BaseHBaseTest extends BaseTestQuery {
 
   protected String getPlanText(String planFile, String tableName) throws IOException {
     return Files.asCharSource(DrillFileUtils.getResourceAsFile(planFile), Charsets.UTF_8).read()
-        .replaceFirst("\"hbase\\.zookeeper\\.property\\.clientPort\".*:.*\\d+", "\"hbase.zookeeper.property.clientPort\" : " + HBaseTestsSuite.getZookeeperPort())
+        .replaceFirst(
+            "\"hbase\\.zookeeper\\.property\\.clientPort\".*:.*\\d+",
+            "\"hbase.zookeeper.property.clientPort\" : " + HBaseTestsSuite.getZookeeperPort())
         .replace("[TABLE_NAME]", tableName);
   }
 
-  protected void runHBasePhysicalVerifyCount(String planFile, String tableName, int expectedRowCount) throws Exception{
+  protected void runHBasePhysicalVerifyCount(String planFile, String tableName, int expectedRowCount)
+      throws Exception {
     String physicalPlan = getPlanText(planFile, tableName);
     List<QueryDataBatch> results = testPhysicalWithResults(physicalPlan);
     logResultAndVerifyRowCount(results, expectedRowCount);
@@ -87,12 +86,13 @@ public class BaseHBaseTest extends BaseTestQuery {
     return testSqlWithResults(sql);
   }
 
-  protected void runHBaseSQLVerifyCount(String sql, int expectedRowCount) throws Exception{
+  protected void runHBaseSQLVerifyCount(String sql, int expectedRowCount) throws Exception {
     List<QueryDataBatch> results = runHBaseSQLlWithResults(sql);
     logResultAndVerifyRowCount(results, expectedRowCount);
   }
 
-  private void logResultAndVerifyRowCount(List<QueryDataBatch> results, int expectedRowCount) throws SchemaChangeException {
+  private void logResultAndVerifyRowCount(List<QueryDataBatch> results, int expectedRowCount)
+      throws SchemaChangeException {
     int rowCount = logResult(results);
     if (expectedRowCount != -1) {
       Assert.assertEquals(expectedRowCount, rowCount);
diff --git a/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical.json b/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical.json
index 7f9015b..d452cc8 100644
--- a/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical.json
+++ b/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical.json
@@ -1,29 +1,30 @@
 {
-  head : {
-    type : "APACHE_DRILL_PHYSICAL",
-    version : 1,
-    generator : {
-      type : "manual"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "manual"
     }
   },
-  graph : [ {
-    pop : "hbase-scan",
-    @id : 1,
-    hbaseScanSpec : {
-      tableName : "[TABLE_NAME]"
+  "graph" : [ {
+    "pop" : "hbase-scan",
+    "@id" : 1,
+    "hbaseScanSpec" : {
+      "tableName" : "[TABLE_NAME]"
     },
-    storage:
+    "storage":
     {
       "type":"hbase",
-      config : {
+      "enabled": true,
+      "config" : {
         "hbase.zookeeper.quorum" : "localhost",
         "hbase.zookeeper.property.clientPort" : 2181
       }
     }
   },
   {
-    pop : "screen",
-    @id : 2,
-    child : 1
+    "pop" : "screen",
+    "@id" : 2,
+    "child" : 1
   } ]
 }
\ No newline at end of file
diff --git a/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical_column_select.json b/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical_column_select.json
index f399f6f..cb48b26 100644
--- a/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical_column_select.json
+++ b/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical_column_select.json
@@ -1,32 +1,33 @@
 {
-  head : {
-    type : "APACHE_DRILL_PHYSICAL",
-    version : 1,
-    generator : {
-      type : "manual"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "manual"
     }
   },
-  graph : [ {
-    pop : "hbase-scan",
-    @id : 1,
-    hbaseScanSpec : {
-      tableName : "[TABLE_NAME]"
+  "graph" : [ {
+    "pop" : "hbase-scan",
+    "@id" : 1,
+    "hbaseScanSpec" : {
+      "tableName" : "[TABLE_NAME]"
     },
-    storage:
+    "storage":
     {
       "type":"hbase",
-      config : {
+      "enabled": true,
+      "config" : {
         "hbase.zookeeper.quorum" : "localhost",
         "hbase.zookeeper.property.clientPort" : 2181
       }
     },
-    columns: [
+    "columns": [
       "`f2`.c1", "`f2`.c2", "row_key"
     ]
   },
   {
-    pop : "screen",
-    @id : 2,
-    child : 1
+    "pop" : "screen",
+    "@id" : 2,
+    "child" : 1
   } ]
 }
\ No newline at end of file
diff --git a/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical_family_select.json b/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical_family_select.json
index 0002164..75c9123 100644
--- a/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical_family_select.json
+++ b/contrib/storage-hbase/src/test/resources/hbase/hbase_scan_screen_physical_family_select.json
@@ -1,32 +1,33 @@
 {
-  head : {
-    type : "APACHE_DRILL_PHYSICAL",
-    version : 1,
-    generator : {
-      type : "manual"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "manual"
     }
   },
-  graph : [ {
-    pop : "hbase-scan",
-    @id : 1,
-    hbaseScanSpec : {
-      tableName : "[TABLE_NAME]"
+  "graph" : [ {
+    "pop" : "hbase-scan",
+    "@id" : 1,
+    "hbaseScanSpec" : {
+      "tableName" : "[TABLE_NAME]"
     },
-    storage:
+    "storage":
     {
       "type":"hbase",
-      config : {
+      "enabled": true,
+      "config" : {
         "hbase.zookeeper.quorum" : "localhost",
         "hbase.zookeeper.property.clientPort" : 2181
       }
     },
-    columns: [
+    "columns": [
       "f2"
     ]
   },
   {
-    pop : "screen",
-    @id : 2,
-    child : 1
+    "pop" : "screen",
+    "@id" : 2,
+    "child" : 1
   } ]
 }
\ No newline at end of file
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
index f63c503..31610db 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
@@ -72,7 +72,6 @@ public class HiveSchemaFactory extends AbstractSchemaFactory {
     isDrillImpersonationEnabled = plugin.getContext().getConfig().getBoolean(ExecConstants.IMPERSONATION_ENABLED);
 
     try {
-      // TODO: DRILL-6412. Clients for plugin should be instantiated only for the case, when plugin is enabled
       processUserMetastoreClient =
           DrillHiveMetaStoreClientFactory.createCloseableClientWithCaching(hiveConf);
     } catch (MetaException e) {
diff --git a/contrib/storage-hive/core/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-hive/core/src/main/resources/bootstrap-storage-plugins.json
index 018189c..e2e55fc 100644
--- a/contrib/storage-hive/core/src/main/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-hive/core/src/main/resources/bootstrap-storage-plugins.json
@@ -1,8 +1,8 @@
 {
   "storage":{
-    hive : {
-      type:"hive",
-      configProps : {
+    "hive" : {
+      "type":"hive",
+      "configProps" : {
         "hive.metastore.uris" : "",
         "javax.jdo.option.ConnectionURL" : "jdbc:derby:;databaseName=../sample-data/drill_hive_db;create=true",
         "hive.metastore.warehouse.dir" : "/tmp/drill_hive_wh",
@@ -11,7 +11,7 @@
         "hive.metastore.schema.verification": "false",
         "datanucleus.schema.autoCreateAll": "true"
       },
-      enabled: false
+      "enabled": false
     }
   }
 }
diff --git a/contrib/storage-kafka/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-kafka/src/main/resources/bootstrap-storage-plugins.json
index 18a1df5..3c3142f 100644
--- a/contrib/storage-kafka/src/main/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-kafka/src/main/resources/bootstrap-storage-plugins.json
@@ -1,9 +1,9 @@
 {
   "storage":{
-    kafka : {
-      type:"kafka",
-      kafkaConsumerProps: {"bootstrap.servers":"localhost:9092", "group.id" : "drill-consumer"},
-      enabled: false
+    "kafka" : {
+      "type":"kafka",
+      "kafkaConsumerProps": {"bootstrap.servers":"localhost:9092", "group.id" : "drill-consumer"},
+      "enabled": false
     }
   }
 }
diff --git a/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json
index 52884a6..e835fbd 100644
--- a/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json
@@ -1,9 +1,9 @@
 {
   "storage":{
-    kudu : {
-      type:"kudu",
-      masterAddresses: "1.2.3.4",
-      enabled: false
+    "kudu" : {
+      "type":"kudu",
+      "masterAddresses": "1.2.3.4",
+      "enabled": false
     }
   }
 }
diff --git a/contrib/storage-kudu/src/test/resources/bootstrap-storage-plugins.json b/contrib/storage-kudu/src/test/resources/bootstrap-storage-plugins.json
index 3ba12c0..b3ba6b1 100644
--- a/contrib/storage-kudu/src/test/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-kudu/src/test/resources/bootstrap-storage-plugins.json
@@ -1,9 +1,9 @@
 {
   "storage":{
-    kudu : {
-      type:"kudu",
-      masterAddresses: "172.31.1.99",
-      enabled: true
+    "kudu" : {
+      "type":"kudu",
+      "masterAddresses": "172.31.1.99",
+      "enabled": true
     }
   }
 }
diff --git a/contrib/storage-mongo/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-mongo/src/main/resources/bootstrap-storage-plugins.json
index 9983596..2010e32 100644
--- a/contrib/storage-mongo/src/main/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-mongo/src/main/resources/bootstrap-storage-plugins.json
@@ -1,9 +1,9 @@
 {
   "storage":{
-    mongo : {
-      type:"mongo",
-      connection:"mongodb://localhost:27017/",
-      enabled: false
+    "mongo" : {
+      "type":"mongo",
+      "connection":"mongodb://localhost:27017/",
+      "enabled": false
     }
   }
 }
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
index 4b4412f..aff0b56 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
@@ -17,79 +17,42 @@
  */
 package org.apache.drill.exec.store.mongo;
 
-import static org.junit.Assert.assertEquals;
-
-import java.util.List;
-
-import org.apache.drill.PlanTestBase;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
 import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.BeforeClass;
 
-public class MongoTestBase extends PlanTestBase implements MongoTestConstants {
-  protected static MongoStoragePlugin storagePlugin;
-  protected static MongoStoragePluginConfig storagePluginConfig;
+public class MongoTestBase extends ClusterTest implements MongoTestConstants {
+  private static StoragePluginRegistry pluginRegistry;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    // Make sure this test is only running as part of the suit
-    Assume.assumeTrue(MongoTestSuit.isRunningSuite());
+    startCluster(ClusterFixture.builder(dirTestWatcher));
+    pluginRegistry = cluster.drillbit().getContext().getStorage();
+
     MongoTestSuit.initMongo();
-    initMongoStoragePlugin();
+    initMongoStoragePlugin(MongoTestSuit.getConnectionURL());
   }
 
-  public static void initMongoStoragePlugin() throws Exception {
-    final StoragePluginRegistry pluginRegistry = getDrillbitContext().getStorage();
-    storagePlugin = (MongoStoragePlugin) pluginRegistry.getPlugin(MongoStoragePluginConfig.NAME);
-    storagePluginConfig = storagePlugin.getConfig();
+  private static void initMongoStoragePlugin(String connectionURI) throws Exception {
+    MongoStoragePluginConfig storagePluginConfig = new MongoStoragePluginConfig(connectionURI);
     storagePluginConfig.setEnabled(true);
     pluginRegistry.createOrUpdate(MongoStoragePluginConfig.NAME, storagePluginConfig, true);
-    if (System.getProperty("drill.mongo.tests.bson.reader", "true").equalsIgnoreCase("false")) {
-      testNoResult(String.format("alter session set `%s` = false", ExecConstants.MONGO_BSON_RECORD_READER));
-    } else {
-      testNoResult(String.format("alter session set `%s` = true", ExecConstants.MONGO_BSON_RECORD_READER));
-    }
-  }
-
-  public List<QueryDataBatch> runMongoSQLWithResults(String sql)
-      throws Exception {
-    return testSqlWithResults(sql);
-  }
-
-  public void runMongoSQLVerifyCount(String sql, int expectedRowCount)
-      throws Exception {
-    List<QueryDataBatch> results = runMongoSQLWithResults(sql);
-    logResultAndVerifyRowCount(results, expectedRowCount);
-  }
-
-  public void logResultAndVerifyRowCount(List<QueryDataBatch> results,
-                                         int expectedRowCount) throws SchemaChangeException {
-    int rowCount = logResult(results);
-    if (expectedRowCount != -1) {
-      Assert.assertEquals(expectedRowCount, rowCount);
-    }
-  }
 
-  public void testHelper(String query, String expectedExprInPlan,
-      int expectedRecordCount) throws Exception {
-    testPhysicalPlan(query, expectedExprInPlan);
-    int actualRecordCount = testSql(query);
-    assertEquals(
-        String.format(
-            "Received unexpected number of rows in output: expected=%d, received=%s",
-            expectedRecordCount, actualRecordCount), expectedRecordCount,
-        actualRecordCount);
+    client.testBuilder()
+        .sqlQuery("alter session set `%s` = %s",
+            ExecConstants.MONGO_BSON_RECORD_READER,
+            System.getProperty("drill.mongo.tests.bson.reader", "true"))
+        .unOrdered()
+        .expectsEmptyResultSet();
   }
 
   @AfterClass
   public static void tearDownMongoTestBase() throws Exception {
+    pluginRegistry.deletePlugin(MongoStoragePluginConfig.NAME);
     MongoTestSuit.tearDownCluster();
-    storagePlugin = null;
   }
 
 }
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
index 5aeaf9f..7485a30 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
@@ -17,34 +17,34 @@
  */
 package org.apache.drill.exec.store.mongo;
 
-public interface MongoTestConstants {
 
-  public static final String LOCALHOST = "localhost";
+public interface MongoTestConstants {
+  String LOCALHOST = "localhost";
   // TODO: DRILL-3934: add some randomization to this as it fails when running concurrent builds
   int CONFIG_SERVER_1_PORT = 61114;
   int CONFIG_SERVER_2_PORT = 61215;
   int CONFIG_SERVER_3_PORT = 61316;
-  public static final int MONGOD_1_PORT = 27020;
-  public static final int MONGOD_2_PORT = 27021;
-  public static final int MONGOD_3_PORT = 27022;
+  int MONGOD_1_PORT = 27020;
+  int MONGOD_2_PORT = 27021;
+  int MONGOD_3_PORT = 27022;
 
-  public static final int MONGOD_4_PORT = 27023;
-  public static final int MONGOD_5_PORT = 27024;
-  public static final int MONGOD_6_PORT = 27025;
+  int MONGOD_4_PORT = 27023;
+  int MONGOD_5_PORT = 27024;
+  int MONGOD_6_PORT = 27025;
 
-  public static final int MONGOS_PORT = 27017;
+  int MONGOS_PORT = 27017;
 
-  public static final String EMPLOYEE_DB = "employee";
-  public static final String DONUTS_DB = "donuts";
+  String EMPLOYEE_DB = "employee";
+  String DONUTS_DB = "donuts";
 
-  public static final String DONUTS_COLLECTION = "donuts";
-  public static final String EMPINFO_COLLECTION = "empinfo";
-  public static final String EMPTY_COLLECTION = "empty";
-  public static final String SCHEMA_CHANGE_COLLECTION = "schema_change";
+  String DONUTS_COLLECTION = "donuts";
+  String EMPINFO_COLLECTION = "empinfo";
+  String EMPTY_COLLECTION = "empty";
+  String SCHEMA_CHANGE_COLLECTION = "schema_change";
 
-  public static final String DONUTS_DATA = "donuts.json";
-  public static final String EMP_DATA = "emp.json";
-  public static final String SCHEMA_CHANGE_DATA = "schema_change_int_to_string.json";
+  String DONUTS_DATA = "donuts.json";
+  String EMP_DATA = "emp.json";
+  String SCHEMA_CHANGE_DATA = "schema_change_int_to_string.json";
 
   String STORAGE_ENGINE = "wiredTiger";
   String DATATYPE_DB = "datatype";
@@ -52,31 +52,31 @@ public interface MongoTestConstants {
   String DATATYPE_DATA = "datatype-oid.json";
 
   String CONFIG_REPLICA_SET = "config_replicas";
-  public static final String REPLICA_SET_1_NAME = "shard_1_replicas";
-  public static final String REPLICA_SET_2_NAME = "shard_2_replicas";
+  String REPLICA_SET_1_NAME = "shard_1_replicas";
+  String REPLICA_SET_2_NAME = "shard_2_replicas";
 
   // test queries
-  public static final String TEST_QUERY_1 = "SELECT * FROM mongo.employee.`empinfo` limit 5";
-  public static final String TEST_QUERY_LIMIT = "SELECT first_name, last_name FROM mongo.employee.`empinfo` limit 2;";
+  String TEST_QUERY_1 = "SELECT * FROM mongo.employee.`empinfo` limit 5";
+  String TEST_QUERY_LIMIT = "SELECT first_name, last_name FROM mongo.employee.`empinfo` limit 2;";
 
   // test query template1
-  public static final String TEST_QUERY_PROJECT_PUSH_DOWN_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s`";
-  public static final String TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_2 = "select `employee_id`, `rating` from mongo.%s.`%s`";
-  public static final String TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_3 = "select * from mongo.%s.`%s`";
-  public static final String TEST_FILTER_PUSH_DOWN_IS_NULL_QUERY_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s` where position_id is null";
-  public static final String TEST_FILTER_PUSH_DOWN_IS_NOT_NULL_QUERY_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s` where position_id is not null";
-  public static final String TEST_FILTER_PUSH_DOWN_EQUAL_QUERY_TEMPLATE_1 = "SELECT `full_name` FROM mongo.%s.`%s` where rating = 52.17";
-  public static final String TEST_FILTER_PUSH_DOWN_NOT_EQUAL_QUERY_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s` where rating != 52.17";
-  public static final String TEST_FILTER_PUSH_DOWN_LESS_THAN_QUERY_TEMPLATE_1 = "SELECT `full_name` FROM mongo.%s.`%s` where rating < 52.17";
-  public static final String TEST_FILTER_PUSH_DOWN_GREATER_THAN_QUERY_TEMPLATE_1 = "SELECT `full_name` FROM mongo.%s.`%s` where rating > 52.17";
-  public static final String TEST_EMPTY_TABLE_QUERY_TEMPLATE = "select count(*) from mongo.%s.`%s`";
+  String TEST_QUERY_PROJECT_PUSH_DOWN_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s`";
+  String TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_2 = "select `employee_id`, `rating` from mongo.%s.`%s`";
+  String TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_3 = "select * from mongo.%s.`%s`";
+  String TEST_FILTER_PUSH_DOWN_IS_NULL_QUERY_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s` where position_id is null";
+  String TEST_FILTER_PUSH_DOWN_IS_NOT_NULL_QUERY_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s` where position_id is not null";
+  String TEST_FILTER_PUSH_DOWN_EQUAL_QUERY_TEMPLATE_1 = "SELECT `full_name` FROM mongo.%s.`%s` where rating = 52.17";
+  String TEST_FILTER_PUSH_DOWN_NOT_EQUAL_QUERY_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s` where rating != 52.17";
+  String TEST_FILTER_PUSH_DOWN_LESS_THAN_QUERY_TEMPLATE_1 = "SELECT `full_name` FROM mongo.%s.`%s` where rating < 52.17";
+  String TEST_FILTER_PUSH_DOWN_GREATER_THAN_QUERY_TEMPLATE_1 = "SELECT `full_name` FROM mongo.%s.`%s` where rating > 52.17";
+  String TEST_EMPTY_TABLE_QUERY_TEMPLATE = "select count(*) from mongo.%s.`%s`";
 
-  public static final String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE1 = "select `employee_id` from mongo.%s.`%s` where isFTE = true";
-  public static final String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE2 = "select `employee_id` from mongo.%s.`%s` where isFTE = false";
-  public static final String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE3 = "select `employee_id` from mongo.%s.`%s` where position_id = 16 and isFTE = true";
-  public static final String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE4 = "select `employee_id` from mongo.%s.`%s` where (position_id = 16 and isFTE = true) or last_name = 'Yonce'";
+  String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE1 = "select `employee_id` from mongo.%s.`%s` where isFTE = true";
+  String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE2 = "select `employee_id` from mongo.%s.`%s` where isFTE = false";
+  String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE3 = "select `employee_id` from mongo.%s.`%s` where position_id = 16 and isFTE = true";
+  String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE4 = "select `employee_id` from mongo.%s.`%s` where (position_id = 16 and isFTE = true) or last_name = 'Yonce'";
 
-  public static final String TEST_STAR_QUERY_UNSHARDED_DB = "select * from mongo.%s.`%s`";
-  public static final String TEST_STAR_QUERY_UNSHARDED_DB_PROJECT_FILTER = "select t.name as name,t.topping.type as type from mongo.%s.`%s` t where t.sales >= 150";
-  public static final String TEST_STAR_QUERY_UNSHARDED_DB_GROUP_PROJECT_FILTER = "select t.topping.type as type,count(t.topping.type) as typeCount from mongo.%s.`%s` t group by t.topping.type order by typeCount";
+  String TEST_STAR_QUERY_UNSHARDED_DB = "select * from mongo.%s.`%s`";
+  String TEST_STAR_QUERY_UNSHARDED_DB_PROJECT_FILTER = "select t.name as name,t.topping.type as type from mongo.%s.`%s` t where t.sales >= 150";
+  String TEST_STAR_QUERY_UNSHARDED_DB_GROUP_PROJECT_FILTER = "select t.topping.type as type,count(t.topping.type) as typeCount from mongo.%s.`%s` t group by t.topping.type order by typeCount";
 }
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
index b41f5b2..eed94f1 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.store.mongo;
 
 import java.io.IOException;
-import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -69,28 +68,22 @@ import de.flapdoodle.embed.process.runtime.Network;
 @Category({SlowTest.class, MongoStorageTest.class})
 public class MongoTestSuit implements MongoTestConstants {
 
-  private static final Logger logger = LoggerFactory
-      .getLogger(MongoTestSuit.class);
-
+  private static final Logger logger = LoggerFactory.getLogger(MongoTestSuit.class);
   protected static MongoClient mongoClient;
 
-  private static boolean distMode = System.getProperty(
-      "drill.mongo.tests.shardMode", "true").equalsIgnoreCase("true");
-  private static boolean authEnabled = System.getProperty(
-      "drill.mongo.tests.authEnabled", "false").equalsIgnoreCase("true");
-
+  private static boolean distMode = Boolean.valueOf(System.getProperty("drill.mongo.tests.shardMode", "false"));
+  private static boolean authEnabled = Boolean.valueOf(System.getProperty("drill.mongo.tests.authEnabled", "false"));
+  private static volatile String connectionURL = null;
   private static volatile AtomicInteger initCount = new AtomicInteger(0);
 
-  private static volatile boolean runningSuite = false;
-
-  public static boolean isRunningSuite() {
-    return runningSuite;
+  public static String getConnectionURL() {
+    return connectionURL;
   }
 
   private static class DistributedMode {
     private static MongosSystemForTestFactory mongosTestFactory;
 
-    private static void setup() throws Exception {
+    private static String setup() throws Exception {
       // creating configServers
       List<IMongodConfig> configServers = new ArrayList<>(1);
       configServers.add(crateConfigServerConfig(CONFIG_SERVER_1_PORT));
@@ -102,42 +95,39 @@ public class MongoTestSuit implements MongoTestConstants {
       Map<String, List<IMongodConfig>> replicaSets = new LinkedHashMap<>();
 
       List<IMongodConfig> replicaSet1 = new ArrayList<>();
-      replicaSet1.add(crateIMongodConfig(MONGOD_1_PORT, false,
-          REPLICA_SET_1_NAME));
-      replicaSet1.add(crateIMongodConfig(MONGOD_2_PORT, false,
-          REPLICA_SET_1_NAME));
-      replicaSet1.add(crateIMongodConfig(MONGOD_3_PORT, false,
-          REPLICA_SET_1_NAME));
+      replicaSet1.add(crateIMongodConfig(MONGOD_1_PORT, false, REPLICA_SET_1_NAME));
+      replicaSet1.add(crateIMongodConfig(MONGOD_2_PORT, false, REPLICA_SET_1_NAME));
+      replicaSet1.add(crateIMongodConfig(MONGOD_3_PORT, false, REPLICA_SET_1_NAME));
+
       List<IMongodConfig> replicaSet2 = new ArrayList<>();
-      replicaSet2.add(crateIMongodConfig(MONGOD_4_PORT, false,
-          REPLICA_SET_2_NAME));
-      replicaSet2.add(crateIMongodConfig(MONGOD_5_PORT, false,
-          REPLICA_SET_2_NAME));
-      replicaSet2.add(crateIMongodConfig(MONGOD_6_PORT, false,
-          REPLICA_SET_2_NAME));
+      replicaSet2.add(crateIMongodConfig(MONGOD_4_PORT, false, REPLICA_SET_2_NAME));
+      replicaSet2.add(crateIMongodConfig(MONGOD_5_PORT, false, REPLICA_SET_2_NAME));
+      replicaSet2.add(crateIMongodConfig(MONGOD_6_PORT, false, REPLICA_SET_2_NAME));
 
       replicaSets.put(CONFIG_REPLICA_SET, configServers);
       replicaSets.put(REPLICA_SET_1_NAME, replicaSet1);
       replicaSets.put(REPLICA_SET_2_NAME, replicaSet2);
 
-      // create mongos
+      // create mongo shards
       IMongosConfig mongosConfig = createIMongosConfig();
-      mongosTestFactory = new MongosSystemForTestFactory(mongosConfig,
-          replicaSets, Lists.newArrayList(), EMPLOYEE_DB, EMPINFO_COLLECTION,
-          "employee_id");
+      mongosTestFactory = new MongosSystemForTestFactory(mongosConfig, replicaSets, Lists.newArrayList(),
+          EMPLOYEE_DB, EMPINFO_COLLECTION,"employee_id");
       try {
         mongosTestFactory.start();
         mongoClient = (MongoClient) mongosTestFactory.getMongo();
       } catch (Throwable e) {
-        logger.error(" Error while starting shrded cluster. ", e);
-        throw new Exception(" Error while starting shrded cluster. ", e);
+        logger.error(" Error while starting sharded cluster. ", e);
+        throw new Exception(" Error while starting sharded cluster. ", e);
       }
       createDbAndCollections(DONUTS_DB, DONUTS_COLLECTION, "id");
       createDbAndCollections(EMPLOYEE_DB, EMPTY_COLLECTION, "field_2");
       createDbAndCollections(DATATYPE_DB, DATATYPE_COLLECTION, "_id");
+
+      // the way how it work: client -> router(mongos) -> Shard1 ... ShardN
+      return String.format("mongodb://%s:%s", LOCALHOST, MONGOS_PORT);
     }
 
-    private static IMongodConfig crateConfigServerConfig(int configServerPort) throws UnknownHostException, IOException {
+    private static IMongodConfig crateConfigServerConfig(int configServerPort) throws IOException {
       IMongoCmdOptions cmdOptions = new MongoCmdOptionsBuilder()
         .useNoPrealloc(false)
         .useSmallFiles(false)
@@ -157,9 +147,8 @@ public class MongoTestSuit implements MongoTestConstants {
       return mongodConfig;
     }
 
-    private static IMongodConfig crateIMongodConfig(int mongodPort,
-        boolean flag, String replicaName) throws UnknownHostException,
-        IOException {
+    private static IMongodConfig crateIMongodConfig(int mongodPort, boolean flag, String replicaName)
+        throws IOException {
       IMongoCmdOptions cmdOptions = new MongoCmdOptionsBuilder()
         .useNoPrealloc(false)
         .useSmallFiles(false)
@@ -179,8 +168,7 @@ public class MongoTestSuit implements MongoTestConstants {
       return mongodConfig;
     }
 
-    private static IMongosConfig createIMongosConfig()
-        throws UnknownHostException, IOException {
+    private static IMongosConfig createIMongosConfig() throws IOException {
       IMongoCmdOptions cmdOptions = new MongoCmdOptionsBuilder()
         .useNoPrealloc(false)
         .useSmallFiles(false)
@@ -208,7 +196,6 @@ public class MongoTestSuit implements MongoTestConstants {
           }
       }
     }
-
   }
 
   private static class SingleMode {
@@ -216,7 +203,7 @@ public class MongoTestSuit implements MongoTestConstants {
     private static MongodExecutable mongodExecutable;
     private static MongodProcess mongod;
 
-    private static void setup() throws UnknownHostException, IOException {
+    private static String setup() throws IOException {
       IMongoCmdOptions cmdOptions = new MongoCmdOptionsBuilder().verbose(false)
           .enableAuth(authEnabled).build();
 
@@ -235,6 +222,8 @@ public class MongoTestSuit implements MongoTestConstants {
       createDbAndCollections(EMPLOYEE_DB, SCHEMA_CHANGE_COLLECTION, "field_2");
       createDbAndCollections(EMPLOYEE_DB, EMPTY_COLLECTION, "field_2");
       createDbAndCollections(DATATYPE_DB, DATATYPE_COLLECTION, "_id");
+
+      return String.format("mongodb://%s:%s", LOCALHOST, MONGOS_PORT);
     }
 
     private static void cleanup() {
@@ -253,26 +242,25 @@ public class MongoTestSuit implements MongoTestConstants {
       if (initCount.get() == 0) {
         if (distMode) {
           logger.info("Executing tests in distributed mode");
-          DistributedMode.setup();
+          connectionURL = DistributedMode.setup();
         } else {
           logger.info("Executing tests in single mode");
-          SingleMode.setup();
+          connectionURL = SingleMode.setup();
         }
+        // ToDo DRILL-7269: fix the way how data are imported for the sharded mongo cluster
         TestTableGenerator.importData(EMPLOYEE_DB, EMPINFO_COLLECTION, EMP_DATA);
         TestTableGenerator.importData(EMPLOYEE_DB, SCHEMA_CHANGE_COLLECTION, SCHEMA_CHANGE_DATA);
         TestTableGenerator.importData(DONUTS_DB, DONUTS_COLLECTION, DONUTS_DATA);
         TestTableGenerator.importData(DATATYPE_DB, DATATYPE_COLLECTION, DATATYPE_DATA);
       }
       initCount.incrementAndGet();
-      runningSuite = true;
     }
   }
 
   private static void createDbAndCollections(String dbName,
       String collectionName, String indexFieldName) {
     MongoDatabase db = mongoClient.getDatabase(dbName);
-    MongoCollection<Document> mongoCollection = db
-        .getCollection(collectionName);
+    MongoCollection<Document> mongoCollection = db.getCollection(collectionName);
     if (mongoCollection == null) {
       db.createCollection(collectionName);
       mongoCollection = db.getCollection(collectionName);
@@ -298,9 +286,7 @@ public class MongoTestSuit implements MongoTestConstants {
             mongoClient.dropDatabase(DATATYPE_DB);
             mongoClient.dropDatabase(DONUTS_DB);
           }
-        }
-        finally {
-          runningSuite = false;
+        } finally {
           if (mongoClient != null) {
             mongoClient.close();
           }
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoFilterPushDown.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoFilterPushDown.java
index 645c018..b0644c1 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoFilterPushDown.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoFilterPushDown.java
@@ -27,29 +27,37 @@ public class TestMongoFilterPushDown extends MongoTestBase {
 
   @Test
   public void testFilterPushDownIsEqual() throws Exception {
-    String queryString = String.format(
-        TEST_FILTER_PUSH_DOWN_EQUAL_QUERY_TEMPLATE_1, EMPLOYEE_DB,
-        EMPINFO_COLLECTION);
-    String expectedExpr = "\"$eq\" : 52.17";
-    testHelper(queryString, expectedExpr, 1);
+    String queryString = String.format(TEST_FILTER_PUSH_DOWN_EQUAL_QUERY_TEMPLATE_1, EMPLOYEE_DB, EMPINFO_COLLECTION);
+
+    testBuilder()
+        .sqlQuery(queryString)
+        .unOrdered()
+        .expectsNumRecords(1)
+        .go();
   }
 
   @Test
   public void testFilterPushDownLessThanWithSingleField() throws Exception {
-    String queryString = String.format(
-        TEST_FILTER_PUSH_DOWN_LESS_THAN_QUERY_TEMPLATE_1, EMPLOYEE_DB,
+    String queryString = String.format(TEST_FILTER_PUSH_DOWN_LESS_THAN_QUERY_TEMPLATE_1, EMPLOYEE_DB,
         EMPINFO_COLLECTION);
-    String expectedExpr = "\"$lt\" : 52.17";
-    testHelper(queryString, expectedExpr, 9);
+
+    testBuilder()
+        .sqlQuery(queryString)
+        .unOrdered()
+        .expectsNumRecords(9)
+        .go();
   }
 
   @Test
   public void testFilterPushDownGreaterThanWithSingleField() throws Exception {
-    String queryString = String.format(
-        TEST_FILTER_PUSH_DOWN_GREATER_THAN_QUERY_TEMPLATE_1, EMPLOYEE_DB,
+    String queryString = String.format(TEST_FILTER_PUSH_DOWN_GREATER_THAN_QUERY_TEMPLATE_1, EMPLOYEE_DB,
         EMPINFO_COLLECTION);
-    String expectedExpr = "\"$gt\" : 52.17";
-    testHelper(queryString, expectedExpr, 9);
+
+    testBuilder()
+        .sqlQuery(queryString)
+        .unOrdered()
+        .expectsNumRecords(9)
+        .go();
   }
 
 }
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
index 241ec2b..48d13ce 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
@@ -63,32 +63,42 @@ public class TestMongoProjectPushDown extends MongoTestBase {
                       "inner_3", mapOf()))
               .go();
     } finally {
-      test(String.format("alter session set `%s` = false", ExecConstants.MONGO_READER_READ_NUMBERS_AS_DOUBLE));
+      run("alter session set `%s` = false", ExecConstants.MONGO_READER_READ_NUMBERS_AS_DOUBLE);
     }
   }
 
   @Test
   public void testSingleColumnProject() throws Exception {
-    String query = String.format(TEST_QUERY_PROJECT_PUSH_DOWN_TEMPLATE_1,
-        EMPLOYEE_DB, EMPINFO_COLLECTION);
-    String expectedColNames = " \"columns\" : [ \"`employee_id`\" ]";
-    testHelper(query, expectedColNames, 19);
+    String query = String.format(TEST_QUERY_PROJECT_PUSH_DOWN_TEMPLATE_1, EMPLOYEE_DB, EMPINFO_COLLECTION);
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("employee_id")
+        .expectsNumRecords(19)
+        .go();
   }
 
   @Test
   public void testMultipleColumnsProject() throws Exception {
-    String query = String.format(TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_2,
-        EMPLOYEE_DB, EMPINFO_COLLECTION);
-    String expectedColNames = "\"columns\" : [ \"`employee_id`\", \"`rating`\" ]";
-    testHelper(query, expectedColNames, 19);
+    String query = String.format(TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_2, EMPLOYEE_DB, EMPINFO_COLLECTION);
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("employee_id", "rating")
+        .expectsNumRecords(19)
+        .go();
   }
 
   @Test
   public void testStarProject() throws Exception {
-    String query = String.format(TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_3,
-        EMPLOYEE_DB, EMPINFO_COLLECTION);
-    String expectedColNames = "*";
-    testHelper(query, expectedColNames, 19);
+    String query = String.format(TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_3, EMPLOYEE_DB, EMPINFO_COLLECTION);
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .expectsNumRecords(19)
+        .go();
   }
 
 }
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoQueries.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoQueries.java
index 49d2746..cdec4f7 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoQueries.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoQueries.java
@@ -27,66 +27,79 @@ public class TestMongoQueries extends MongoTestBase {
 
   @Test
   public void testBooleanFilter() throws Exception {
-    String queryString = String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE1,
-        EMPLOYEE_DB, EMPINFO_COLLECTION);
-    runMongoSQLVerifyCount(queryString, 11);
-    queryString = String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE2,
-        EMPLOYEE_DB, EMPINFO_COLLECTION);
-    runMongoSQLVerifyCount(queryString, 8);
+    testBuilder()
+        .sqlQuery(String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE1, EMPLOYEE_DB, EMPINFO_COLLECTION))
+        .unOrdered()
+        .expectsNumRecords(11)
+        .go();
+
+    testBuilder()
+        .sqlQuery(String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE2, EMPLOYEE_DB, EMPINFO_COLLECTION))
+        .unOrdered()
+        .expectsNumRecords(8)
+        .go();
   }
 
   @Test
   public void testWithANDOperator() throws Exception {
-    String queryString = String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE3,
-        EMPLOYEE_DB, EMPINFO_COLLECTION);
-    runMongoSQLVerifyCount(queryString, 4);
+    testBuilder()
+        .sqlQuery(String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE3, EMPLOYEE_DB, EMPINFO_COLLECTION))
+        .unOrdered()
+        .expectsNumRecords(4)
+        .go();
   }
 
   @Test
   public void testWithOROperator() throws Exception {
-    String queryString = String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE3,
-        EMPLOYEE_DB, EMPINFO_COLLECTION);
-    runMongoSQLVerifyCount(queryString, 4);
+    testBuilder()
+        .sqlQuery(String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE3, EMPLOYEE_DB, EMPINFO_COLLECTION))
+        .unOrdered()
+        .expectsNumRecords(4)
+        .go();
   }
 
   @Test
   public void testResultCount() throws Exception {
-    String queryString = String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE4,
-        EMPLOYEE_DB, EMPINFO_COLLECTION);
-    runMongoSQLVerifyCount(queryString, 5);
+    testBuilder()
+        .sqlQuery(String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE4, EMPLOYEE_DB, EMPINFO_COLLECTION))
+        .unOrdered()
+        .expectsNumRecords(5)
+        .go();
   }
 
   @Test
   public void testUnShardedDBInShardedCluster() throws Exception {
-    String queryString = String.format(TEST_STAR_QUERY_UNSHARDED_DB,
-        DONUTS_DB, DONUTS_COLLECTION);
-    runMongoSQLVerifyCount(queryString, 5);
+    testBuilder()
+        .sqlQuery(String.format(TEST_STAR_QUERY_UNSHARDED_DB, DONUTS_DB, DONUTS_COLLECTION))
+        .unOrdered()
+        .expectsNumRecords(5)
+        .go();
   }
 
   @Test
   public void testEmptyCollection() throws Exception {
-    String queryString = String.format(TEST_STAR_QUERY_UNSHARDED_DB, EMPLOYEE_DB, EMPTY_COLLECTION);
-    runMongoSQLVerifyCount(queryString, 0);
+    testBuilder()
+        .sqlQuery(String.format(TEST_STAR_QUERY_UNSHARDED_DB, EMPLOYEE_DB, EMPTY_COLLECTION))
+        .unOrdered()
+        .expectsNumRecords(0)
+        .go();
   }
 
   @Test
   public void testUnShardedDBInShardedClusterWithProjectionAndFilter() throws Exception {
-    String queryString = String.format(TEST_STAR_QUERY_UNSHARDED_DB_PROJECT_FILTER,
-        DONUTS_DB, DONUTS_COLLECTION);
-    runMongoSQLVerifyCount(queryString, 2);
+    testBuilder()
+        .sqlQuery(String.format(TEST_STAR_QUERY_UNSHARDED_DB_PROJECT_FILTER, DONUTS_DB, DONUTS_COLLECTION))
+        .unOrdered()
+        .expectsNumRecords(2)
+        .go();
   }
 
   @Test
   public void testUnShardedDBInShardedClusterWithGroupByProjectionAndFilter() throws Exception {
-    String queryString = String.format(TEST_STAR_QUERY_UNSHARDED_DB_GROUP_PROJECT_FILTER,
-        DONUTS_DB, DONUTS_COLLECTION);
-    runMongoSQLVerifyCount(queryString, 5);
-  }
-
-  @Test
-  public void testPhysicalPlanSubmission() throws Exception {
-    String query = String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE1,
-        EMPLOYEE_DB, EMPINFO_COLLECTION);
-    testPhysicalPlanExecutionBasedOnQuery(query);
+    testBuilder()
+        .sqlQuery(String.format(TEST_STAR_QUERY_UNSHARDED_DB_GROUP_PROJECT_FILTER, DONUTS_DB, DONUTS_COLLECTION))
+        .unOrdered()
+        .expectsNumRecords(5)
+        .go();
   }
 }
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestTableGenerator.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestTableGenerator.java
index 4f2c6c8..3ef4617 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestTableGenerator.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestTableGenerator.java
@@ -17,10 +17,11 @@
  */
 package org.apache.drill.exec.store.mongo;
 
+import java.io.File;
 import java.io.IOException;
+import java.net.URISyntaxException;
 
 import de.flapdoodle.embed.mongo.MongoImportProcess;
-import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -40,10 +41,9 @@ public class TestTableGenerator implements MongoTestConstants {
       .getLogger(TestTableGenerator.class);
 
   public static void importData(String dbName, String collectionName,
-      String fileName) throws InterruptedException,IOException {
-    String jsonFile = Resources.getResource(fileName).toString();
-    jsonFile = jsonFile.replaceFirst("file:", StringUtils.EMPTY);
-    generateTable(dbName, collectionName, jsonFile, true, true, false);
+                                String fileName) throws InterruptedException, IOException, URISyntaxException {
+    File jsonFile = new File(Resources.getResource(fileName).toURI());
+    generateTable(dbName, collectionName, jsonFile.getAbsolutePath(), true, true, false);
   }
 
   public static void generateTable(String dbName, String collection,
diff --git a/contrib/storage-opentsdb/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-opentsdb/src/main/resources/bootstrap-storage-plugins.json
index d1055c1..3c683d0 100644
--- a/contrib/storage-opentsdb/src/main/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-opentsdb/src/main/resources/bootstrap-storage-plugins.json
@@ -1,9 +1,9 @@
 {
   "storage": {
-    openTSDB: {
-      type: "openTSDB",
-      connection: "http://localhost:10000",
-      enabled: false
+    "openTSDB": {
+      "type": "openTSDB",
+      "connection": "http://localhost:10000",
+      "enabled": false
     }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicRootSchema.java b/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicRootSchema.java
index 7de5d6a..744f8b5 100644
--- a/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicRootSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicRootSchema.java
@@ -44,12 +44,8 @@ import java.util.List;
 public class DynamicRootSchema extends DynamicSchema {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DynamicRootSchema.class);
 
-  protected SchemaConfig schemaConfig;
-  protected StoragePluginRegistry storages;
-
-  public StoragePluginRegistry getSchemaFactories() {
-    return storages;
-  }
+  private SchemaConfig schemaConfig;
+  private StoragePluginRegistry storages;
 
   /** Creates a root schema. */
   DynamicRootSchema(StoragePluginRegistry storages, SchemaConfig schemaConfig) {
@@ -78,11 +74,11 @@ public class DynamicRootSchema extends DynamicSchema {
    * @param schemaName the name of the schema
    * @param caseSensitive whether matching for the schema name is case sensitive
    */
-  public void loadSchemaFactory(String schemaName, boolean caseSensitive) {
+  private void loadSchemaFactory(String schemaName, boolean caseSensitive) {
     try {
       SchemaPlus schemaPlus = this.plus();
-      StoragePlugin plugin = getSchemaFactories().getPlugin(schemaName);
-      if (plugin != null && plugin.getConfig().isEnabled()) {
+      StoragePlugin plugin = storages.getPlugin(schemaName);
+      if (plugin != null) {
         plugin.registerSchemas(schemaConfig, schemaPlus);
         return;
       }
@@ -90,7 +86,7 @@ public class DynamicRootSchema extends DynamicSchema {
       // Could not find the plugin of schemaName. The schemaName could be `dfs.tmp`, a 2nd level schema under 'dfs'
       List<String> paths = SchemaUtilites.getSchemaPathAsList(schemaName);
       if (paths.size() == 2) {
-        plugin = getSchemaFactories().getPlugin(paths.get(0));
+        plugin = storages.getPlugin(paths.get(0));
         if (plugin == null) {
           return;
         }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java
index 7bce8f9..c3de7f5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java
@@ -49,8 +49,8 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.server.rest.DrillRestServer.UserAuthEnabled;
-import org.apache.drill.exec.store.StoragePlugin;
 import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.sys.PersistentStore;
 import org.glassfish.jersey.server.mvc.Viewable;
 
 import com.fasterxml.jackson.core.JsonParseException;
@@ -121,10 +121,9 @@ public class StorageResources {
   @Produces(MediaType.APPLICATION_JSON)
   public PluginConfigWrapper getPluginConfig(@PathParam("name") String name) {
     try {
-      // TODO: DRILL-6412: No need to get StoragePlugin. It is enough to have plugin name and config here
-      StoragePlugin plugin = storage.getPlugin(name);
-      if (plugin != null) {
-        return new PluginConfigWrapper(name, plugin.getConfig());
+      PersistentStore<StoragePluginConfig> configStorage = storage.getStore();
+      if (configStorage.contains(name)) {
+        return new PluginConfigWrapper(name, configStorage.get(name));
       }
     } catch (Exception e) {
       logger.error("Failure while trying to access storage config: {}", name, e);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
index 4718a20..23029ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
@@ -133,23 +133,17 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
       boolean done = false;
       try {
         if (oldPlugin != null) {
-          if (config.isEnabled()) {
-            done = enabledPlugins.replace(name, oldPlugin, newPlugin);
-          } else {
-            done = enabledPlugins.remove(name, oldPlugin);
-          }
-          if (done) {
-            closePlugin(oldPlugin);
-          }
-        } else if (config.isEnabled()) {
+          done = newPlugin == null
+              ? enabledPlugins.remove(name, oldPlugin)
+              : enabledPlugins.replace(name, oldPlugin, newPlugin);
+        } else if (newPlugin != null) {
           done = (null == enabledPlugins.putIfAbsent(name, newPlugin));
         } else {
           done = true;
         }
       } finally {
-        if (!done) {
-          closePlugin(newPlugin);
-        }
+        StoragePlugin pluginToClose = done ? oldPlugin : newPlugin;
+        closePlugin(pluginToClose);
       }
 
       if (done) {
@@ -511,17 +505,27 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
     enabledPlugins.putAll(activePlugins);
   }
 
+  /**
+   * Creates plugin instance with the given {@code name} and configuration {@code pluginConfig}.
+   * The plugin need to be present in a list of available plugins and be enabled in the configuration
+   *
+   * @param name name of the plugin
+   * @param pluginConfig plugin configuration
+   * @return plugin client or {@code null} if plugin is disabled
+   */
   private StoragePlugin create(String name, StoragePluginConfig pluginConfig) throws ExecutionSetupException {
-    // TODO: DRILL-6412: clients for storage plugins shouldn't be created, if storage plugin is disabled
-    // Creating of the StoragePlugin leads to instantiating storage clients
+    if (!pluginConfig.isEnabled()) {
+      return null;
+    }
+
     StoragePlugin plugin;
-    Constructor<? extends StoragePlugin> c = availablePlugins.get(pluginConfig.getClass());
-    if (c == null) {
+    Constructor<? extends StoragePlugin> constructor = availablePlugins.get(pluginConfig.getClass());
+    if (constructor == null) {
       throw new ExecutionSetupException(String.format("Failure finding StoragePlugin constructor for config %s",
           pluginConfig));
     }
     try {
-      plugin = c.newInstance(pluginConfig, context, name);
+      plugin = constructor.newInstance(pluginConfig, context, name);
       plugin.start();
       return plugin;
     } catch (ReflectiveOperationException | IOException e) {
diff --git a/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json b/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
index afcf53d..75e363f 100644
--- a/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
+++ b/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
@@ -1,178 +1,166 @@
 {
-  "storage":{
-    "dfs": {
-      "type": "file",
-      "connection": "file:///",
-      "workspaces": {
+  "storage" : {
+    "dfs" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "workspaces" : {
         "root" : {
-          "location": "/",
-          "writable": false,
-          "allowAccessOutsideWorkspace": false
+          "location" : "/",
+          "writable" : false,
+          "allowAccessOutsideWorkspace" : false
         },
         "tmp" : {
-          "location": "/tmp",
-          "writable": true,
-          "allowAccessOutsideWorkspace": false
+          "location" : "/tmp",
+          "writable" : true,
+          "allowAccessOutsideWorkspace" : false
         }
       },
-      "formats": {
+      "formats" : {
         "psv" : {
-          "type": "text",
-          "extensions": [ "tbl" ],
-          "delimiter": "|"
+          "type" : "text",
+          "extensions" : [ "tbl" ],
+          "delimiter" : "|"
         },
         "csv" : {
-          "type": "text",
-          "extensions": [ "csv" ],
-          "delimiter": ","
+          "type" : "text",
+          "extensions" : [ "csv" ],
+          "delimiter" : ","
         },
         "tsv" : {
-          "type": "text",
-          "extensions": [ "tsv" ],
-          "delimiter": "\t"
+          "type" : "text",
+          "extensions" : [ "tsv" ],
+          "delimiter" : "\t"
         },
         "httpd" : {
-          "type": "httpd",
-          "logFormat": "%h %l %u %t \"%r\" %>s %b \"%{Referer}i\" \"%{User-agent}i\"",
-          "timestampFormat": "dd/MMM/yyyy:HH:mm:ss ZZ"
+          "type" : "httpd",
+          "logFormat" : "%h %l %u %t \"%r\" %>s %b \"%{Referer}i\" \"%{User-agent}i\"",
+          "timestampFormat" : "dd/MMM/yyyy:HH:mm:ss ZZ"
         },
         "parquet" : {
-          "type": "parquet"
+          "type" : "parquet"
         },
         "json" : {
-          "type": "json",
-          "extensions": [ "json" ]
+          "type" : "json",
+          "extensions" : [ "json" ]
         },
         "pcap" : {
-          "type": "pcap"
+          "type" : "pcap"
         },
         "pcapng" : {
-          "type": "pcapng"
+          "type" : "pcapng"
         },
         "avro" : {
-          "type": "avro"
+          "type" : "avro"
         },
-        "sequencefile": {
-          "type": "sequencefile",
-          "extensions": [ "seq" ]
+        "sequencefile" : {
+          "type" : "sequencefile",
+          "extensions" : [ "seq" ]
         },
         "csvh" : {
-          "type": "text",
-          "extensions": [ "csvh" ],
-          "delimiter": ",",
-          "extractHeader": true
+          "type" : "text",
+          "extensions" : [ "csvh" ],
+          "delimiter" : ",",
+          "extractHeader" : true
         },
         "image" : {
-          "type": "image",
-          "extensions": [
-            "jpg", "jpeg", "jpe", "tif", "tiff", "dng", "psd", "png", "bmp", "gif",
-            "ico", "pcx", "wav", "wave", "avi", "webp", "mov", "mp4", "m4a", "m4p",
-            "m4b", "m4r", "m4v", "3gp", "3g2", "eps", "epsf", "epsi", "ai", "arw",
-            "crw", "cr2", "nef", "orf", "raf", "rw2", "rwl", "srw", "x3f"
-          ]
+          "type" : "image",
+          "extensions" : [ "jpg", "jpeg", "jpe", "tif", "tiff", "dng", "psd", "png", "bmp", "gif", "ico", "pcx", "wav", "wave", "avi", "webp", "mov", "mp4", "m4a", "m4p", "m4b", "m4r", "m4v", "3gp", "3g2", "eps", "epsf", "epsi", "ai", "arw", "crw", "cr2", "nef", "orf", "raf", "rw2", "rwl", "srw", "x3f" ]
         }
       },
       "enabled" : true
     },
-
-    "s3": {
-      "type": "file",
-      "connection": "s3a://my.bucket.location.com",
+    "s3" : {
+      "type" : "file",
+      "connection" : "s3a://my.bucket.location.com",
       "config" : {
-        "fs.s3a.access.key": "ID",
-        "fs.s3a.secret.key": "SECRET"
+        "fs.s3a.access.key" : "ID",
+        "fs.s3a.secret.key" : "SECRET"
       },
-      "workspaces": {
+      "workspaces" : {
         "root" : {
-          "location": "/",
-          "writable": false
+          "location" : "/",
+          "writable" : false
         },
         "tmp" : {
-          "location": "/tmp",
-          "writable": true
+          "location" : "/tmp",
+          "writable" : true
         }
       },
-      "formats": {
+      "formats" : {
         "psv" : {
-          "type": "text",
-          "extensions": [ "tbl" ],
-          "delimiter": "|"
+          "type" : "text",
+          "extensions" : [ "tbl" ],
+          "delimiter" : "|"
         },
         "csv" : {
-          "type": "text",
-          "extensions": [ "csv" ],
-          "delimiter": ","
+          "type" : "text",
+          "extensions" : [ "csv" ],
+          "delimiter" : ","
         },
         "tsv" : {
-          "type": "text",
-          "extensions": [ "tsv" ],
-          "delimiter": "\t"
+          "type" : "text",
+          "extensions" : [ "tsv" ],
+          "delimiter" : "\t"
         },
         "parquet" : {
-          "type": "parquet"
+          "type" : "parquet"
         },
         "json" : {
-          "type": "json",
-          "extensions": [ "json" ]
+          "type" : "json",
+          "extensions" : [ "json" ]
         },
         "avro" : {
-          "type": "avro"
+          "type" : "avro"
         },
-        "sequencefile": {
-          "type": "sequencefile",
-          "extensions": [ "seq" ]
+        "sequencefile" : {
+          "type" : "sequencefile",
+          "extensions" : [ "seq" ]
         },
         "csvh" : {
-          "type": "text",
-          "extensions": [ "csvh" ],
-          "delimiter": ",",
-          "extractHeader": true
+          "type" : "text",
+          "extensions" : [ "csvh" ],
+          "delimiter" : ",",
+          "extractHeader" : true
         }
       },
       "enabled" : false
     },
-
-    "cp": {
-      "type": "file",
-      "connection": "classpath:///",
-      "formats": {
+    "cp" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "formats" : {
         "csv" : {
-          "type": "text",
-          "extensions": [ "csv" ],
-          "delimiter": ","
+          "type" : "text",
+          "extensions" : [ "csv" ],
+          "delimiter" : ","
         },
         "tsv" : {
-                  "type": "text",
-                  "extensions": [ "tsv" ],
-                  "delimiter": "\t"
+          "type" : "text",
+          "extensions" : [ "tsv" ],
+          "delimiter" : "\t"
         },
         "json" : {
-          "type": "json",
-          "extensions": [ "json" ]
+          "type" : "json",
+          "extensions" : [ "json" ]
         },
         "parquet" : {
-          "type": "parquet"
+          "type" : "parquet"
         },
         "avro" : {
-          "type": "avro"
+          "type" : "avro"
         },
         "csvh" : {
-          "type": "text",
-          "extensions": [ "csvh" ],
-          "delimiter": ",",
-          "extractHeader": true
+          "type" : "text",
+          "extensions" : [ "csvh" ],
+          "delimiter" : ",",
+          "extractHeader" : true
         },
         "image" : {
-          "type": "image",
-          "extensions": [
-            "jpg", "jpeg", "jpe", "tif", "tiff", "dng", "psd", "png", "bmp", "gif",
-            "ico", "pcx", "wav", "wave", "avi", "webp", "mov", "mp4", "m4a", "m4p",
-            "m4b", "m4r", "m4v", "3gp", "3g2", "eps", "epsf", "epsi", "ai", "arw",
-            "crw", "cr2", "nef", "orf", "raf", "rw2", "rwl", "srw", "x3f"
-          ]
+          "type" : "image",
+          "extensions" : [ "jpg", "jpeg", "jpe", "tif", "tiff", "dng", "psd", "png", "bmp", "gif", "ico", "pcx", "wav", "wave", "avi", "webp", "mov", "mp4", "m4a", "m4p", "m4b", "m4r", "m4v", "3gp", "3g2", "eps", "epsf", "epsi", "ai", "arw", "crw", "cr2", "nef", "orf", "raf", "rw2", "rwl", "srw", "x3f" ]
         }
       },
       "enabled" : true
     }
   }
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/store/TestDisabledPlugin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/store/TestDisabledPlugin.java
index 0f31e57..c342e8f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/store/TestDisabledPlugin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/store/TestDisabledPlugin.java
@@ -31,6 +31,7 @@ import org.junit.experimental.categories.Category;
 
 import static org.apache.drill.exec.util.StoragePluginTestUtils.CP_PLUGIN_NAME;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -77,4 +78,9 @@ public class TestDisabledPlugin extends ClusterTest {
         e.getMessage().contains("VALIDATION ERROR: Schema"));
     }
   }
+
+  @Test
+  public void testDisabledPluginGet() throws Exception {
+    assertNull(pluginRegistry.getPlugin(CP_PLUGIN_NAME));
+  }
 }
diff --git a/exec/java-exec/src/test/resources/agg/hashagg/q6.json b/exec/java-exec/src/test/resources/agg/hashagg/q6.json
index 35b200a..a824020 100644
--- a/exec/java-exec/src/test/resources/agg/hashagg/q6.json
+++ b/exec/java-exec/src/test/resources/agg/hashagg/q6.json
@@ -1,13 +1,13 @@
 {
-  head : {
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     },
-    type : "APACHE_DRILL_PHYSICAL"
+    "type" : "APACHE_DRILL_PHYSICAL"
   },
-  graph : [ {
+  "graph" : [ {
     "pop" : "parquet-scan",
     "@id" : 1,
     "entries" : [ {
@@ -15,50 +15,51 @@
     } ],
     "storage" : {
       "type" : "file",
-      "connection" : "classpath:///"
+      "connection" : "classpath:///",
+      "enabled" : true
     },
     "format" : {
       "type" : "parquet"
     }
   }, {
-    pop : "project",
-    @id : 2,
-    exprs : [ {
-      ref : "$f0",
-      expr : "N_REGIONKEY"
+    "pop" : "project",
+    "@id" : 2,
+    "exprs" : [ {
+      "ref" : "$f0",
+      "expr" : "N_REGIONKEY"
     }, {
-      ref : "$f1",
-      expr : "N_NATIONKEY"
+      "ref" : "$f1",
+      "expr" : "N_NATIONKEY"
     } ],
-    child : 1
+    "child" : 1
   }, {
-    pop : "hash-aggregate",
-    @id : 3,
-    child : 2,
-    phase : "PHASE_1of1",
-    keys : [ {
-      ref : "$f0",
-      expr : "$f0"
+    "pop" : "hash-aggregate",
+    "@id" : 3,
+    "child" : 2,
+    "phase" : "PHASE_1of1",
+    "keys" : [ {
+      "ref" : "$f0",
+      "expr" : "$f0"
     } ],
-    exprs : [ {
-      ref : "SUM",
-      expr : "sum($f1) "
+    "exprs" : [ {
+      "ref" : "SUM",
+      "expr" : "sum($f1) "
     }, {
-      ref : "MIN",
-      expr : "min($f1) "
+      "ref" : "MIN",
+      "expr" : "min($f1) "
     }, {
-      ref : "MAX",
-      expr : "max($f1) "
+      "ref" : "MAX",
+      "expr" : "max($f1) "
     }, {
-      ref : "COUNT",
-      expr : "count($f1) "
+      "ref" : "COUNT",
+      "expr" : "count($f1) "
     }, {
-      ref : "AVG",
-      expr : "avg($f1) "
+      "ref" : "AVG",
+      "expr" : "avg($f1) "
     } ]
   }, {
-    pop : "screen",
-    @id : 4,
-    child : 3
+    "pop" : "screen",
+    "@id" : 4,
+    "child" : 3
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/agg/hashagg/q7_1.json b/exec/java-exec/src/test/resources/agg/hashagg/q7_1.json
index 3494c79..c09fff2 100644
--- a/exec/java-exec/src/test/resources/agg/hashagg/q7_1.json
+++ b/exec/java-exec/src/test/resources/agg/hashagg/q7_1.json
@@ -1,13 +1,13 @@
 {
-  head : {
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     },
-    type : "APACHE_DRILL_PHYSICAL"
+    "type" : "APACHE_DRILL_PHYSICAL"
   },
-  graph : [ {
+  "graph" : [ {
     "pop" : "parquet-scan",
     "@id" : 1,
     "entries" : [ {
@@ -15,44 +15,45 @@
     } ],
     "storage" : {
       "type" : "file",
-      "connection" : "classpath:///"
+      "connection" : "classpath:///",
+      "enabled" : true
     },
     "format" : {
       "type" : "parquet"
     }
   }, {
-    pop : "project",
-    @id : 2,
-    exprs : [ {
-      ref : "$f0",
-      expr : "N_REGIONKEY"
+    "pop" : "project",
+    "@id" : 2,
+    "exprs" : [ {
+      "ref" : "$f0",
+      "expr" : "N_REGIONKEY"
     }, {
-      ref : "$f1",
-      expr : "N_NATIONKEY"
+      "ref" : "$f1",
+      "expr" : "N_NATIONKEY"
     }, {
-      ref : "$f2",
-      expr : "N_NAME"
+      "ref" : "$f2",
+      "expr" : "N_NAME"
     } ],
-    child : 1
+    "child" : 1
   }, {
-    pop : "hash-aggregate",
-    @id : 3,
-    child : 2,
-    phase : "PHASE_1of1",
-    keys : [ {
-      ref : "$f0",
-      expr : "$f0"
+    "pop" : "hash-aggregate",
+    "@id" : 3,
+    "child" : 2,
+    "phase" : "PHASE_1of1",
+    "keys" : [ {
+      "ref" : "$f0",
+      "expr" : "$f0"
     }, {
-      ref : "$f2",
-      expr : "$f2"
+      "ref" : "$f2",
+      "expr" : "$f2"
     } ],
-    exprs : [ {
-      ref : "Y",
-      expr : "sum($f1) "
+    "exprs" : [ {
+      "ref" : "Y",
+      "expr" : "sum($f1) "
     } ]
   }, {
-    pop : "screen",
-    @id : 4,
-    child : 3
+    "pop" : "screen",
+    "@id" : 4,
+    "child" : 3
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/agg/hashagg/q7_2.json b/exec/java-exec/src/test/resources/agg/hashagg/q7_2.json
index f77c08b..20921ac 100644
--- a/exec/java-exec/src/test/resources/agg/hashagg/q7_2.json
+++ b/exec/java-exec/src/test/resources/agg/hashagg/q7_2.json
@@ -1,13 +1,13 @@
 {
-  head : {
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     },
-    type : "APACHE_DRILL_PHYSICAL"
+    "type" : "APACHE_DRILL_PHYSICAL"
   },
-  graph : [ {
+  "graph" : [ {
     "pop" : "parquet-scan",
     "@id" : 1,
     "entries" : [ {
@@ -15,44 +15,45 @@
     } ],
     "storage" : {
       "type" : "file",
-      "connection" : "classpath:///"
+      "connection" : "classpath:///",
+      "enabled": true
     },
     "format" : {
       "type" : "parquet"
     }
   }, {
-    pop : "project",
-    @id : 2,
-    exprs : [ {
-      ref : "$f0",
-      expr : "N_REGIONKEY"
+    "pop" : "project",
+    "@id" : 2,
+    "exprs" : [ {
+      "ref" : "$f0",
+      "expr" : "N_REGIONKEY"
     }, {
-      ref : "$f1",
-      expr : "N_NAME"
+      "ref" : "$f1",
+      "expr" : "N_NAME"
     }, {
-      ref : "$f2",
-      expr : "N_NATIONKEY"
+      "ref" : "$f2",
+      "expr" : "N_NATIONKEY"
     } ],
-    child : 1
+    "child" : 1
   }, {
-    pop : "hash-aggregate",
-    @id : 3,
-    child : 2,
-    phase : "PHASE_1of1",
-    keys : [ {
-      ref : "$f0",
-      expr : "$f0"
+    "pop" : "hash-aggregate",
+    "@id" : 3,
+    "child" : 2,
+    "phase" : "PHASE_1of1",
+    "keys" : [ {
+      "ref" : "$f0",
+      "expr" : "$f0"
     }, {
-      ref : "$f1",
-      expr : "$f1"
+      "ref" : "$f1",
+      "expr" : "$f1"
     } ],
-    exprs : [ {
-      ref : "Y",
-      expr : "sum($f2) "
+    "exprs" : [ {
+      "ref" : "Y",
+      "expr" : "sum($f2) "
     } ]
   }, {
-    pop : "screen",
-    @id : 4,
-    child : 3
+    "pop" : "screen",
+    "@id" : 4,
+    "child" : 3
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/agg/hashagg/q7_3.json b/exec/java-exec/src/test/resources/agg/hashagg/q7_3.json
index 37ba0ad..a38e6e7 100644
--- a/exec/java-exec/src/test/resources/agg/hashagg/q7_3.json
+++ b/exec/java-exec/src/test/resources/agg/hashagg/q7_3.json
@@ -1,13 +1,13 @@
 {
-  head : {
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     },
-    type : "APACHE_DRILL_PHYSICAL"
+    "type" : "APACHE_DRILL_PHYSICAL"
   },
-  graph : [ {
+  "graph" : [ {
     "pop" : "parquet-scan",
     "@id" : 1,
     "entries" : [ {
@@ -15,43 +15,44 @@
     } ],
     "storage" : {
       "type" : "file",
-      "connection" : "classpath:///"
+      "connection" : "classpath:///",
+      "enabled" : true
     },
     "format" : {
       "type" : "parquet"
     }
   }, {
-    pop : "project",
-    @id : 2,
-    exprs : [ {
-      ref : "$f0",
-      expr : "N_REGIONKEY"
+    "pop" : "project",
+    "@id" : 2,
+    "exprs" : [ {
+      "ref" : "$f0",
+      "expr" : "N_REGIONKEY"
     }, {
-      ref : "$f1",
-      expr : "N_NATIONKEY"
+      "ref" : "$f1",
+      "expr" : "N_NATIONKEY"
     }, {
-      ref : "$f2",
-      expr : "N_NAME"
+      "ref" : "$f2",
+      "expr" : "N_NAME"
     } ],
-    child : 1
+    "child" : 1
   }, {
-    pop : "hash-aggregate",
-    @id : 3,
-    child : 2,
-    phase : "PHASE_1of1",
-    keys : [ {
-      ref : "$f0",
-      expr : "$f0"
+    "pop" : "hash-aggregate",
+    "@id" : 3,
+    "child" : 2,
+    "phase" : "PHASE_1of1",
+    "keys" : [ {
+      "ref" : "$f0",
+      "expr" : "$f0"
     }, {
-      ref : "$f1",
-      expr : "$f1"
+      "ref" : "$f1",
+      "expr" : "$f1"
     }, {
-      ref : "$f2",
-      expr : "$f2 "
+      "ref" : "$f2",
+      "expr" : "$f2 "
     } ]
   }, {
-    pop : "screen",
-    @id : 4,
-    child : 3
+    "pop" : "screen",
+    "@id" : 4,
+    "child" : 3
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/agg/hashagg/q8.json b/exec/java-exec/src/test/resources/agg/hashagg/q8.json
index f67cd29..180ea9a 100644
--- a/exec/java-exec/src/test/resources/agg/hashagg/q8.json
+++ b/exec/java-exec/src/test/resources/agg/hashagg/q8.json
@@ -1,13 +1,13 @@
 {
-  head : {
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     },
-    type : "APACHE_DRILL_PHYSICAL"
+    "type" : "APACHE_DRILL_PHYSICAL"
   },
-  graph : [ {
+  "graph" : [ {
     "pop" : "parquet-scan",
     "@id" : 1,
     "entries" : [ {
@@ -17,38 +17,39 @@
       "type" : "file",
       "connection" : "classpath:///",
       "workspaces" : null,
-      "formats" : null
+      "formats" : null,
+      "enabled" : true
     },
     "format" : {
       "type" : "parquet"
     }
   }, {
-    pop : "project",
-    @id : 2,
-    exprs : [ {
-      ref : "$f0",
-      expr : "O_CUSTKEY"
+    "pop" : "project",
+    "@id" : 2,
+    "exprs" : [ {
+      "ref" : "$f0",
+      "expr" : "O_CUSTKEY"
     }, {
-      ref : "$f1",
-      expr : "O_ORDERKEY"
+      "ref" : "$f1",
+      "expr" : "O_ORDERKEY"
     } ],
-    child : 1
+    "child" : 1
   }, {
-    pop : "hash-aggregate",
-    @id : 3,
-    child : 2,
-    phase : "PHASE_1of1",
-    keys : [ {
-      ref : "$f0",
-      expr : "$f0"
+    "pop" : "hash-aggregate",
+    "@id" : 3,
+    "child" : 2,
+    "phase" : "PHASE_1of1",
+    "keys" : [ {
+      "ref" : "$f0",
+      "expr" : "$f0"
     } ],
-    exprs : [ {
-      ref : "Y",
-      expr : "sum($f1) "
+    "exprs" : [ {
+      "ref" : "Y",
+      "expr" : "sum($f1) "
     } ]
   }, {
-    pop : "screen",
-    @id : 4,
-    child : 3
+    "pop" : "screen",
+    "@id" : 4,
+    "child" : 3
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/agg/hashagg/q8_1.json b/exec/java-exec/src/test/resources/agg/hashagg/q8_1.json
index 73ce4be..0278e1f 100644
--- a/exec/java-exec/src/test/resources/agg/hashagg/q8_1.json
+++ b/exec/java-exec/src/test/resources/agg/hashagg/q8_1.json
@@ -1,13 +1,13 @@
 {
-  head : {
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     },
-    type : "APACHE_DRILL_PHYSICAL"
+    "type" : "APACHE_DRILL_PHYSICAL"
   },
-  graph : [ {
+  "graph" : [ {
     "pop" : "parquet-scan",
     "@id" : 1,
     "entries" : [ {
@@ -15,60 +15,61 @@
     } ],
     "storage" : {
       "type" : "file",
-      "connection" : "classpath:///"
+      "connection" : "classpath:///",
+      "enabled" : true
     },
     "format" : {
       "type" : "parquet"
     }
   }, {
-    pop : "project",
-    @id : 2,
-    exprs : [ {
-      ref : "$f0",
-      expr : "O_CUSTKEY"
+    "pop" : "project",
+    "@id" : 2,
+    "exprs" : [ {
+      "ref" : "$f0",
+      "expr" : "O_CUSTKEY"
     }, {
-      ref : "$f1",
-      expr : "O_ORDERKEY"
+      "ref" : "$f1",
+      "expr" : "O_ORDERKEY"
     } ],
-    child : 1
+    "child" : 1
   }, {
-    pop : "hash-aggregate",
-    @id : 3,
-    child : 2,
-    phase : "PHASE_1of1",
-    keys : [ {
-      ref : "$f0",
-      expr : "$f0"
+    "pop" : "hash-aggregate",
+    "@id" : 3,
+    "child" : 2,
+    "phase" : "PHASE_1of1",
+    "keys" : [ {
+      "ref" : "$f0",
+      "expr" : "$f0"
     } ],
-    exprs : [ {
-      ref : "Y",
-      expr : "sum($f1) "
+    "exprs" : [ {
+      "ref" : "Y",
+      "expr" : "sum($f1) "
     } ]
   }, {
-   pop : "sort",
-    @id : 4,
-    child : 3,
-    orderings : [ {
-      order : "ASC",
-      expr : "$f0"
+    "pop" : "sort",
+    "@id" : 4,
+    "child" : 3,
+    "orderings" : [ {
+      "order" : "ASC",
+      "expr" : "$f0"
     }, {
-      order : "ASC",
-      expr : "Y"
+      "order" : "ASC",
+      "expr" : "Y"
     } ],
-    reverse : false
+    "reverse" : false
   }, {
-    pop : "selection-vector-remover",
-    @id : 5,
-    child : 4
+    "pop" : "selection-vector-remover",
+    "@id" : 5,
+    "child" : 4
   }, {
-    pop: "limit",
-    @id : 6,
-    child: 5,
-    first: 0,
-    last: 100
+    "pop" : "limit",
+    "@id" : 6,
+    "child" : 5,
+    "first" : 0,
+    "last" : 100
   }, {
-    pop : "screen",
-    @id : 7,
-    child : 6
+    "pop" : "screen",
+    "@id" : 7,
+    "child" : 6
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/common/test_hashtable1.json b/exec/java-exec/src/test/resources/common/test_hashtable1.json
index 7f0ccaa..39c47a5 100644
--- a/exec/java-exec/src/test/resources/common/test_hashtable1.json
+++ b/exec/java-exec/src/test/resources/common/test_hashtable1.json
@@ -1,40 +1,38 @@
 {
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
-    },
-  graph : [ {
-    pop : "parquet-scan",
-    @id : 1,
-    entries : [ {
-      path : "/tmp/parquet/nation.parquet"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
+    }
+  },
+  "graph" : [ {
+    "pop" : "parquet-scan",
+    "@id" : 1,
+    "entries" : [ {
+      "path" : "/tmp/parquet/nation.parquet"
     } ],
-    storageengine : {
-      type : "parquet",
-      dfsName : "file:///"
+    "storageengine" : {
+      "type" : "parquet",
+      "dfsName" : "file:///",
+      "enabled": true
     },
-    ref : "_MAP",
-    fragmentPointer : 0
+    "ref" : "_MAP",
+    "fragmentPointer" : 0
   }, {
-    pop : "project",
-    @id : 2,
-    exprs : [ {
-      ref : "regionkey",
-      expr : "_MAP.N_REGIONKEY"
+    "pop" : "project",
+    "@id" : 2,
+    "exprs" : [ {
+      "ref" : "regionkey",
+      "expr" : "_MAP.N_REGIONKEY"
     }, {
-      ref : "nationkey",
-      expr : "_MAP.N_NATIONKEY"
+      "ref" : "nationkey",
+      "expr" : "_MAP.N_NATIONKEY"
     } ],
-    child : 1
+    "child" : 1
   }, {
-     @id: 3,
-     child: 2,
-     pop: "screen"
-   }
-  ]
-}
-
-
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/cast_decimal_float.json b/exec/java-exec/src/test/resources/decimal/cast_decimal_float.json
index af15504..1df37fe 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_decimal_float.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_decimal_float.json
@@ -1,45 +1,42 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
   },
-  graph:[
+  "graph": [
+    {
+      "@id": 1,
+      "pop": "fs-scan",
+      "format": {"type": "json"},
+      "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+      "files": ["/input_simple_decimal.json"]
+    },
     {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["/input_simple_decimal.json"]
-    }, {
-      "pop" : "project",
-      "@id" : 2,
-      "exprs" : [
-        { "ref" : "DEC9_COL", "expr": "(cast(DEC9 as vardecimal(9, 4)))" },
-        { "ref" : "DEC38_COL", "expr": "(cast(DEC18 as vardecimal(38, 4)))" }
+      "pop": "project",
+      "@id": 2,
+      "exprs": [
+        {"ref": "DEC9_COL", "expr": "(cast(DEC9 as vardecimal(9, 4)))"},
+        {"ref": "DEC38_COL", "expr": "(cast(DEC18 as vardecimal(38, 4)))"}
       ],
-
-      "child" : 1
+      "child": 1
     },
     {
-      "pop" : "project",
-      "@id" : 4,
-      "exprs" : [
-        {"ref": "DEC9_FLOAT", "expr" : "cast(DEC9_COL as float4)"},
-        {"ref": "DEC38_FLOAT", "expr" : "cast(DEC38_COL as float4)"},
-        {"ref": "DEC9_DOUBLE", "expr" : "cast(DEC9_COL as float8)"},
-        {"ref": "DEC38_DOUBLE", "expr" : "cast(DEC38_COL as float8)"}
+      "pop": "project",
+      "@id": 4,
+      "exprs": [
+        {"ref": "DEC9_FLOAT", "expr": "cast(DEC9_COL as float4)"},
+        {"ref": "DEC38_FLOAT", "expr": "cast(DEC38_COL as float4)"},
+        {"ref": "DEC9_DOUBLE", "expr": "cast(DEC9_COL as float8)"},
+        {"ref": "DEC38_DOUBLE", "expr": "cast(DEC38_COL as float8)"}
       ],
-
-      "child" : 2
+      "child": 2
     },
     {
-      "pop" : "screen",
-      "@id" : 5,
-      "child" : 4
-    } ]
+      "pop": "screen",
+      "@id": 5,
+      "child": 4
+    }
+  ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json b/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json
index 4e8b700..b518699 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json
@@ -1,45 +1,42 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
   },
-  graph:[
+  "graph": [
+    {
+      "@id": 1,
+      "pop": "fs-scan",
+      "format": {"type": "json"},
+      "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+      "files": ["/input_simple_decimal.json"]
+    },
     {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["/input_simple_decimal.json"]
-    }, {
-      "pop" : "project",
-      "@id" : 2,
-      "exprs" : [
-        { "ref" : "DEC9_COL", "expr": "(cast(cast(INT_COL as int) as vardecimal(10, 0)))" },
-        { "ref" : "DEC38_COL", "expr": "(cast(BIGINT_COL as vardecimal(38, 0)))" }
+      "pop": "project",
+      "@id": 2,
+      "exprs": [
+        {"ref": "DEC9_COL", "expr": "(cast(cast(INT_COL as int) as vardecimal(10, 0)))"},
+        {"ref": "DEC38_COL", "expr": "(cast(BIGINT_COL as vardecimal(38, 0)))"}
       ],
-
-      "child" : 1
+      "child": 1
     },
     {
-      "pop" : "project",
-      "@id" : 4,
-      "exprs" : [
-        {"ref": "DEC9_INT", "expr" : "cast(DEC9_COL as int)"},
-        {"ref": "DEC38_INT", "expr" : "cast(DEC38_COL as int)"},
-        {"ref": "DEC9_BIGINT", "expr" : "cast(DEC9_COL as bigint)"},
-        {"ref": "DEC38_BIGINT", "expr" : "cast(DEC38_COL as bigint)"}
+      "pop": "project",
+      "@id": 4,
+      "exprs": [
+        {"ref": "DEC9_INT", "expr": "cast(DEC9_COL as int)"},
+        {"ref": "DEC38_INT", "expr": "cast(DEC38_COL as int)"},
+        {"ref": "DEC9_BIGINT", "expr": "cast(DEC9_COL as bigint)"},
+        {"ref": "DEC38_BIGINT", "expr": "cast(DEC38_COL as bigint)"}
       ],
-
-      "child" : 2
+      "child": 2
     },
     {
-      "pop" : "screen",
-      "@id" : 5,
-      "child" : 4
-    } ]
+      "pop": "screen",
+      "@id": 5,
+      "child": 4
+    }
+  ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/cast_decimal_vardecimal.json b/exec/java-exec/src/test/resources/decimal/cast_decimal_vardecimal.json
index 4f79194..44dcb8a 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_decimal_vardecimal.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_decimal_vardecimal.json
@@ -1,56 +1,54 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
+  }, "graph": [
+  {
+    "pop": "fs-scan",
+    "@id": 4,
+    "format": {"type": "json"},
+    "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+    "files": ["/input_complex_decimal.json"]
   },
-  "graph":[
-    {
-      "pop" : "fs-scan",
-      "@id" : 4,
-      "format": {"type": "json"},
-      "storage":{"type": "file", "connection": "classpath:///"},
-      "files":["/input_complex_decimal.json"]
-    }, {
-      "pop" : "project",
-      "@id" : 3,
-      "exprs" : [
-        { "ref" : "DEC28", "expr": "(cast(B as vardecimal(38, 20)))" },
-        { "ref" : "DEC38", "expr": "(cast(A as vardecimal(28, 16)))" },
-        { "ref" : "DEC18", "expr": "(cast(B as vardecimal(19, 0)))" },
-        { "ref" : "DEC9", "expr": "(cast(A as vardecimal(10, 0)))" }
-      ],
-
-      "child" : 4
-    }, {
-      "pop" : "project",
-      "@id" : 2,
-      "exprs" : [
-        {"ref": "DEC28_COL", "expr" : "cast(DEC28 as decimal38sparse(38, 20))"},
-        {"ref": "DEC38_COL", "expr" : "cast(DEC38 as decimal28sparse(28, 16))"},
-        {"ref": "DEC18_COL", "expr" : "cast(DEC18 as decimal18(19, 0))"},
-        {"ref": "DEC9_COL", "expr" : "cast(DEC9 as decimal9(10, 0))"}
-      ],
-
-      "child" : 3
-    }, {
-      "pop" : "project",
-      "@id" : 1,
-      "exprs" : [
-        {"ref": "DEC28_COL", "expr" : "cast(DEC28_COL as decimal38sparse(38, 16))"},
-        {"ref": "DEC38_COL", "expr" : "cast(DEC38_COL as decimal28sparse(28, 16))"},
-        {"ref": "DEC18_COL", "expr" : "cast(DEC18_COL as decimal18(18, 0))"},
-        {"ref": "DEC9_COL", "expr" : "cast(DEC9_COL as decimal9(9, 0))"}
-      ],
-
-      "child" : 2
-    }, {
-      "pop" : "screen",
-      "@id" : 0,
-      "child" : 1
-    } ]
-}
+  {
+    "pop": "project",
+    "@id": 3,
+    "exprs": [
+      {"ref": "DEC28", "expr": "(cast(B as vardecimal(38, 20)))"},
+      {"ref": "DEC38", "expr": "(cast(A as vardecimal(28, 16)))"},
+      {"ref": "DEC18", "expr": "(cast(B as vardecimal(19, 0)))"},
+      {"ref": "DEC9", "expr": "(cast(A as vardecimal(10, 0)))"}
+    ],
+    "child": 4
+  },
+  {
+    "pop": "project",
+    "@id": 2,
+    "exprs": [
+      {"ref": "DEC28_COL", "expr": "cast(DEC28 as decimal38sparse(38, 20))"},
+      {"ref": "DEC38_COL", "expr": "cast(DEC38 as decimal28sparse(28, 16))"},
+      {"ref": "DEC18_COL", "expr": "cast(DEC18 as decimal18(19, 0))"},
+      {"ref": "DEC9_COL", "expr": "cast(DEC9 as decimal9(10, 0))"}
+    ],
+    "child": 3
+  },
+  {
+    "pop": "project",
+    "@id": 1,
+    "exprs": [
+      {"ref": "DEC28_COL", "expr": "cast(DEC28_COL as decimal38sparse(38, 16))"},
+      {"ref": "DEC38_COL", "expr": "cast(DEC38_COL as decimal28sparse(28, 16))"},
+      {"ref": "DEC18_COL", "expr": "cast(DEC18_COL as decimal18(18, 0))"},
+      {"ref": "DEC9_COL", "expr": "cast(DEC9_COL as decimal9(9, 0))"}
+    ],
+    "child": 2
+  },
+  {
+    "pop": "screen",
+    "@id": 0,
+    "child": 1
+  }
+]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/cast_float_decimal.json b/exec/java-exec/src/test/resources/decimal/cast_float_decimal.json
index e8f318a..04906d7 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_float_decimal.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_float_decimal.json
@@ -1,47 +1,37 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  }, {
-    "pop" : "project",
-    "@id" : 2,
-    "exprs" : [ {
-      "ref" : "F4",
-      "expr" : " (cast(DEC9 as float4)) "
+  "graph": [
+    {
+      "@id": 1,
+      "pop": "fs-scan",
+      "format": {"type": "json"},
+      "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+      "files": ["#{TEST_FILE}"]
     },
-    { "ref" : "F8", "expr": "(cast(DEC18 as float8))" }
-    ],
-
-    "child" : 1
-  },
-{
-    "pop" : "project",
-    "@id" : 4,
-    "exprs" : [ {
-      "ref" : "DECIMAL_9",
-      "expr" : " cast(F4 as vardecimal(9, 4))  "
+    {
+      "pop": "project",
+      "@id": 2,
+      "exprs": [{"ref": "F4", "expr": " (cast(DEC9 as float4)) "}, {"ref": "F8", "expr": "(cast(DEC18 as float8))"}],
+      "child": 1
     },
-    {"ref": "DECIMAL38", "expr" : "cast(F8 as vardecimal(38, 4))"}
-    ],
-
-    "child" : 2
-  },
-{
-    "pop" : "screen",
-    "@id" : 5,
-    "child" : 4
-  } ]
+    {
+      "pop": "project",
+      "@id": 4,
+      "exprs": [
+        {"ref": "DECIMAL_9", "expr": " cast(F4 as vardecimal(9, 4))  "},
+        {"ref": "DECIMAL38", "expr": "cast(F8 as vardecimal(38, 4))"}
+      ],
+      "child": 2
+    },
+    {
+      "pop": "screen",
+      "@id": 5,
+      "child": 4
+    }
+  ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json b/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json
index 0ea7076..b4ceea8 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json
@@ -1,45 +1,42 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
   },
-  graph:[
+  "graph": [
+    {
+      "@id": 1,
+      "pop": "fs-scan",
+      "format": {"type": "json"},
+      "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+      "files": ["/input_simple_decimal.json"]
+    },
     {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["/input_simple_decimal.json"]
-    }, {
-      "pop" : "project",
-      "@id" : 2,
-      "exprs" : [
-        { "ref" : "INT_COL", "expr": "(cast(INT_COL as int))" },
-        { "ref" : "BIGINT_COL", "expr": "(cast(BIGINT_COL as bigint))" }
+      "pop": "project",
+      "@id": 2,
+      "exprs": [
+        {"ref": "INT_COL", "expr": "(cast(INT_COL as int))"},
+        {"ref": "BIGINT_COL", "expr": "(cast(BIGINT_COL as bigint))"}
       ],
-
-      "child" : 1
+      "child": 1
     },
     {
-      "pop" : "project",
-      "@id" : 4,
-      "exprs" : [
-        {"ref": "DEC9_INT", "expr" : "cast(INT_COL as vardecimal(10, 0))"},
-        {"ref": "DEC38_INT", "expr" : "cast(INT_COL as vardecimal(38, 0))"},
-        {"ref": "DEC9_BIGINT", "expr" : "cast(BIGINT_COL as vardecimal(19, 0))"},
-        {"ref": "DEC38_BIGINT", "expr" : "cast(BIGINT_COL as vardecimal(38, 0))"}
+      "pop": "project",
+      "@id": 4,
+      "exprs": [
+        {"ref": "DEC9_INT", "expr": "cast(INT_COL as vardecimal(10, 0))"},
+        {"ref": "DEC38_INT", "expr": "cast(INT_COL as vardecimal(38, 0))"},
+        {"ref": "DEC9_BIGINT", "expr": "cast(BIGINT_COL as vardecimal(19, 0))"},
+        {"ref": "DEC38_BIGINT", "expr": "cast(BIGINT_COL as vardecimal(38, 0))"}
       ],
-
-      "child" : 2
+      "child": 2
     },
     {
-      "pop" : "screen",
-      "@id" : 5,
-      "child" : 4
-    } ]
+      "pop": "screen",
+      "@id": 5,
+      "child": 4
+    }
+  ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/cast_simple_decimal.json b/exec/java-exec/src/test/resources/decimal/cast_simple_decimal.json
index 53ff360..4551ffb 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_simple_decimal.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_simple_decimal.json
@@ -1,47 +1,40 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  }, {
-    "pop" : "project",
-    "@id" : 2,
-    "exprs" : [ {
-      "ref" : "DECIMAL_9",
-      "expr" : " (cast(DEC9 as vardecimal(9, 4))) "
+  "graph": [
+    {
+      "@id": 1,
+      "pop": "fs-scan",
+      "format": {"type": "json"},
+      "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+      "files": ["#{TEST_FILE}"]
     },
-    { "ref" : "DECIMAL_18", "expr": "(cast(DEC18 as vardecimal(18, 9)))" }
-    ],
-
-    "child" : 1
-  },
-{
-    "pop" : "project",
-    "@id" : 4,
-    "exprs" : [ {
-      "ref" : "DECIMAL_9",
-      "expr" : " cast(DECIMAL_9 as varchar(100))  "
+    {
+      "pop": "project",
+      "@id": 2,
+      "exprs": [
+        {"ref": "DECIMAL_9", "expr": " (cast(DEC9 as vardecimal(9, 4))) "},
+        {"ref": "DECIMAL_18", "expr": "(cast(DEC18 as vardecimal(18, 9)))"}
+      ],
+      "child": 1
     },
-    {"ref": "DECIMAL_18", "expr" : "cast(DECIMAL_18 as varchar(100))"}
-    ],
-
-    "child" : 2
-  },
-{
-    "pop" : "screen",
-    "@id" : 5,
-    "child" : 4
-  } ]
+    {
+      "pop": "project",
+      "@id": 4,
+      "exprs": [
+        {"ref": "DECIMAL_9", "expr": " cast(DECIMAL_9 as varchar(100))  "},
+        {"ref": "DECIMAL_18", "expr": "cast(DECIMAL_18 as varchar(100))"}
+      ],
+      "child": 2
+    },
+    {
+      "pop": "screen",
+      "@id": 5,
+      "child": 4
+    }
+  ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/cast_vardecimal_decimal.json b/exec/java-exec/src/test/resources/decimal/cast_vardecimal_decimal.json
index 1b5252d..fdd67d0 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_vardecimal_decimal.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_vardecimal_decimal.json
@@ -1,47 +1,44 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
   },
-  graph:[
+  "graph": [
+    {
+      "@id": 1,
+      "pop": "fs-scan",
+      "format": {"type": "json"},
+      "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+      "files": ["/input_complex_decimal.json"]
+    },
     {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["/input_complex_decimal.json"]
-    }, {
-      "pop" : "project",
-      "@id" : 2,
-      "exprs" : [
-        { "ref" : "DEC28", "expr": "(cast(B as vardecimal(38, 16)))" },
-        { "ref" : "DEC38", "expr": "(cast(A as vardecimal(28, 16)))" },
-        { "ref" : "DEC18", "expr": "(cast(B as vardecimal(18, 0)))" },
-        { "ref" : "DEC9", "expr": "(cast(A as vardecimal(10, 0)))" }
+      "pop": "project",
+      "@id": 2,
+      "exprs": [
+        {"ref": "DEC28", "expr": "(cast(B as vardecimal(38, 16)))"},
+        {"ref": "DEC38", "expr": "(cast(A as vardecimal(28, 16)))"},
+        {"ref": "DEC18", "expr": "(cast(B as vardecimal(18, 0)))"},
+        {"ref": "DEC9", "expr": "(cast(A as vardecimal(10, 0)))"}
       ],
-
-      "child" : 1
+      "child": 1
     },
     {
-      "pop" : "project",
-      "@id" : 4,
-      "exprs" : [
-        {"ref": "DEC28_COL", "expr" : "cast(DEC28 as decimal38sparse(38, 16))"},
-        {"ref": "DEC38_COL", "expr" : "cast(DEC38 as decimal28sparse(28, 16))"},
-        {"ref": "DEC18_COL", "expr" : "cast(DEC18 as decimal18(18, 0))"},
-        {"ref": "DEC9_COL", "expr" : "cast(DEC9 as decimal9(10, 0))"}
+      "pop": "project",
+      "@id": 4,
+      "exprs": [
+        {"ref": "DEC28_COL", "expr": "cast(DEC28 as decimal38sparse(38, 16))"},
+        {"ref": "DEC38_COL", "expr": "cast(DEC38 as decimal28sparse(28, 16))"},
+        {"ref": "DEC18_COL", "expr": "cast(DEC18 as decimal18(18, 0))"},
+        {"ref": "DEC9_COL", "expr": "cast(DEC9 as decimal9(10, 0))"}
       ],
-
-      "child" : 2
+      "child": 2
     },
     {
-      "pop" : "screen",
-      "@id" : 5,
-      "child" : 4
-    } ]
-}
+      "pop": "screen",
+      "@id": 5,
+      "child": 4
+    }
+  ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/simple_decimal_arithmetic.json b/exec/java-exec/src/test/resources/decimal/simple_decimal_arithmetic.json
index c774230..98fcd55 100644
--- a/exec/java-exec/src/test/resources/decimal/simple_decimal_arithmetic.json
+++ b/exec/java-exec/src/test/resources/decimal/simple_decimal_arithmetic.json
@@ -1,55 +1,51 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },  {
-          "pop" : "project",
-          "@id" : 2,
-          "exprs" : [
-          { "ref" : "DECIMAL18_1", "expr": "(cast(DEC18 as vardecimal(18, 1)))" },
-          { "ref" : "DECIMAL18_2", "expr": "(cast(DEC9 as vardecimal(18, 1)))" }
-          ],
-
-          "child" : 1
-        },
-        {
-          "pop" : "project",
-          "@id" : 3,
-          "exprs" : [
-          { "ref": "DEC18ADD", "expr" : "DECIMAL18_1 + DECIMAL18_2"},
-          { "ref": "DEC18SUB", "expr" : "DECIMAL18_1 - DECIMAL18_2"},
-          { "ref": "DEC18MUL", "expr" : "DECIMAL18_1 * DECIMAL18_2"}
-          ],
-
-          "child" : 2
-        },
-      {
-          "pop" : "project",
-          "@id" : 4,
-          "exprs" : [
-          {"ref": "DECIMAL18ADD", "expr" : "cast(DEC18ADD as varchar(100))" },
-          {"ref": "DECIMAL18SUB", "expr" : "cast(DEC18SUB as varchar(100))" },
-          {"ref": "DECIMAL18MUL", "expr" : "cast(DEC18MUL as varchar(100))" }
-          ],
-
-          "child" : 3
-        },
-      {
-          "pop" : "screen",
-          "@id" : 5,
-          "child" : 4
-        } ]
-      }
+  "graph": [
+    {
+      "@id": 1,
+      "pop": "fs-scan",
+      "format": {"type": "json"},
+      "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+      "files": ["#{TEST_FILE}"]
+    },
+    {
+      "pop": "project",
+      "@id": 2,
+      "exprs": [
+        {"ref": "DECIMAL18_1", "expr": "(cast(DEC18 as vardecimal(18, 1)))"},
+        {"ref": "DECIMAL18_2", "expr": "(cast(DEC9 as vardecimal(18, 1)))"}
+      ],
+      "child": 1
+    },
+    {
+      "pop": "project",
+      "@id": 3,
+      "exprs": [
+        {"ref": "DEC18ADD", "expr": "DECIMAL18_1 + DECIMAL18_2"},
+        {"ref": "DEC18SUB", "expr": "DECIMAL18_1 - DECIMAL18_2"},
+        {"ref": "DEC18MUL", "expr": "DECIMAL18_1 * DECIMAL18_2"}
+      ],
+      "child": 2
+    },
+    {
+      "pop": "project",
+      "@id": 4,
+      "exprs": [
+        {"ref": "DECIMAL18ADD", "expr": "cast(DEC18ADD as varchar(100))"},
+        {"ref": "DECIMAL18SUB", "expr": "cast(DEC18SUB as varchar(100))"},
+        {"ref": "DECIMAL18MUL", "expr": "cast(DEC18MUL as varchar(100))"}
+      ],
+      "child": 3
+    },
+    {
+      "pop": "screen",
+      "@id": 5,
+      "child": 4
+    }
+  ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/simple_decimal_math.json b/exec/java-exec/src/test/resources/decimal/simple_decimal_math.json
index f7eb7f2..36e84e9 100644
--- a/exec/java-exec/src/test/resources/decimal/simple_decimal_math.json
+++ b/exec/java-exec/src/test/resources/decimal/simple_decimal_math.json
@@ -1,41 +1,34 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
-  },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  }, {
-    "pop" : "project",
-    "@id" : 2,
-    "exprs" : [
-    { "ref" : "DECIMAL_18", "expr": "log((cast(DEC18 as vardecimal(18, 9))))" }
-    ],
-
-    "child" : 1
-  },
-{
-    "pop" : "project",
-    "@id" : 4,
-    "exprs" : [
-    {"ref": "DECIMAL_18", "expr" : "cast(DECIMAL_18 as varchar(100))"}
-    ],
-
-    "child" : 2
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
   },
-{
-    "pop" : "screen",
-    "@id" : 5,
-    "child" : 4
-  } ]
+  "graph": [
+    {
+      "@id": 1,
+      "pop": "fs-scan",
+      "format": {"type": "json"},
+      "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+      "files": ["#{TEST_FILE}"]
+    },
+    {
+      "pop": "project",
+      "@id": 2,
+      "exprs": [{"ref": "DECIMAL_18", "expr": "log((cast(DEC18 as vardecimal(18, 9))))"}],
+      "child": 1
+    },
+    {
+      "pop": "project",
+      "@id": 4,
+      "exprs": [{"ref": "DECIMAL_18", "expr": "cast(DECIMAL_18 as varchar(100))"}],
+      "child": 2
+    },
+    {
+      "pop": "screen",
+      "@id": 5,
+      "child": 4
+    }
+  ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/test_decimal_complex.json b/exec/java-exec/src/test/resources/decimal/test_decimal_complex.json
index f341421..85ef0ed 100644
--- a/exec/java-exec/src/test/resources/decimal/test_decimal_complex.json
+++ b/exec/java-exec/src/test/resources/decimal/test_decimal_complex.json
@@ -1,59 +1,46 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },{
-        "pop" : "project",
-        "@id" : 2,
-        "exprs" : [ {
-          "ref" : "DE",
-          "expr" : " (cast(B as vardecimal(28, 9))) "
-        },
-        {"ref" : "DE1", "expr": " cast(A as vardecimal(15, 6))" }
-        ],
-
-        "child" : 1
-      },
+  "graph": [
+    {
+      "@id": 1,
+      "pop": "fs-scan",
+      "format": {"type": "json"},
+      "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+      "files": ["#{TEST_FILE}"]
+    },
     {
-        "pop" : "project",
-        "@id" : 3,
-        "exprs" : [ {
-          "ref" : "DEC38ADD",
-          "expr" : " (DE + DE1)  "
-        },
-        {"ref" : "DEC38SUB" , "expr" : " (DE - DE1) " }
-        ],
-
-        "child" : 2
-      },
+      "pop": "project",
+      "@id": 2,
+      "exprs": [
+        {"ref": "DE", "expr": " (cast(B as vardecimal(28, 9))) "},
+        {"ref": "DE1", "expr": " cast(A as vardecimal(15, 6))"}
+      ],
+      "child": 1
+    },
     {
-        "pop" : "project",
-        "@id" : 4,
-        "exprs" : [ {
-          "ref" : "DEC38ADD",
-          "expr" : " cast(DEC38ADD as varchar(100))  "
-        },
-        {"ref" : "DEC38SUB" , "expr" : " cast(DEC38SUB as varchar(100)) " }
-        ],
-
-        "child" : 3
-      },
+      "pop": "project",
+      "@id": 3,
+      "exprs": [{"ref": "DEC38ADD", "expr": " (DE + DE1)  "}, {"ref": "DEC38SUB", "expr": " (DE - DE1) "}],
+      "child": 2
+    },
+    {
+      "pop": "project",
+      "@id": 4,
+      "exprs": [
+        {"ref": "DEC38ADD", "expr": " cast(DEC38ADD as varchar(100))  "},
+        {"ref": "DEC38SUB", "expr": " cast(DEC38SUB as varchar(100)) "}
+      ],
+      "child": 3
+    },
     {
-        "pop" : "screen",
-        "@id" : 5,
-        "child" : 4
-      } ]
-    }
\ No newline at end of file
+      "pop": "screen",
+      "@id": 5,
+      "child": 4
+    }
+  ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/test_decimal_dense_sparse.json b/exec/java-exec/src/test/resources/decimal/test_decimal_dense_sparse.json
index bc10f50..31db7fb 100644
--- a/exec/java-exec/src/test/resources/decimal/test_decimal_dense_sparse.json
+++ b/exec/java-exec/src/test/resources/decimal/test_decimal_dense_sparse.json
@@ -1,78 +1,57 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
-  },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  }, {
-
-    "pop" : "project",
-    "@id" : 2,
-    "exprs" : [ {
-      "ref" : "SPARSE",
-      "expr" : " (cast(B as decimal38sparse(38, 12))) "
-    }
-    ],
-
-    "child" : 1
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
   },
-{
-    "pop" : "project",
-    "@id" : 3,
-    "exprs" : [ {
-      "ref" : "DENSE",
-      "expr" : " cast(SPARSE as decimal38dense(38, 12))  "
+  "graph": [
+    {
+      "@id": 1,
+      "pop": "fs-scan",
+      "format": {"type": "json"},
+      "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+      "files": ["#{TEST_FILE}"]
+    },
+    {
+      "pop": "project",
+      "@id": 2,
+      "exprs": [{"ref": "SPARSE", "expr": " (cast(B as decimal38sparse(38, 12))) "}],
+      "child": 1
+    },
+    {
+      "pop": "project",
+      "@id": 3,
+      "exprs": [{"ref": "DENSE", "expr": " cast(SPARSE as decimal38dense(38, 12))  "}],
+      "child": 2
+    },
+    {
+      "@id": 4,
+      "child": 3,
+      "pop": "sort",
+      "orderings": [{"expr": "DENSE"}],
+    },
+    {
+      "@id": 5,
+      "child": 4,
+      "pop": "selection-vector-remover",
+    },
+    {
+      "pop": "project",
+      "@id": 6,
+      "exprs": [{"ref": "SPARSE", "expr": "cast(DENSE as decimal38sparse(38, 12))"}],
+      "child": 5
+    },
+    {
+      "pop": "project",
+      "@id": 7,
+      "exprs": [{"ref": "STRINGSPARSE", "expr": "cast(SPARSE as varchar(100))"}],
+      "child": 6
+    },
+    {
+      "pop": "screen",
+      "@id": 8,
+      "child": 7
     }
-    ],
-
-    "child" : 2
-  },
-{
-            @id:4,
-            child: 3,
-            pop:"sort",
-            orderings: [
-              {expr: "DENSE"}
-            ]
-        },
-        {
-            @id:5,
-            child: 4,
-            pop:"selection-vector-remover"
-        },
-{
-    "pop" : "project",
-    "@id" : 6,
-    "exprs" : [
-    { "ref" : "SPARSE", "expr" : "cast(DENSE as decimal38sparse(38, 12))" }
-    ],
-
-    "child" : 5
-  },
-{
-    "pop" : "project",
-    "@id" : 7,
-    "exprs" : [
-    { "ref" : "STRINGSPARSE", "expr" : "cast(SPARSE as varchar(100))" }
-    ],
-
-    "child" : 6
-  },
-{
-    "pop" : "screen",
-    "@id" : 8,
-    "child" : 7
-  } ]
-}
-
+  ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/test_decimal_sort_complex.json b/exec/java-exec/src/test/resources/decimal/test_decimal_sort_complex.json
index 8ff0253..efcdf7a 100644
--- a/exec/java-exec/src/test/resources/decimal/test_decimal_sort_complex.json
+++ b/exec/java-exec/src/test/resources/decimal/test_decimal_sort_complex.json
@@ -1,56 +1,48 @@
 {
-  "head" : {
-    "version" : 1,
-    "generator" : {
-      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
-      "info" : ""
-    },
-    "type" : "APACHE_DRILL_PHYSICAL",
-    "resultMode" : "EXEC"
+  "head": {
+    "version": 1,
+    "generator": {"type": "org.apache.drill.exec.planner.logical.DrillImplementor", "info": ""},
+    "type": "APACHE_DRILL_PHYSICAL",
+    "resultMode": "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },{
-        "pop" : "project",
-        "@id" : 2,
-        "exprs" : [ {
-          "ref" : "DEC",
-          "expr" : " (cast(B as vardecimal(38, 12)))"
-        }
-        ],
-        "child" : 1
-      },
-     {
-                @id:3,
-                child: 2,
-                pop:"external-sort",
-                orderings: [
-                  {expr: "DEC"}
-                ]
-            },
-            {
-                @id:4,
-                child: 3,
-                pop:"selection-vector-remover"
-            },
-     {
-        "pop" : "project",
-        "@id" : 5,
-        "exprs" : [ {
-          "ref" : "DESTR",
-          "expr" : " (cast(DEC as varchar(100)))"
-        }
-        ],
-        "child" : 4
-      },
+  "graph": [
+    {
+      "@id": 1,
+      "pop": "fs-scan",
+      "format": {"type": "json"},
+      "storage": {"type": "file", "connection": "classpath:///", "enabled": true},
+      "files": ["#{TEST_FILE}"]
+    },
+    {
+      "pop": "project",
+      "@id": 2,
+      "exprs": [
+        {"ref": "DEC", "expr": " (cast(B as vardecimal(38, 12)))"}
+      ],
+      "child": 1
+    },
+    {
+      "@id": 3,
+      "child": 2,
+      "pop": "external-sort",
+      "orderings": [{"expr": "DEC"}]},
+    {
+      "@id": 4,
+      "child": 3,
+      "pop": "selection-vector-remover"
+    },
+    {
+      "pop": "project",
+      "@id": 5,
+      "exprs": [
+        {"ref": "DESTR", "expr": " (cast(DEC as varchar(100)))"}
+      ],
+      "child": 4
+    },
     {
-        "pop" : "screen",
-        "@id" : 6,
-        "child" : 5
-      } ]
-    }
\ No newline at end of file
+      "pop": "screen",
+      "@id": 6,
+      "child": 5
+    }
+  ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/decimal/test_decimal_sparse_dense_dense.json b/exec/java-exec/src/test/resources/decimal/test_decimal_sparse_dense_dense.json
index b1a2c43..eae1abb 100644
--- a/exec/java-exec/src/test/resources/decimal/test_decimal_sparse_dense_dense.json
+++ b/exec/java-exec/src/test/resources/decimal/test_decimal_sparse_dense_dense.json
@@ -8,47 +8,43 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled" : true
+    },
+    "files" : [ "#{TEST_FILE}" ]
   }, {
     "pop" : "project",
     "@id" : 2,
     "exprs" : [ {
       "ref" : "DECIMAL28SPARSE_1",
       "expr" : " (cast(DEC9 as decimal28sparse(28, 4))) "
-    }
-    ],
-
+    } ],
     "child" : 1
-  },
-{
+  }, {
     "pop" : "project",
     "@id" : 3,
     "exprs" : [ {
       "ref" : "DECIMAL28DENSE_1",
       "expr" : " cast(DECIMAL28SPARSE_1 as decimal28dense(28, 4))  "
-    }
-    ],
-
+    } ],
     "child" : 2
-  },
-{
+  }, {
     "pop" : "project",
     "@id" : 4,
     "exprs" : [ {
       "ref" : "DECIMAL38DENSE_1",
       "expr" : " cast(DECIMAL28DENSE_1 as decimal38dense(38, 4))  "
-    }
-    ],
-
+    } ],
     "child" : 3
-  },
-{
+  }, {
     "pop" : "screen",
     "@id" : 5,
     "child" : 4
diff --git a/exec/java-exec/src/test/resources/functions/cast/testCastVarCharNull.json b/exec/java-exec/src/test/resources/functions/cast/testCastVarCharNull.json
index dceecd1..d17919c 100644
--- a/exec/java-exec/src/test/resources/functions/cast/testCastVarCharNull.json
+++ b/exec/java-exec/src/test/resources/functions/cast/testCastVarCharNull.json
@@ -1,31 +1,37 @@
 {
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
+    }
+  },
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
     },
-    graph:[
-        {
-            @id:1,
-          pop:"fs-scan",
-          format: {type: "json"},
-          storage:{type: "file", connection: "classpath:///"},
-          files:["#{TEST_FILE}"]
-       }, {
-            @id:2,
-            child: 1,
-            pop:"project",
-            exprs: [
-                { ref: "int2varchar", expr:"cast(integer as varchar(20))" },
-                { ref: "float2varchar", expr:"cast(float as varchar(20))" }
-            ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled": true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "@id" : 2,
+    "child" : 1,
+    "pop" : "project",
+    "exprs" : [ {
+      "ref" : "int2varchar",
+      "expr" : "cast(integer as varchar(20))"
+    }, {
+      "ref" : "float2varchar",
+      "expr" : "cast(float as varchar(20))"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/functions/cast/test_cast_varchar_to_bigint.json b/exec/java-exec/src/test/resources/functions/cast/test_cast_varchar_to_bigint.json
index 018dbad..174036d 100644
--- a/exec/java-exec/src/test/resources/functions/cast/test_cast_varchar_to_bigint.json
+++ b/exec/java-exec/src/test/resources/functions/cast/test_cast_varchar_to_bigint.json
@@ -1,32 +1,34 @@
 {
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
+    }
+  },
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
     },
-        graph:[
-        {
-            @id:1,
-            pop:"fs-scan",
-            format: {type: "json"},
-            storage:{type: "file", connection: "classpath:///"},
-            files:["#{TEST_FILE}"]
-        },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [ {
-              ref: "INTEGER",
-              expr: "cast(integer as bigint)"
-            } ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled": true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "INTEGER",
+      "expr" : "cast(integer as bigint)"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/functions/conv/conversionTestWithLogicalPlan.json b/exec/java-exec/src/test/resources/functions/conv/conversionTestWithLogicalPlan.json
index d137f85..acae9e7 100644
--- a/exec/java-exec/src/test/resources/functions/conv/conversionTestWithLogicalPlan.json
+++ b/exec/java-exec/src/test/resources/functions/conv/conversionTestWithLogicalPlan.json
@@ -1,61 +1,87 @@
 {
-  head: {
-    version: 1,
-    generator: {
-      type: "org.apache.drill.exec.planner.logical.DrillImplementor",
-      info: ""
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
+      "info" : ""
     },
-    type: "APACHE_DRILL_LOGICAL",
-    resultMode: "EXEC"
+    "type" : "APACHE_DRILL_LOGICAL",
+    "resultMode" : "EXEC"
   },
-  storage: {
-    mock : {
-      type : "mock"
+  "storage" : {
+    "mock" : {
+      "type" : "mock",
+      "enabled": true
     }
   },
-  query: [{
-    op: "scan",
-    @id: 1,
-    storageengine: "mock",
-    selection: [
-    {
-      records : 10,
-      types : [
-        {name : "tinyint_val", type : "TINYINT", mode : "REQUIRED"},
-        {name : "smallint_val", type : "SMALLINT", mode : "REQUIRED"},
-        {name : "int_val", type : "INT", mode : "REQUIRED"},
-        {name : "bigint_val", type : "BIGINT", mode : "REQUIRED"},
-        {name : "uint8_val", type : "UINT8", mode : "REQUIRED"}
-       ]
-     }]
-  },
-  {
-    op: "project",
-    @id: 2,
-    input: 1,
-    projections: [
-      {ref: "col_tinyint", expr: "convert_from(convert_to(tinyint_val, 'TINYINT_BE'), 'TINYINT_BE')"},
-      {ref: "col_smallint", expr: "convert_from(convert_to(smallint_val, 'SMALLINT'), 'SMALLINT')"},
-      {ref: "col_int", expr: "convert_from(convert_to(int_val, 'INT'), 'INT')"},
-      {ref: "col_bigint", expr: "convert_from(convert_to(bigint_val, 'BIGINT'), 'BIGINT')"},
-      {ref: "col_uint8", expr: "convert_from(convert_to(uint8_val, 'UINT8'), 'UINT8')"},
-      {ref: "col_smallint_be", expr: "convert_from(convert_to(smallint_val, 'SMALLINT_BE'), 'SMALLINT_BE')"},
-      {ref: "col_int_be", expr: "convert_from(convert_to(int_val, 'INT_BE'), 'INT_BE')"},
-      {ref: "col_bigint_be", expr: "convert_from(convert_to(bigint_val, 'BIGINT_BE'), 'BIGINT_BE')"}
-    ]
-  },
-  {
-    op: "limit",
-    @id: 3,
-    input: 2,
-    first: 0,
-    last: 10
-  },
-  {
-    op: "store",
-    @id: 4,
-    input: 3,
-    target: null,
-    storageEngine: "--SCREEN--"
-  }]
-}
+  "query" : [ {
+    "op" : "scan",
+    "@id" : 1,
+    "storageengine" : "mock",
+    "selection" : [ {
+      "records" : 10,
+      "types" : [ {
+        "name" : "tinyint_val",
+        "type" : "TINYINT",
+        "mode" : "REQUIRED"
+      }, {
+        "name" : "smallint_val",
+        "type" : "SMALLINT",
+        "mode" : "REQUIRED"
+      }, {
+        "name" : "int_val",
+        "type" : "INT",
+        "mode" : "REQUIRED"
+      }, {
+        "name" : "bigint_val",
+        "type" : "BIGINT",
+        "mode" : "REQUIRED"
+      }, {
+        "name" : "uint8_val",
+        "type" : "UINT8",
+        "mode" : "REQUIRED"
+      } ]
+    } ]
+  }, {
+    "op" : "project",
+    "@id" : 2,
+    "input" : 1,
+    "projections" : [ {
+      "ref" : "col_tinyint",
+      "expr" : "convert_from(convert_to(tinyint_val, 'TINYINT_BE'), 'TINYINT_BE')"
+    }, {
+      "ref" : "col_smallint",
+      "expr" : "convert_from(convert_to(smallint_val, 'SMALLINT'), 'SMALLINT')"
+    }, {
+      "ref" : "col_int",
+      "expr" : "convert_from(convert_to(int_val, 'INT'), 'INT')"
+    }, {
+      "ref" : "col_bigint",
+      "expr" : "convert_from(convert_to(bigint_val, 'BIGINT'), 'BIGINT')"
+    }, {
+      "ref" : "col_uint8",
+      "expr" : "convert_from(convert_to(uint8_val, 'UINT8'), 'UINT8')"
+    }, {
+      "ref" : "col_smallint_be",
+      "expr" : "convert_from(convert_to(smallint_val, 'SMALLINT_BE'), 'SMALLINT_BE')"
+    }, {
+      "ref" : "col_int_be",
+      "expr" : "convert_from(convert_to(int_val, 'INT_BE'), 'INT_BE')"
+    }, {
+      "ref" : "col_bigint_be",
+      "expr" : "convert_from(convert_to(bigint_val, 'BIGINT_BE'), 'BIGINT_BE')"
+    } ]
+  }, {
+    "op" : "limit",
+    "@id" : 3,
+    "input" : 2,
+    "first" : 0,
+    "last" : 10
+  }, {
+    "op" : "store",
+    "@id" : 4,
+    "input" : 3,
+    "target" : null,
+    "storageEngine" : "--SCREEN--"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/functions/date/age.json b/exec/java-exec/src/test/resources/functions/date/age.json
index 96dc981..23ebf38 100644
--- a/exec/java-exec/src/test/resources/functions/date/age.json
+++ b/exec/java-exec/src/test/resources/functions/date/age.json
@@ -8,29 +8,38 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [
-              { ref: "TS1", expr: "age(cast('2010-01-01 10:10:10' as timestamp), cast('2001-01-01 11:10:10' as timestamp))"},
-              { ref: "D1", expr: "age(cast('2005-07-11' as date), cast('1991-04-29' as date))"},
-              { ref: "D2", expr: "age(cast('1991-04-29' as date), cast('2005-07-11' as date))"},
-              { ref: "TS2", expr: "age(cast('1997-12-17 23:37:54' as timestamp), cast('2001-03-20 17:17:27' as timestamp))"}
-            ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled": true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "TS1",
+      "expr" : "age(cast('2010-01-01 10:10:10' as timestamp), cast('2001-01-01 11:10:10' as timestamp))"
+    }, {
+      "ref" : "D1",
+      "expr" : "age(cast('2005-07-11' as date), cast('1991-04-29' as date))"
+    }, {
+      "ref" : "D2",
+      "expr" : "age(cast('1991-04-29' as date), cast('2005-07-11' as date))"
+    }, {
+      "ref" : "TS2",
+      "expr" : "age(cast('1997-12-17 23:37:54' as timestamp), cast('2001-03-20 17:17:27' as timestamp))"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/functions/date/date_difference_arithmetic.json b/exec/java-exec/src/test/resources/functions/date/date_difference_arithmetic.json
index e215b7b..4e4b070 100644
--- a/exec/java-exec/src/test/resources/functions/date/date_difference_arithmetic.json
+++ b/exec/java-exec/src/test/resources/functions/date/date_difference_arithmetic.json
@@ -8,28 +8,35 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [
-              { ref: "Date1", expr: "cast('2008-2-23' as date) - cast('2007-2-23' as date)"},
-              { ref: "TimeStamp1", expr: "cast('2008-2-23 12:00:00' as timestamp) - cast('2009-2-23 12:01:00' as timestamp) "},
-              { ref: "Time2", expr: "cast('12:20:30' as time) - cast('1:20:30' as time)"}
-            ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled": true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "Date1",
+      "expr" : "cast('2008-2-23' as date) - cast('2007-2-23' as date)"
+    }, {
+      "ref" : "TimeStamp1",
+      "expr" : "cast('2008-2-23 12:00:00' as timestamp) - cast('2009-2-23 12:01:00' as timestamp) "
+    }, {
+      "ref" : "Time2",
+      "expr" : "cast('12:20:30' as time) - cast('1:20:30' as time)"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/functions/date/date_interval_arithmetic.json b/exec/java-exec/src/test/resources/functions/date/date_interval_arithmetic.json
index c1e10a3..0845cd5 100644
--- a/exec/java-exec/src/test/resources/functions/date/date_interval_arithmetic.json
+++ b/exec/java-exec/src/test/resources/functions/date/date_interval_arithmetic.json
@@ -8,39 +8,67 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [
-              { ref: "Date1", expr: "cast('2008-2-23' as date) + cast('P1Y' as intervalyear)"},
-              { ref: "Date2", expr: "cast('2008-2-23' as date) + cast('P1D' as intervalday)"},
-              { ref: "Time1", expr: "cast('12:20:30' as time) + cast('PT1H0M3S' as intervalday)"},
-              { ref: "TimeStamp1", expr: "cast('2008-2-23 12:00:00' as timestamp) + cast('P1D' as intervalday)"},
-              { ref: "TimeStamp2", expr: "cast('2008-2-23 12:00:00' as timestamp) + cast('P1Y2M' as intervalyear)"},
-              { ref: "TimeStampTZ1", expr: "cast('2008-2-23 12:00:00 America/Los_Angeles' as timestamptz) + cast('P1D' as intervalday)"},
-              { ref: "TimeStampTZ2", expr: "cast('2008-2-23 12:00:00 America/Los_Angeles' as timestamptz) + cast('P1Y2M' as intervalyear)"},
-              { ref: "Date3", expr: "cast('P1Y' as intervalyear) + cast('2008-2-23' as date)"},
-              { ref: "Date4", expr: "cast('P1D' as intervalday) + cast('2008-2-23' as date) "},
-              { ref: "Time2", expr: "cast('PT1H0M3S' as intervalday) + cast('12:20:30' as time)"},
-              { ref: "TimeStamp3", expr: "cast('P1D' as intervalday) + cast('2008-2-23 12:00:00' as timestamp)"},
-              { ref: "TimeStamp4", expr: "cast('P1Y2M' as intervalyear) + cast('2008-2-23 12:00:00' as timestamp)"},
-              { ref: "TimeStampTZ3", expr: "cast('P1D' as intervalday) + cast('2008-2-23 12:00:00 America/Los_Angeles' as timestamptz) "},
-              { ref: "TimeStampTZ4", expr: "cast('P1Y2M' as intervalyear) + cast('2008-2-23 12:00:00 America/Los_Angeles' as timestamptz) "}
-            ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///"
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "Date1",
+      "expr" : "cast('2008-2-23' as date) + cast('P1Y' as intervalyear)"
+    }, {
+      "ref" : "Date2",
+      "expr" : "cast('2008-2-23' as date) + cast('P1D' as intervalday)"
+    }, {
+      "ref" : "Time1",
+      "expr" : "cast('12:20:30' as time) + cast('PT1H0M3S' as intervalday)"
+    }, {
+      "ref" : "TimeStamp1",
+      "expr" : "cast('2008-2-23 12:00:00' as timestamp) + cast('P1D' as intervalday)"
+    }, {
+      "ref" : "TimeStamp2",
+      "expr" : "cast('2008-2-23 12:00:00' as timestamp) + cast('P1Y2M' as intervalyear)"
+    }, {
+      "ref" : "TimeStampTZ1",
+      "expr" : "cast('2008-2-23 12:00:00 America/Los_Angeles' as timestamptz) + cast('P1D' as intervalday)"
+    }, {
+      "ref" : "TimeStampTZ2",
+      "expr" : "cast('2008-2-23 12:00:00 America/Los_Angeles' as timestamptz) + cast('P1Y2M' as intervalyear)"
+    }, {
+      "ref" : "Date3",
+      "expr" : "cast('P1Y' as intervalyear) + cast('2008-2-23' as date)"
+    }, {
+      "ref" : "Date4",
+      "expr" : "cast('P1D' as intervalday) + cast('2008-2-23' as date) "
+    }, {
+      "ref" : "Time2",
+      "expr" : "cast('PT1H0M3S' as intervalday) + cast('12:20:30' as time)"
+    }, {
+      "ref" : "TimeStamp3",
+      "expr" : "cast('P1D' as intervalday) + cast('2008-2-23 12:00:00' as timestamp)"
+    }, {
+      "ref" : "TimeStamp4",
+      "expr" : "cast('P1Y2M' as intervalyear) + cast('2008-2-23 12:00:00' as timestamp)"
+    }, {
+      "ref" : "TimeStampTZ3",
+      "expr" : "cast('P1D' as intervalday) + cast('2008-2-23 12:00:00 America/Los_Angeles' as timestamptz) "
+    }, {
+      "ref" : "TimeStampTZ4",
+      "expr" : "cast('P1Y2M' as intervalyear) + cast('2008-2-23 12:00:00 America/Los_Angeles' as timestamptz) "
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/functions/date/interval_arithmetic.json b/exec/java-exec/src/test/resources/functions/date/interval_arithmetic.json
index 3bf5f6c..24ac9b9 100644
--- a/exec/java-exec/src/test/resources/functions/date/interval_arithmetic.json
+++ b/exec/java-exec/src/test/resources/functions/date/interval_arithmetic.json
@@ -8,37 +8,62 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [
-              { ref: "IntervalYear1", expr: "cast('P1Y2M' as intervalyear) + cast('P1Y' as intervalyear)"},
-              { ref: "IntervalDay1", expr: "cast('P1DT1H2M3S' as intervalday) + cast('P1D' as intervalday)"},
-              { ref: "IntervalYear2", expr: "cast('P1Y2M' as intervalyear) - cast('P1Y' as intervalyear)"},
-              { ref: "IntervalDay2", expr: "cast('P1DT1H2M3S' as intervalday) - cast('P1D' as intervalday)"},
-              { ref: "IntervalYear3", expr: "cast('P1Y2M' as intervalyear) * 2"},
-              { ref: "IntervalDay3", expr: " 2 * cast('PT1H0M3S' as intervalday)"},
-              { ref: "IntervalYear4", expr: "cast('P1Y2M' as intervalyear) / 2"},
-              { ref: "IntervalDay4", expr: " cast('PT1H0M3S' as intervalday) / 2"},
-              { ref: "IntervalYear5", expr: "cast('P1Y2M' as intervalyear) * 2.4"},
-              { ref: "IntervalDay5", expr: " 2.4 * cast('PT1H0M3S' as intervalday)"},
-              { ref: "IntervalYear6", expr: "cast('P1Y2M' as intervalyear) / 2.1"},
-              { ref: "IntervalDay6", expr: " cast('PT1H0M3S' as intervalday) / 2.1"}
-            ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled": true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "IntervalYear1",
+      "expr" : "cast('P1Y2M' as intervalyear) + cast('P1Y' as intervalyear)"
+    }, {
+      "ref" : "IntervalDay1",
+      "expr" : "cast('P1DT1H2M3S' as intervalday) + cast('P1D' as intervalday)"
+    }, {
+      "ref" : "IntervalYear2",
+      "expr" : "cast('P1Y2M' as intervalyear) - cast('P1Y' as intervalyear)"
+    }, {
+      "ref" : "IntervalDay2",
+      "expr" : "cast('P1DT1H2M3S' as intervalday) - cast('P1D' as intervalday)"
+    }, {
+      "ref" : "IntervalYear3",
+      "expr" : "cast('P1Y2M' as intervalyear) * 2"
+    }, {
+      "ref" : "IntervalDay3",
+      "expr" : " 2 * cast('PT1H0M3S' as intervalday)"
+    }, {
+      "ref" : "IntervalYear4",
+      "expr" : "cast('P1Y2M' as intervalyear) / 2"
+    }, {
+      "ref" : "IntervalDay4",
+      "expr" : " cast('PT1H0M3S' as intervalday) / 2"
+    }, {
+      "ref" : "IntervalYear5",
+      "expr" : "cast('P1Y2M' as intervalyear) * 2.4"
+    }, {
+      "ref" : "IntervalDay5",
+      "expr" : " 2.4 * cast('PT1H0M3S' as intervalday)"
+    }, {
+      "ref" : "IntervalYear6",
+      "expr" : "cast('P1Y2M' as intervalyear) / 2.1"
+    }, {
+      "ref" : "IntervalDay6",
+      "expr" : " cast('PT1H0M3S' as intervalday) / 2.1"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/functions/date/to_char.json b/exec/java-exec/src/test/resources/functions/date/to_char.json
index 57bee61..53e5040 100644
--- a/exec/java-exec/src/test/resources/functions/date/to_char.json
+++ b/exec/java-exec/src/test/resources/functions/date/to_char.json
@@ -8,28 +8,35 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [
-              { ref: "Date1", expr: "to_char((cast('2008-2-23' as date)), 'yyyy-MMM-dd')"},
-              { ref: "Time1", expr: "to_char(cast('12:20:30' as time), 'HH mm ss')"},
-              { ref: "TimeStamp2", expr: "to_char(cast('2008-2-23 12:00:00' as timestamp), 'yyyy MMM dd HH:mm:ss')"}
-            ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled": true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "Date1",
+      "expr" : "to_char((cast('2008-2-23' as date)), 'yyyy-MMM-dd')"
+    }, {
+      "ref" : "Time1",
+      "expr" : "to_char(cast('12:20:30' as time), 'HH mm ss')"
+    }, {
+      "ref" : "TimeStamp2",
+      "expr" : "to_char(cast('2008-2-23 12:00:00' as timestamp), 'yyyy MMM dd HH:mm:ss')"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/functions/date/to_date_type.json b/exec/java-exec/src/test/resources/functions/date/to_date_type.json
index ab429d1..4692d7b 100644
--- a/exec/java-exec/src/test/resources/functions/date/to_date_type.json
+++ b/exec/java-exec/src/test/resources/functions/date/to_date_type.json
@@ -8,29 +8,37 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [
-              { ref: "Date1", expr: "to_date('2008-FEB-23', 'yyyy-MMM-dd')"},
-              { ref: "Time1", expr: "to_time('12:20:30', 'HH:mm:ss')"},
-              { ref: "TimeStamp2", expr: "to_timestamp('2008-2-23 12:00:00', 'yyyy-MM-dd HH:mm:ss')"},
-              { ref: "TimeStampTZ1", expr: "to_timestamptz('2008-2-23 12:00:00 America/Los_Angeles', 'yyyy-MM-dd HH:mm:ss ZZZ')"}
-            ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///"
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "Date1",
+      "expr" : "to_date('2008-FEB-23', 'yyyy-MMM-dd')"
+    }, {
+      "ref" : "Time1",
+      "expr" : "to_time('12:20:30', 'HH:mm:ss')"
+    }, {
+      "ref" : "TimeStamp2",
+      "expr" : "to_timestamp('2008-2-23 12:00:00', 'yyyy-MM-dd HH:mm:ss')"
+    }, {
+      "ref" : "TimeStampTZ1",
+      "expr" : "to_timestamptz('2008-2-23 12:00:00 America/Los_Angeles', 'yyyy-MM-dd HH:mm:ss ZZZ')"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/functions/extractFrom.json b/exec/java-exec/src/test/resources/functions/extractFrom.json
index 4e58a3c..6d31e35 100644
--- a/exec/java-exec/src/test/resources/functions/extractFrom.json
+++ b/exec/java-exec/src/test/resources/functions/extractFrom.json
@@ -8,38 +8,49 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-        {
-            @id:1,
-            pop:"fs-scan",
-            format: {type: "json"},
-            storage:{type: "file", connection: "classpath:///"},
-            files:["#{TEST_FILE}"]
-        },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [ {
-              ref: "castExp", expr: "cast(#{COLUMN_NAME} as #{TEST_TYPE})"
-            } ]
-        },
-        {
-            pop:"project",
-            @id:3,
-            child: 2,
-            exprs: [
-              { ref: "extractMinute", expr: "extractMinute(castExp)" },
-              { ref: "extractHour", expr: "extractHour(castExp)" },
-              { ref: "extractDay", expr: "extractDay(castExp)" },
-              { ref: "extractMonth", expr: "extractMonth(castExp)" },
-              { ref: "extractYear", expr: "extractYear(castExp)" }
-            ]
-        },
-        {
-            @id: 4,
-            child: 3,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled": true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "castExp",
+      "expr" : "cast(#{COLUMN_NAME} as #{TEST_TYPE})"
+    } ]
+  }, {
+    "pop" : "project",
+    "@id" : 3,
+    "child" : 2,
+    "exprs" : [ {
+      "ref" : "extractMinute",
+      "expr" : "extractMinute(castExp)"
+    }, {
+      "ref" : "extractHour",
+      "expr" : "extractHour(castExp)"
+    }, {
+      "ref" : "extractDay",
+      "expr" : "extractDay(castExp)"
+    }, {
+      "ref" : "extractMonth",
+      "expr" : "extractMonth(castExp)"
+    }, {
+      "ref" : "extractYear",
+      "expr" : "extractYear(castExp)"
+    } ]
+  }, {
+    "@id" : 4,
+    "child" : 3,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/functions/test_covariance.json b/exec/java-exec/src/test/resources/functions/test_covariance.json
index 3572090..0d74f70 100644
--- a/exec/java-exec/src/test/resources/functions/test_covariance.json
+++ b/exec/java-exec/src/test/resources/functions/test_covariance.json
@@ -8,30 +8,31 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
- },
- {
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled" : true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
     "pop" : "project",
     "@id" : 2,
     "exprs" : [ {
       "ref" : "`A`",
       "expr" : "`A`"
-    },
-    {
+    }, {
       "ref" : "`B`",
       "expr" : "`B`"
-    },
-    {
+    }, {
       "ref" : "`C`",
       "expr" : "`C`"
-    },
-    {
+    }, {
       "ref" : "`D`",
       "expr" : "`D`"
     } ],
@@ -44,36 +45,28 @@
     "exprs" : [ {
       "ref" : "`EXPR$1`",
       "expr" : "covar_samp(`A`, `B`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$2`",
       "expr" : "covar_samp(`A`, `C`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$3`",
       "expr" : "covar_samp(`C`, `D`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$4`",
       "expr" : "covar_pop(`A`, `B`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$5`",
       "expr" : "covar_pop(`A`, `C`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$6`",
       "expr" : "covar_pop(`C`, `D`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$7`",
       "expr" : "corr(`A`, `B`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$7`",
       "expr" : "corr(`A`, `C`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$8`",
       "expr" : "corr(`C`, `D`) "
     } ]
diff --git a/exec/java-exec/src/test/resources/functions/test_logical_aggr.json b/exec/java-exec/src/test/resources/functions/test_logical_aggr.json
index b6680be..f6d83b5 100644
--- a/exec/java-exec/src/test/resources/functions/test_logical_aggr.json
+++ b/exec/java-exec/src/test/resources/functions/test_logical_aggr.json
@@ -8,30 +8,31 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
- },
- {
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled": true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
     "pop" : "project",
     "@id" : 2,
     "exprs" : [ {
       "ref" : "`A`",
       "expr" : "`A`"
-    },
-    {
+    }, {
       "ref" : "`B`",
       "expr" : "`B`"
-    },
-    {
+    }, {
       "ref" : "`C`",
       "expr" : "`C`"
-    },
-    {
+    }, {
       "ref" : "`D`",
       "expr" : "`D`"
     } ],
@@ -47,34 +48,28 @@
     }, {
       "ref" : "`EXPR$1`",
       "expr" : "bit_and(`A`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$2`",
       "expr" : "every(`A`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$3`",
       "expr" : "bit_or(`A`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$4`",
       "expr" : "bit_or(`B`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$5`",
       "expr" : "bit_or(`D`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$5`",
       "expr" : "bool_or(`C`) "
-    },
-    {
+    }, {
       "ref" : "`EXPR$6`",
       "expr" : "bool_and(`C`) "
-    }]
+    } ]
   }, {
     "pop" : "screen",
     "@id" : 4,
     "child" : 3
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/functions/test_stddev_variance.json b/exec/java-exec/src/test/resources/functions/test_stddev_variance.json
index 5ba8d3e..97fdb96 100644
--- a/exec/java-exec/src/test/resources/functions/test_stddev_variance.json
+++ b/exec/java-exec/src/test/resources/functions/test_stddev_variance.json
@@ -8,15 +8,19 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
- },
- {
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled": true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
     "pop" : "project",
     "@id" : 2,
     "exprs" : [ {
@@ -53,4 +57,4 @@
     "@id" : 4,
     "child" : 3
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/hive/test.json b/exec/java-exec/src/test/resources/hive/test.json
index a039d9e..d55444a 100644
--- a/exec/java-exec/src/test/resources/hive/test.json
+++ b/exec/java-exec/src/test/resources/hive/test.json
@@ -1,75 +1,74 @@
 {
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
+    }
+  },
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "hive-scan",
+    "storageengine" : {
+      "type" : "hive"
     },
-    graph:[
-        {
-            @id:1,
-            pop:"hive-scan",
-            storageengine: { type: "hive"},
-            hive-table: {
-                          "tableName" : "nation",
-                          "dbName" : "default",
-                          "owner" : "root",
-                          "createTime" : 1386876893,
-                          "lastAccessTime" : 0,
-                          "retention" : 0,
-                          "sd" : {
-                            "cols" : [ {
-                              "name" : "n_nationkey",
-                              "type" : "bigint",
-                              "comment" : null
-                            }, {
-                              "name" : "n_name",
-                              "type" : "string",
-                              "comment" : null
-                            }, {
-                              "name" : "n_regionkey",
-                              "type" : "bigint",
-                              "comment" : null
-                            }, {
-                              "name" : "n_comment",
-                              "type" : "string",
-                              "comment" : null
-                            } ],
-                            "location" : "maprfs:/user/hive/warehouse/nation",
-                            "inputFormat" : "org.apache.hadoop.mapred.TextInputFormat",
-                            "outputFormat" : "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
-                            "compressed" : false,
-                            "numBuckets" : 0,
-                            "serDeInfo" : {
-                              "name" : null,
-                              "serializationLib" : "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
-                              "parameters" : {
-                                "serialization.format" : "|",
-                                "field.delim" : "|"
-                              }
-                            },
-                            "sortCols" : [ ],
-                            "parameters" : { }
-                          },
-                          "partitionKeys" : [ ],
-                          "parameters" : {
-                            "numPartitions" : "0",
-                            "numFiles" : "1",
-                            "transient_lastDdlTime" : "1386877487",
-                            "totalSize" : "2224",
-                            "numRows" : "0",
-                            "rawDataSize" : "0"
-                          },
-                          "viewOriginalText" : null,
-                          "viewExpandedText" : null,
-                          "tableType" : "MANAGED_TABLE"
-                        }
+    "hive-table" : {
+      "tableName" : "nation",
+      "dbName" : "default",
+      "owner" : "root",
+      "createTime" : 1386876893,
+      "lastAccessTime" : 0,
+      "retention" : 0,
+      "sd" : {
+        "cols" : [ {
+          "name" : "n_nationkey",
+          "type" : "bigint",
+          "comment" : null
+        }, {
+          "name" : "n_name",
+          "type" : "string",
+          "comment" : null
+        }, {
+          "name" : "n_regionkey",
+          "type" : "bigint",
+          "comment" : null
+        }, {
+          "name" : "n_comment",
+          "type" : "string",
+          "comment" : null
+        } ],
+        "location" : "maprfs:/user/hive/warehouse/nation",
+        "inputFormat" : "org.apache.hadoop.mapred.TextInputFormat",
+        "outputFormat" : "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
+        "compressed" : false,
+        "numBuckets" : 0,
+        "serDeInfo" : {
+          "name" : null,
+          "serializationLib" : "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
+          "parameters" : {
+            "serialization.format" : "|",
+            "field.delim" : "|"
+          }
         },
-        {
-            @id: 2,
-            child: 1,
-            pop: "screen"
-        }
-    ]
+        "sortCols" : [ ],
+        "parameters" : { }
+      },
+      "partitionKeys" : [ ],
+      "parameters" : {
+        "numPartitions" : "0",
+        "numFiles" : "1",
+        "transient_lastDdlTime" : "1386877487",
+        "totalSize" : "2224",
+        "numRows" : "0",
+        "rawDataSize" : "0"
+      },
+      "viewOriginalText" : null,
+      "viewExpandedText" : null,
+      "tableType" : "MANAGED_TABLE"
+    }
+  }, {
+    "@id" : 2,
+    "child" : 1,
+    "pop" : "screen"
+  } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/join/hashJoinExpr.json b/exec/java-exec/src/test/resources/join/hashJoinExpr.json
index 13a1c0f..2ef2e9a 100644
--- a/exec/java-exec/src/test/resources/join/hashJoinExpr.json
+++ b/exec/java-exec/src/test/resources/join/hashJoinExpr.json
@@ -127,4 +127,4 @@
       "outputRowCount" : 25.0
     }
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/join/hash_join.json b/exec/java-exec/src/test/resources/join/hash_join.json
index dbeb4ef..596b971 100644
--- a/exec/java-exec/src/test/resources/join/hash_join.json
+++ b/exec/java-exec/src/test/resources/join/hash_join.json
@@ -8,57 +8,66 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-    {
-        @id:1,
-        pop:"fs-scan",
-        format: {type: "json"},
-        storage:{type: "file", connection: "file:///"},
-        files:["#{TEST_FILE_1}"]
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
     },
-    {
-        @id:2,
-        pop:"fs-scan",
-        format: {type: "json"},
-        storage:{type: "file", connection: "file:///"},
-        files:["#{TEST_FILE_2}"]
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "enabled": true
     },
-    {
-        "pop" : "project",
-        "@id" : 3,
-        "exprs" : [ {
-          "ref" : "A",
-          "expr" : "A"
-        },
-        { "ref" : "CCOL", "expr" : "C" }
-        ],
-
-        "child" : 1
+    "files" : [ "#{TEST_FILE_1}" ]
+  }, {
+    "@id" : 2,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
     },
-    {
-        "pop" : "project",
-        "@id" : 4,
-        "exprs" : [ {
-          "ref" : "B",
-          "expr" : "B"
-        },
-        { "ref" : "DCOL", "expr" : "D" }
-        ],
-
-        "child" : 2
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "enabled": true
     },
-    {
-        @id: 5,
-        right: 3,
-        left: 4,
-        pop: "hash-join",
-        conditions: [ {relationship: "==", left: "B", right: "A"} ],
-        joinType : "INNER"
-    },
-    {
-        @id: 6,
-        child: 5,
-        pop: "screen"
-     }
-  ]
-}
+    "files" : [ "#{TEST_FILE_2}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 3,
+    "exprs" : [ {
+      "ref" : "A",
+      "expr" : "A"
+    }, {
+      "ref" : "CCOL",
+      "expr" : "C"
+    } ],
+    "child" : 1
+  }, {
+    "pop" : "project",
+    "@id" : 4,
+    "exprs" : [ {
+      "ref" : "B",
+      "expr" : "B"
+    }, {
+      "ref" : "DCOL",
+      "expr" : "D"
+    } ],
+    "child" : 2
+  }, {
+    "@id" : 5,
+    "right" : 3,
+    "left" : 4,
+    "pop" : "hash-join",
+    "conditions" : [ {
+      "relationship" : "==",
+      "left" : "B",
+      "right" : "A"
+    } ],
+    "joinType" : "INNER"
+  }, {
+    "@id" : 6,
+    "child" : 5,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/join/hj_exchanges.json b/exec/java-exec/src/test/resources/join/hj_exchanges.json
index 318da06..e3f0e45 100644
--- a/exec/java-exec/src/test/resources/join/hj_exchanges.json
+++ b/exec/java-exec/src/test/resources/join/hj_exchanges.json
@@ -17,14 +17,15 @@
     "storage" : {
       "type" : "file",
       "connection" : "classpath:///",
+      "enabled": true,
       "workspaces" : {
         "default" : {
-          location: "/",
-          writable: false
+          "location" : "/",
+          "writable" : false
         },
         "home" : {
-          location: "/",
-          writable: false
+          "location" : "/",
+          "writable" : false
         }
       },
       "formats" : null
@@ -48,12 +49,12 @@
       "connection" : "classpath:///",
       "workspaces" : {
         "default" : {
-          location: "/",
-          writable: false
+          "location" : "/",
+          "writable" : false
         },
         "home" : {
-          location: "/",
-          writable: false
+          "location" : "/",
+          "writable" : false
         }
       },
       "formats" : null
@@ -102,4 +103,4 @@
     "@id" : 11,
     "child" : 10
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/join/hj_exchanges1.json b/exec/java-exec/src/test/resources/join/hj_exchanges1.json
index 6517845..3122b3a 100644
--- a/exec/java-exec/src/test/resources/join/hj_exchanges1.json
+++ b/exec/java-exec/src/test/resources/join/hj_exchanges1.json
@@ -17,6 +17,7 @@
     "storage" : {
       "type" : "file",
       "connection" : "classpath:///",
+      "enabled": true,
       "workspaces" : null,
       "formats" : {
         "json" : {
@@ -188,4 +189,4 @@
     "@id" : 20,
     "child" : 19
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/join/hj_multi_condition_join.json b/exec/java-exec/src/test/resources/join/hj_multi_condition_join.json
index 4033ca9..eab609c 100644
--- a/exec/java-exec/src/test/resources/join/hj_multi_condition_join.json
+++ b/exec/java-exec/src/test/resources/join/hj_multi_condition_join.json
@@ -8,60 +8,69 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "file:///"},
-      files:["#{TEST_FILE_1}"]
-  },
-  {
-      @id:2,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "file:///"},
-      files:["#{TEST_FILE_2}"]
-  },
-  {
-      "pop" : "project",
-      "@id" : 3,
-      "exprs" : [ {
-        "ref" : "A",
-        "expr" : "A"
-      },
-      { "ref" : "CCOL", "expr" : "C" }
-      ],
-
-      "child" : 1
-  },
-  {
-      "pop" : "project",
-      "@id" : 4,
-      "exprs" : [ {
-        "ref" : "B",
-        "expr" : "B"
-      },
-      { "ref" : "DCOL", "expr" : "D" }
-      ],
-
-      "child" : 2
-  },
-      {
-        @id: 5,
-        right: 3,
-        left: 4,
-        pop: "hash-join",
-        conditions: [
-        {relationship: "==", left: "B", right: "A"},
-        {relationship: "==", left: "DCOL", right: "CCOL"}
-        ],
-        joinType : "INNER"
-      },
-      {
-        @id: 6,
-        child: 5,
-        pop: "screen"
-      }
-    ]
-  }
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "enabled": true
+    },
+    "files" : [ "#{TEST_FILE_1}" ]
+  }, {
+    "@id" : 2,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///"
+    },
+    "files" : [ "#{TEST_FILE_2}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 3,
+    "exprs" : [ {
+      "ref" : "A",
+      "expr" : "A"
+    }, {
+      "ref" : "CCOL",
+      "expr" : "C"
+    } ],
+    "child" : 1
+  }, {
+    "pop" : "project",
+    "@id" : 4,
+    "exprs" : [ {
+      "ref" : "B",
+      "expr" : "B"
+    }, {
+      "ref" : "DCOL",
+      "expr" : "D"
+    } ],
+    "child" : 2
+  }, {
+    "@id" : 5,
+    "right" : 3,
+    "left" : 4,
+    "pop" : "hash-join",
+    "conditions" : [ {
+      "relationship" : "==",
+      "left" : "B",
+      "right" : "A"
+    }, {
+      "relationship" : "==",
+      "left" : "DCOL",
+      "right" : "CCOL"
+    } ],
+    "joinType" : "INNER"
+  }, {
+    "@id" : 6,
+    "child" : 5,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/join/mergeJoinExpr.json b/exec/java-exec/src/test/resources/join/mergeJoinExpr.json
index c0027ea..d596d7c 100644
--- a/exec/java-exec/src/test/resources/join/mergeJoinExpr.json
+++ b/exec/java-exec/src/test/resources/join/mergeJoinExpr.json
@@ -170,4 +170,4 @@
       "outputRowCount" : 25.0
     }
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/join/merge_inner_single_batch.json b/exec/java-exec/src/test/resources/join/merge_inner_single_batch.json
index 37ba9a4..1e7cdf4 100644
--- a/exec/java-exec/src/test/resources/join/merge_inner_single_batch.json
+++ b/exec/java-exec/src/test/resources/join/merge_inner_single_batch.json
@@ -1,42 +1,51 @@
 {
-  head:{
-    type:"APACHE_DRILL_PHYSICAL",
-    version:"1",
-    generator:{
-      type:"manual"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
     }
   },
-  graph:[
-    {
-      @id:1,
-      pop:"fs-sub-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "file:///"},
-      files:[
-        { path: "#{LEFT_FILE}" }
-      ]
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-sub-scan",
+    "format" : {
+      "type" : "json"
     },
-    {
-      @id:2,
-      pop:"fs-sub-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "file:///"},
-      files:[
-        { path: "#{RIGHT_FILE}" }
-      ]
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///"
     },
-    {
-      @id: 3,
-      left: 1,
-      right: 2,
-      pop: "merge-join",
-      joinType: "INNER",
-      conditions: [ { relationship: "==", left: "a", right: "aa" } ]
+    "files" : [ {
+      "path" : "#{LEFT_FILE}"
+    } ]
+  }, {
+    "@id" : 2,
+    "pop" : "fs-sub-scan",
+    "format" : {
+      "type" : "json"
     },
-    {
-      @id: 4,
-      child: 3,
-      pop: "screen"
-    }
-  ]
-}
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///"
+    },
+    "files" : [ {
+      "path" : "#{RIGHT_FILE}"
+    } ]
+  }, {
+    "@id" : 3,
+    "left" : 1,
+    "right" : 2,
+    "pop" : "merge-join",
+    "joinType" : "INNER",
+    "conditions" : [ {
+      "relationship" : "==",
+      "left" : "a",
+      "right" : "aa"
+    } ]
+  }, {
+    "@id" : 4,
+    "child" : 3,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/join/merge_join_nullkey.json b/exec/java-exec/src/test/resources/join/merge_join_nullkey.json
index fb6bfc2..490500e 100644
--- a/exec/java-exec/src/test/resources/join/merge_join_nullkey.json
+++ b/exec/java-exec/src/test/resources/join/merge_join_nullkey.json
@@ -204,4 +204,4 @@
       "outputRowCount" : 18.0
     }
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/join/merge_multi_batch.json b/exec/java-exec/src/test/resources/join/merge_multi_batch.json
index f209a98..975f620 100644
--- a/exec/java-exec/src/test/resources/join/merge_multi_batch.json
+++ b/exec/java-exec/src/test/resources/join/merge_multi_batch.json
@@ -1,42 +1,52 @@
 {
-  head:{
-    type:"APACHE_DRILL_PHYSICAL",
-    version:"1",
-    generator:{
-      type:"manual"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
     }
   },
-  graph:[
-    {
-      @id:1,
-      pop:"fs-sub-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "file:///"},
-      files:[
-        { path: "#{LEFT_FILE}" }
-      ]
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-sub-scan",
+    "format" : {
+      "type" : "json"
     },
-    {
-      @id:2,
-      pop:"fs-sub-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "file:///"},
-      files:[
-        { path: "#{RIGHT_FILE}" }
-      ]
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "enabled": true
     },
-    {
-      @id: 3,
-      left: 1,
-      right: 2,
-      pop: "merge-join",
-      joinType: "LEFT",
-      conditions: [ { relationship: "==", left: "a", right: "aa" } ]
+    "files" : [ {
+      "path" : "#{LEFT_FILE}"
+    } ]
+  }, {
+    "@id" : 2,
+    "pop" : "fs-sub-scan",
+    "format" : {
+      "type" : "json"
     },
-    {
-      @id: 4,
-      child: 3,
-      pop: "screen"
-    }
-  ]
-}
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///"
+    },
+    "files" : [ {
+      "path" : "#{RIGHT_FILE}"
+    } ]
+  }, {
+    "@id" : 3,
+    "left" : 1,
+    "right" : 2,
+    "pop" : "merge-join",
+    "joinType" : "LEFT",
+    "conditions" : [ {
+      "relationship" : "==",
+      "left" : "a",
+      "right" : "aa"
+    } ]
+  }, {
+    "@id" : 4,
+    "child" : 3,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/join/merge_single_batch.json b/exec/java-exec/src/test/resources/join/merge_single_batch.json
index f209a98..975f620 100644
--- a/exec/java-exec/src/test/resources/join/merge_single_batch.json
+++ b/exec/java-exec/src/test/resources/join/merge_single_batch.json
@@ -1,42 +1,52 @@
 {
-  head:{
-    type:"APACHE_DRILL_PHYSICAL",
-    version:"1",
-    generator:{
-      type:"manual"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
     }
   },
-  graph:[
-    {
-      @id:1,
-      pop:"fs-sub-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "file:///"},
-      files:[
-        { path: "#{LEFT_FILE}" }
-      ]
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-sub-scan",
+    "format" : {
+      "type" : "json"
     },
-    {
-      @id:2,
-      pop:"fs-sub-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "file:///"},
-      files:[
-        { path: "#{RIGHT_FILE}" }
-      ]
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "enabled": true
     },
-    {
-      @id: 3,
-      left: 1,
-      right: 2,
-      pop: "merge-join",
-      joinType: "LEFT",
-      conditions: [ { relationship: "==", left: "a", right: "aa" } ]
+    "files" : [ {
+      "path" : "#{LEFT_FILE}"
+    } ]
+  }, {
+    "@id" : 2,
+    "pop" : "fs-sub-scan",
+    "format" : {
+      "type" : "json"
     },
-    {
-      @id: 4,
-      child: 3,
-      pop: "screen"
-    }
-  ]
-}
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///"
+    },
+    "files" : [ {
+      "path" : "#{RIGHT_FILE}"
+    } ]
+  }, {
+    "@id" : 3,
+    "left" : 1,
+    "right" : 2,
+    "pop" : "merge-join",
+    "joinType" : "LEFT",
+    "conditions" : [ {
+      "relationship" : "==",
+      "left" : "a",
+      "right" : "aa"
+    } ]
+  }, {
+    "@id" : 4,
+    "child" : 3,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/join/mj_multi_condition.json b/exec/java-exec/src/test/resources/join/mj_multi_condition.json
index 9cc1beb..05ad328 100644
--- a/exec/java-exec/src/test/resources/join/mj_multi_condition.json
+++ b/exec/java-exec/src/test/resources/join/mj_multi_condition.json
@@ -18,7 +18,8 @@
       "type" : "file",
       "connection" : "classpath:///",
       "workspaces" : null,
-      "formats" : null
+      "formats" : null,
+      "enabled": true
     },
     "format" : {
       "type" : "parquet"
@@ -126,4 +127,4 @@
     "@id" : 11,
     "child" : 10
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/largefiles/merging_receiver_large_data.json b/exec/java-exec/src/test/resources/largefiles/merging_receiver_large_data.json
index 265486d..b88ce82 100644
--- a/exec/java-exec/src/test/resources/largefiles/merging_receiver_large_data.json
+++ b/exec/java-exec/src/test/resources/largefiles/merging_receiver_large_data.json
@@ -1,9 +1,9 @@
 {
-  head:{
-    type:"APACHE_DRILL_PHYSICAL",
-    version:"1",
-    generator:{
-      type:"manual"
+  "head":{
+    "type":"APACHE_DRILL_PHYSICAL",
+    "version":"1",
+    "generator":{
+      "type":"manual"
     }
   },
   "graph" : [ {
@@ -13,6 +13,7 @@
     "storage" : {
       "type" : "file",
       "connection" : "file:///",
+      "enabled": true,
       "workspaces" : {
         "root" : {
           "location" : "/",
diff --git a/exec/java-exec/src/test/resources/lateraljoin/lateralplan1.json b/exec/java-exec/src/test/resources/lateraljoin/lateralplan1.json
index 9be5878..34ac456 100644
--- a/exec/java-exec/src/test/resources/lateraljoin/lateralplan1.json
+++ b/exec/java-exec/src/test/resources/lateraljoin/lateralplan1.json
@@ -53,7 +53,7 @@
     "exprs" : [ {
       "ref" : "`ITEM`",
       "expr" : "`o_shop`"
-    }],
+    } ],
     "child" : 7,
     "outputProj" : false,
     "initialAllocation" : 1000000,
@@ -64,7 +64,7 @@
     "@id" : 4,
     "left" : 5,
     "right" : 6,
-    "unnestForLateralJoin": 7,
+    "unnestForLateralJoin" : 7,
     "initialAllocation" : 1000000,
     "maxAllocation" : 10000000000,
     "cost" : 1000.0
@@ -83,7 +83,7 @@
     "initialAllocation" : 1000000,
     "maxAllocation" : 10000000000,
     "cost" : 1000.0
-  },  {
+  }, {
     "pop" : "screen",
     "@id" : 0,
     "child" : 3,
@@ -91,5 +91,4 @@
     "maxAllocation" : 10000000000,
     "cost" : 1000.0
   } ]
-}
-
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/limit/limit_exchanges.json b/exec/java-exec/src/test/resources/limit/limit_exchanges.json
index c2303c8..b706537 100644
--- a/exec/java-exec/src/test/resources/limit/limit_exchanges.json
+++ b/exec/java-exec/src/test/resources/limit/limit_exchanges.json
@@ -18,6 +18,7 @@
       "type" : "file",
       "connection" : "classpath:///",
       "workspaces" : null,
+      "enabled": true,
       "formats" : {
         "psv" : {
           "type" : "text",
@@ -63,8 +64,7 @@
     "pop" : "union-exchange",
     "@id" : 4,
     "child" : 3
-  },
-{
+  }, {
     "pop" : "project",
     "@id" : 5,
     "exprs" : [ {
@@ -72,8 +72,7 @@
       "expr" : "`N_NATIONKEY`"
     } ],
     "child" : 4
-  },
-{
+  }, {
     "pop" : "limit",
     "@id" : 6,
     "child" : 5,
@@ -84,4 +83,4 @@
     "@id" : 7,
     "child" : 6
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/logical_group.json b/exec/java-exec/src/test/resources/logical_group.json
index f4dd169..dd4bd84 100644
--- a/exec/java-exec/src/test/resources/logical_group.json
+++ b/exec/java-exec/src/test/resources/logical_group.json
@@ -1,90 +1,91 @@
 {
-  head : {
-    type : "APACHE_DRILL_LOGICAL",
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "type" : "APACHE_DRILL_LOGICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     }
   },
-  storage : {
-    parquet-local : {
-      type : "parquet",
-      dfsName : "file:///"
+  "storage" : {
+    "parquet-local" : {
+      "type" : "parquet",
+      "dfsName" : "file:///",
+      "enabled": true
     }
   },
-  query : [ {
-    op : "scan",
-    @id : 1,
-    memo : "initial_scan",
-    storageengine : "parquet-local",
-    selection : [ {
-      path : "/tmp/parquet"
+  "query" : [ {
+    "op" : "scan",
+    "@id" : 1,
+    "memo" : "initial_scan",
+    "storageengine" : "parquet-local",
+    "selection" : [ {
+      "path" : "/tmp/parquet"
     } ],
-    ref : "_MAP"
+    "ref" : "_MAP"
   }, {
-    op : "project",
-    @id : 2,
-    input : 1,
-    projections : [ {
-      ref : "$f0",
-      expr : "_MAP.b"
+    "op" : "project",
+    "@id" : 2,
+    "input" : 1,
+    "projections" : [ {
+      "ref" : "$f0",
+      "expr" : "_MAP.b"
     }, {
-      ref : "$f1",
-      expr : "_MAP.d"
+      "ref" : "$f1",
+      "expr" : "_MAP.d"
     }, {
-      ref : "$f2",
-      expr : "1"
+      "ref" : "$f2",
+      "expr" : "1"
     }, {
-      ref : "$f3",
-      expr : "2"
+      "ref" : "$f3",
+      "expr" : "2"
     } ]
   }, {
-    op : "segment",
-    @id : 3,
-    input : 2,
-    exprs : [ "$f0", "$f1" ],
-    name : "segment"
+    "op" : "segment",
+    "@id" : 3,
+    "input" : 2,
+    "exprs" : [ "$f0", "$f1" ],
+    "name" : "segment"
   }, {
-    op : "collapsingaggregate",
-    @id : 4,
-    input : 3,
-    within : "segment",
-    target : null,
-    carryovers : [ "$f0", "$f1" ],
-    aggregations : [ {
-      ref : "EXPR$0",
-      expr : "count($f2) "
+    "op" : "collapsingaggregate",
+    "@id" : 4,
+    "input" : 3,
+    "within" : "segment",
+    "target" : null,
+    "carryovers" : [ "$f0", "$f1" ],
+    "aggregations" : [ {
+      "ref" : "EXPR$0",
+      "expr" : "count($f2) "
     }, {
-      ref : "EXPR$1",
-      expr : "count($f3) "
+      "ref" : "EXPR$1",
+      "expr" : "count($f3) "
     } ]
   }, {
-    op : "project",
-    @id : 5,
-    input : 4,
-    projections : [ {
-      ref : "EXPR$0",
-      expr : "EXPR$0"
+    "op" : "project",
+    "@id" : 5,
+    "input" : 4,
+    "projections" : [ {
+      "ref" : "EXPR$0",
+      "expr" : "EXPR$0"
     }, {
-      ref : "EXPR$1",
-      expr : "EXPR$1"
+      "ref" : "EXPR$1",
+      "expr" : "EXPR$1"
     }, {
-      ref : "C",
-      expr : "$f0"
+      "ref" : "C",
+      "expr" : "$f0"
     }, {
-      ref : "X",
-      expr : "$f1"
+      "ref" : "X",
+      "expr" : "$f1"
     } ]
   }, {
-    op : "store",
-    @id : 6,
-    memo : "output sink",
-    input : 5,
-    target : {
-      number : 0
+    "op" : "store",
+    "@id" : 6,
+    "memo" : "output sink",
+    "input" : 5,
+    "target" : {
+      "number" : 0
     },
-    partition : null,
-    storageEngine : "queue"
+    "partition" : null,
+    "storageEngine" : "queue"
   } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/logical_json_scan.json b/exec/java-exec/src/test/resources/logical_json_scan.json
index b1cc4ce..2fa2d6f 100644
--- a/exec/java-exec/src/test/resources/logical_json_scan.json
+++ b/exec/java-exec/src/test/resources/logical_json_scan.json
@@ -1,34 +1,35 @@
 {
-  head : {
-    type : "APACHE_DRILL_LOGICAL",
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "type" : "APACHE_DRILL_LOGICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     }
   },
-  storage : {
-    jsonl : {
-      type : "file",
-      connection : "file:///"
+  "storage" : {
+    "jsonl" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "enabled": true
     }
   },
-  query : [ {
-    op : "scan",
-    @id : 1,
-    memo : "initial_scan",
-    storageengine : "jsonl",
-    files : [ path : "/src/drill/sandbox/prototype/exec/ref/src/test/resources/employees.json" ],
-    ref : "_MAP"
+  "query" : [ {
+    "op" : "scan",
+    "@id" : 1,
+    "memo" : "initial_scan",
+    "storageengine" : "jsonl",
+    "files" : { "path" : "/src/drill/sandbox/prototype/exec/ref/src/test/resources/employees.json" },
+    "ref" : "_MAP"
   }, {
-    op : "store",
-    @id : 2,
-    memo : "output sink",
-    input : 1,
-    target : {
-      number : 0
+    "op" : "store",
+    "@id" : 2,
+    "memo" : "output sink",
+    "input" : 1,
+    "target" : {
+      "number" : 0
     },
-    partition : null,
-    storageEngine : "queue"
+    "partition" : null,
+    "storageEngine" : "queue"
   } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/logical_order.json b/exec/java-exec/src/test/resources/logical_order.json
index 3e7fdfa..c77a57b 100644
--- a/exec/java-exec/src/test/resources/logical_order.json
+++ b/exec/java-exec/src/test/resources/logical_order.json
@@ -1,57 +1,58 @@
 {
-  head : {
-    type : "APACHE_DRILL_LOGICAL",
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "type" : "APACHE_DRILL_LOGICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     }
   },
-  storage : {
-    parquet-local : {
-      type : "parquet",
-      dfsName : "file:///"
+  "storage" : {
+    "parquet-local" : {
+      "type" : "parquet",
+      "dfsName" : "file:///",
+      "enabled" : true
     }
   },
-  query : [ {
-    op : "scan",
-    @id : 1,
-    memo : "initial_scan",
-    storageengine : "parquet-local",
-    selection : [ {
-      path : "/tmp/parquet"
+  "query" : [ {
+    "op" : "scan",
+    "@id" : 1,
+    "memo" : "initial_scan",
+    "storageengine" : "parquet-local",
+    "selection" : [ {
+      "path" : "/tmp/parquet"
     } ],
-    ref : "_MAP"
+    "ref" : "_MAP"
   }, {
-    op : "project",
-    @id : 2,
-    input : 1,
-    projections : [ {
-      ref : "EXPR$0",
-      expr : "_MAP.d"
+    "op" : "project",
+    "@id" : 2,
+    "input" : 1,
+    "projections" : [ {
+      "ref" : "EXPR$0",
+      "expr" : "_MAP.d"
     }, {
-      ref : "C",
-      expr : "_MAP.b"
+      "ref" : "C",
+      "expr" : "_MAP.b"
     } ]
   }, {
-    op : "order",
-    @id : 3,
-    input : 2,
-    within : null,
-    orderings : [ {
-      order : "asc",
-      expr : "EXPR$0",
-      nullCollation : "NULLS_LAST"
+    "op" : "order",
+    "@id" : 3,
+    "input" : 2,
+    "within" : null,
+    "orderings" : [ {
+      "order" : "asc",
+      "expr" : "EXPR$0",
+      "nullCollation" : "NULLS_LAST"
     } ]
   }, {
-    op : "store",
-    @id : 4,
-    memo : "output sink",
-    input : 3,
-    target : {
-      number : 0
+    "op" : "store",
+    "@id" : 4,
+    "memo" : "output sink",
+    "input" : 3,
+    "target" : {
+      "number" : 0
     },
-    partition : null,
-    storageEngine : "queue"
+    "partition" : null,
+    "storageEngine" : "queue"
   } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/logical_string_filter.json b/exec/java-exec/src/test/resources/logical_string_filter.json
index d8875d3..5e9f6c2 100644
--- a/exec/java-exec/src/test/resources/logical_string_filter.json
+++ b/exec/java-exec/src/test/resources/logical_string_filter.json
@@ -1,52 +1,53 @@
 {
-  head : {
-    type : "APACHE_DRILL_LOGICAL",
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "type" : "APACHE_DRILL_LOGICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     }
   },
-  storage : {
-    parquet-local : {
-      type : "parquet",
-      dfsName : "file:///"
+  "storage" : {
+    "parquet-local" : {
+      "type" : "parquet",
+      "dfsName" : "file:///",
+      "enabled" : true
     }
   },
-  query : [ {
-    op : "scan",
-    @id : 1,
-    memo : "initial_scan",
-    storageengine : "parquet-local",
-    selection : [ {
-      path : "/tmp/region.parquet"
+  "query" : [ {
+    "op" : "scan",
+    "@id" : 1,
+    "memo" : "initial_scan",
+    "storageengine" : "parquet-local",
+    "selection" : [ {
+      "path" : "/tmp/region.parquet"
     } ],
-    ref : "_MAP"
+    "ref" : "_MAP"
   }, {
-    op : "filter",
-    @id : 2,
-    input : 1,
-    expr : " (_MAP.R_NAME)  == ('AFRICA') "
+    "op" : "filter",
+    "@id" : 2,
+    "input" : 1,
+    "expr" : " (_MAP.R_NAME)  == ('AFRICA') "
   }, {
-    op : "project",
-    @id : 3,
-    input : 2,
-    projections : [ {
-      ref : "EXPR$0",
-      expr : "_MAP.R_REGIONKEY"
+    "op" : "project",
+    "@id" : 3,
+    "input" : 2,
+    "projections" : [ {
+      "ref" : "EXPR$0",
+      "expr" : "_MAP.R_REGIONKEY"
     }, {
-      ref : "NAME",
-      expr : "_MAP.R_NAME"
+      "ref" : "NAME",
+      "expr" : "_MAP.R_NAME"
     } ]
   }, {
-    op : "store",
-    @id : 4,
-    memo : "output sink",
-    input : 3,
-    target : {
-      number : 0
+    "op" : "store",
+    "@id" : 4,
+    "memo" : "output sink",
+    "input" : 3,
+    "target" : {
+      "number" : 0
     },
-    partition : null,
-    storageEngine : "queue"
+    "partition" : null,
+    "storageEngine" : "queue"
   } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/parquet/par_writer_test.json b/exec/java-exec/src/test/resources/parquet/par_writer_test.json
index 34f2ba6..6737ae8 100644
--- a/exec/java-exec/src/test/resources/parquet/par_writer_test.json
+++ b/exec/java-exec/src/test/resources/parquet/par_writer_test.json
@@ -1,26 +1,26 @@
-  {
-    head : {
-      version : 1,
-          generator : {
-        type : "manual",
-            info : "na"
-      },
-      type : "APACHE_DRILL_PHYSICAL"
+{
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "manual",
+      "info" : "na"
     },
-    graph : [ {
-    pop : "parquet-scan",
-    @id : 1,
-        entries : [ {
-      path : "/tpch/lineitem.parquet"
+    "type" : "APACHE_DRILL_PHYSICAL"
+  },
+  "graph" : [ {
+    "pop" : "parquet-scan",
+    "@id" : 1,
+    "entries" : [ {
+      "path" : "/tpch/lineitem.parquet"
     } ],
-    storage : {
-      type : "file",
-      connection : "classpath:///"
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///"
     },
-    columns: [ "L_COMMENT"]
+    "columns" : [ "L_COMMENT" ]
   }, {
-    pop : "screen",
-    @id : 2,
-        child : 1
+    "pop" : "screen",
+    "@id" : 2,
+    "child" : 1
   } ]
-  }
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/parquet/parquet_group_scan_union_screen.json b/exec/java-exec/src/test/resources/parquet/parquet_group_scan_union_screen.json
index 0ec2994..8d9a4dd 100644
--- a/exec/java-exec/src/test/resources/parquet/parquet_group_scan_union_screen.json
+++ b/exec/java-exec/src/test/resources/parquet/parquet_group_scan_union_screen.json
@@ -1,54 +1,55 @@
 {
-  head : {
-    type : "APACHE_DRILL_PHYSICAL",
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     }
   },
-  graph : [ {
-    pop : "parquet-scan",
-    @id : 1,
-    entries : [ {
-      path : "/tmp/tpc-h/supplier"
+  "graph" : [ {
+    "pop" : "parquet-scan",
+    "@id" : 1,
+    "entries" : [ {
+      "path" : "/tmp/tpc-h/supplier"
     } ],
-    storageengine : {
-      type : "parquet",
-      dfsName : "file:///"
+    "storageengine" : {
+      "type" : "parquet",
+      "dfsName" : "file:///",
+      "enabled": true
     },
-    ref : "_MAP",
-    fragmentPointer : 0
+    "ref" : "_MAP",
+    "fragmentPointer" : 0
   }, {
-    pop : "project",
-    @id : 2,
-    exprs : [ {
-      ref : "output.$f0",
-      expr : "1"
+    "pop" : "project",
+    "@id" : 2,
+    "exprs" : [ {
+      "ref" : "output.$f0",
+      "expr" : "1"
     } ],
-    child : 1
+    "child" : 1
   }, {
-    pop : "sort",
-    @id : 3,
-    child : 2,
-    orderings : [ ],
-    reverse : false
+    "pop" : "sort",
+    "@id" : 3,
+    "child" : 2,
+    "orderings" : [ ],
+    "reverse" : false
   }, {
-    pop : "streaming-aggregate",
-    @id : 4,
-    child : 3,
-    keys : [ ],
-    exprs : [ {
-      ref : "EXPR$0",
-      expr : "count($f0) "
+    "pop" : "streaming-aggregate",
+    "@id" : 4,
+    "child" : 3,
+    "keys" : [ ],
+    "exprs" : [ {
+      "ref" : "EXPR$0",
+      "expr" : "count($f0) "
     } ]
   }, {
-    pop : "selection-vector-remover",
-    @id : 5,
-    child : 4
+    "pop" : "selection-vector-remover",
+    "@id" : 5,
+    "child" : 4
   }, {
-    pop : "screen",
-    @id : 6,
-    child : 5
+    "pop" : "screen",
+    "@id" : 6,
+    "child" : 5
   } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/parquet/parquet_nullable.json b/exec/java-exec/src/test/resources/parquet/parquet_nullable.json
index 3e09f83..e7df17d 100644
--- a/exec/java-exec/src/test/resources/parquet/parquet_nullable.json
+++ b/exec/java-exec/src/test/resources/parquet/parquet_nullable.json
@@ -1,42 +1,36 @@
 {
-  head:{
-    type:"APACHE_DRILL_LOGICAL",
-    version:"1",
-    generator:{
-      type:"manual",
-      info:"na"
+  "head" : {
+    "type" : "APACHE_DRILL_LOGICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual",
+      "info" : "na"
     }
   },
-  storage:{
-    "dfs" :
-      {
-        "type":"named",
-        "name":"dfs"
-      }
+  "storage" : {
+    "dfs" : {
+      "type" : "named",
+      "name" : "dfs"
+    }
   },
-  query:[
-    {
-      @id:"1",
-      op:"scan",
-      memo:"initial_scan",
-      storageengine:"dfs",
-      selection: {
-          format: {type: "parquet"},
-          files: [
-            "/tmp/nullable_test.parquet"
-          ]
-       }
-    },
-    {
-      @id:"2",
-      input: 1,
-      op: "store",
-      memo: "output sink",
-      target: {
-        file: "console:///stdout"
-      }
-
+  "query" : [ {
+    "@id" : "1",
+    "op" : "scan",
+    "memo" : "initial_scan",
+    "storageengine" : "dfs",
+    "selection" : {
+      "format" : {
+        "type" : "parquet"
+      },
+      "files" : [ "/tmp/nullable_test.parquet" ]
+    }
+  }, {
+    "@id" : "2",
+    "input" : 1,
+    "op" : "store",
+    "memo" : "output sink",
+    "target" : {
+      "file" : "console:///stdout"
     }
-
-  ]
+  } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/parquet/parquet_nullable_varlen.json b/exec/java-exec/src/test/resources/parquet/parquet_nullable_varlen.json
index 9547f00..a450a2d 100644
--- a/exec/java-exec/src/test/resources/parquet/parquet_nullable_varlen.json
+++ b/exec/java-exec/src/test/resources/parquet/parquet_nullable_varlen.json
@@ -1,42 +1,36 @@
 {
-  head:{
-    type:"APACHE_DRILL_LOGICAL",
-    version:"1",
-    generator:{
-      type:"manual",
-      info:"na"
+  "head" : {
+    "type" : "APACHE_DRILL_LOGICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual",
+      "info" : "na"
     }
   },
-  storage:{
-    "dfs" :
-      {
-        "type":"named",
-        "name":"dfs"
-      }
+  "storage" : {
+    "dfs" : {
+      "type" : "named",
+      "name" : "dfs"
+    }
   },
-  query:[
-    {
-      @id:"1",
-      op:"scan",
-      memo:"initial_scan",
-      storageengine:"dfs",
-      selection: {
-          format: {type: "parquet"},
-          files: [
-            "/tmp/nullable_varlen.parquet"
-          ]
-       }
-    },
-    {
-      @id:"2",
-      input: 1,
-      op: "store",
-      memo: "output sink",
-      target: {
-        file: "console:///stdout"
-      }
-
+  "query" : [ {
+    "@id" : "1",
+    "op" : "scan",
+    "memo" : "initial_scan",
+    "storageengine" : "dfs",
+    "selection" : {
+      "format" : {
+        "type" : "parquet"
+      },
+      "files" : [ "/tmp/nullable_varlen.parquet" ]
+    }
+  }, {
+    "@id" : "2",
+    "input" : 1,
+    "op" : "store",
+    "memo" : "output sink",
+    "target" : {
+      "file" : "console:///stdout"
     }
-
-  ]
-}
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/parquet/parquet_scan_screen.json b/exec/java-exec/src/test/resources/parquet/parquet_scan_screen.json
index 50281f4..ad817bb 100644
--- a/exec/java-exec/src/test/resources/parquet/parquet_scan_screen.json
+++ b/exec/java-exec/src/test/resources/parquet/parquet_scan_screen.json
@@ -1,43 +1,37 @@
 {
-  head:{
-    type:"APACHE_DRILL_LOGICAL",
-    version:"1",
-    generator:{
-      type:"manual",
-      info:"na"
+  "head" : {
+    "type" : "APACHE_DRILL_LOGICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual",
+      "info" : "na"
     }
   },
-  storage:{
-    "dfs" :
-      {
-        "type":"named",
-        "name":"dfs"
-      }
+  "storage" : {
+    "dfs" : {
+      "type" : "named",
+      "name" : "dfs",
+      "enable" : true
+    }
   },
-  query:[
-    {
-      @id:"1",
-      op:"scan",
-      memo:"initial_scan",
-      storageengine:"dfs",
-      selection: {
-        format: {type: "parquet"},
-        files: [
-          "/tmp/parquet_test_file_many_types",
-          "/tmp/parquet_test_file_many_types"
-        ]
-      }
-    },
-    {
-      @id:"2",
-      input: 1,
-      op: "store",
-      memo: "output sink",
-      target: {
-        file: "console:///stdout"
-      }
-
+  "query" : [ {
+    "@id" : "1",
+    "op" : "scan",
+    "memo" : "initial_scan",
+    "storageengine" : "dfs",
+    "selection" : {
+      "format" : {
+        "type" : "parquet"
+      },
+      "files" : [ "/tmp/parquet_test_file_many_types", "/tmp/parquet_test_file_many_types" ]
+    }
+  }, {
+    "@id" : "2",
+    "input" : 1,
+    "op" : "store",
+    "memo" : "output sink",
+    "target" : {
+      "file" : "console:///stdout"
     }
-
-  ]
-}
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/parquet/parquet_scan_screen_read_entry_replace.json b/exec/java-exec/src/test/resources/parquet/parquet_scan_screen_read_entry_replace.json
index 1bd52b4..6b68192 100644
--- a/exec/java-exec/src/test/resources/parquet/parquet_scan_screen_read_entry_replace.json
+++ b/exec/java-exec/src/test/resources/parquet/parquet_scan_screen_read_entry_replace.json
@@ -11,7 +11,8 @@
     "fs" :
       {
         "type":"file",
-        "connection" : "file:///"
+        "connection" : "file:///",
+        "enabled": true
       }
   },
   query:[
diff --git a/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json b/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
index a49b914..7e24b60 100644
--- a/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
+++ b/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
@@ -1,32 +1,28 @@
 {
-  head : {
-    type : "APACHE_DRILL_PHYSICAL",
-    version : 1,
-    generator : {
-      type : "manual"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "manual"
     }
   },
-  graph : [ {
-    pop : "parquet-scan",
-    @id : 1,
-    entries : [
-    {
-        path : "/tmp/parquet_test_file_many_types"
+  "graph" : [ {
+    "pop" : "parquet-scan",
+    "@id" : 1,
+    "entries" : [ {
+      "path" : "/tmp/parquet_test_file_many_types"
+    } ],
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///"
     }
-    ],
-    storage:{
-                         "type":"file",
-                         "connection" : "file:///"
-                   }
-  },
-  {
-     "@id": 2,
-     "child": 1,
-     "pop": "union-exchange"
-  },
-  {
-    pop : "screen",
-    @id : 3,
-    child : 2
+  }, {
+    "@id" : 2,
+    "child" : 1,
+    "pop" : "union-exchange"
+  }, {
+    "pop" : "screen",
+    "@id" : 3,
+    "child" : 2
   } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/parquet/parquet_selective_column_read.json b/exec/java-exec/src/test/resources/parquet/parquet_selective_column_read.json
index 515affc..519a016 100644
--- a/exec/java-exec/src/test/resources/parquet/parquet_selective_column_read.json
+++ b/exec/java-exec/src/test/resources/parquet/parquet_selective_column_read.json
@@ -1,27 +1,27 @@
 {
-  head : {
-    version : 1,
-    generator : {
-      type : "manual",
-      info : "na"
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "manual",
+      "info" : "na"
     },
-    type : "APACHE_DRILL_PHYSICAL"
+    "type" : "APACHE_DRILL_PHYSICAL"
   },
-  graph : [ {
-    pop : "parquet-scan",
-    @id : 1,
-    entries : [ {
-      path : "/tmp/test.parquet"
+  "graph" : [ {
+    "pop" : "parquet-scan",
+    "@id" : 1,
+    "entries" : [ {
+      "path" : "/tmp/test.parquet"
     } ],
-    storage : {
-      type : "file",
-      connection : "file:///"
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "enabled" : true
     },
-    columns: [ "integer", "bigInt", "bin", "bin2"]
+    "columns" : [ "integer", "bigInt", "bin", "bin2" ]
   }, {
-    pop : "screen",
-    @id : 2,
-    child : 1
+    "pop" : "screen",
+    "@id" : 2,
+    "child" : 1
   } ]
-}
-
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/physical_filter.json b/exec/java-exec/src/test/resources/physical_filter.json
index 878f79d..3a61b83 100644
--- a/exec/java-exec/src/test/resources/physical_filter.json
+++ b/exec/java-exec/src/test/resources/physical_filter.json
@@ -1,44 +1,44 @@
 {
-  head : {
-    type : "APACHE_DRILL_PHYSICAL",
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     }
   },
-  graph : [ {
-    pop : "parquet-scan",
-    @id : 1,
-    entries : [ {
-      path : "/tmp/parquet"
+  "graph" : [ {
+    "pop" : "parquet-scan",
+    "@id" : 1,
+    "entries" : [ {
+      "path" : "/tmp/parquet"
     } ],
-    storageengine: {
-      type: "parquet",
-      dfsName: "file:///"
+    "storageengine" : {
+      "type" : "parquet",
+      "dfsName" : "file:///"
     },
-    ref : "_MAP",
-    fragmentPointer : 0
+    "ref" : "_MAP",
+    "fragmentPointer" : 0
   }, {
-    pop : "filter",
-    @id : 2,
-    child : 1,
-    expr : " (_MAP.bigInt)  < (0) "
+    "pop" : "filter",
+    "@id" : 2,
+    "child" : 1,
+    "expr" : " (_MAP.bigInt)  < (0) "
   }, {
-    pop : "selection-vector-remover",
-    @id : 3,
-    child : 2
+    "pop" : "selection-vector-remover",
+    "@id" : 3,
+    "child" : 2
   }, {
-    pop : "project",
-    @id : 4,
-    exprs : [ {
-      ref : "KEY",
-      expr : "_MAP.bigInt"
+    "pop" : "project",
+    "@id" : 4,
+    "exprs" : [ {
+      "ref" : "KEY",
+      "expr" : "_MAP.bigInt"
     } ],
-    child : 3
+    "child" : 3
   }, {
-    pop : "screen",
-    @id : 5,
-    child : 4
+    "pop" : "screen",
+    "@id" : 5,
+    "child" : 4
   } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/physical_join.json b/exec/java-exec/src/test/resources/physical_join.json
index 0ad268b..bf593d7 100644
--- a/exec/java-exec/src/test/resources/physical_join.json
+++ b/exec/java-exec/src/test/resources/physical_join.json
@@ -1,117 +1,123 @@
 {
-  head : {
-    type : "APACHE_DRILL_PHYSICAL",
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     }
   },
-  graph : [ {
-    pop : "parquet-scan",
-    @id : 1,
-    entries : [ {
-      path : "/tmp/region.parquet"
+  "graph" : [ {
+    "pop" : "parquet-scan",
+    "@id" : 1,
+    "entries" : [ {
+      "path" : "/tmp/region.parquet"
     } ],
-    ref : "_MAP",
-    storageengine: {type: "parquet", dfsName: "file:///" },
-    fragmentPointer : 0
+    "ref" : "_MAP",
+    "storageengine" : {
+      "type" : "parquet",
+      "dfsName" : "file:///"
+    },
+    "fragmentPointer" : 0
   }, {
-    pop : "project",
-    @id : 2,
-    exprs : [ {
-      ref : "REGIONKEY",
-      expr : "_MAP.R_REGIONKEY"
+    "pop" : "project",
+    "@id" : 2,
+    "exprs" : [ {
+      "ref" : "REGIONKEY",
+      "expr" : "_MAP.R_REGIONKEY"
     }, {
-      ref : "NAME",
-      expr : "_MAP.R_NAME"
+      "ref" : "NAME",
+      "expr" : "_MAP.R_NAME"
     } ],
-    child : 1
+    "child" : 1
   }, {
-    pop : "project",
-    @id : 3,
-    exprs : [ {
-      ref : "REGIONKEY0",
-      expr : "REGIONKEY"
+    "pop" : "project",
+    "@id" : 3,
+    "exprs" : [ {
+      "ref" : "REGIONKEY0",
+      "expr" : "REGIONKEY"
     }, {
-      ref : "NAME0",
-      expr : "NAME"
+      "ref" : "NAME0",
+      "expr" : "NAME"
     } ],
-    child : 2
+    "child" : 2
   }, {
-    pop : "sort",
-    @id : 4,
-    child : 3,
-    orderings : [ {
-      order : "ASC",
-      expr : "NAME0"
+    "pop" : "sort",
+    "@id" : 4,
+    "child" : 3,
+    "orderings" : [ {
+      "order" : "ASC",
+      "expr" : "NAME0"
     } ],
-    reverse : false
+    "reverse" : false
   }, {
-    pop : "selection-vector-remover",
-    @id : 5,
-    child : 4
+    "pop" : "selection-vector-remover",
+    "@id" : 5,
+    "child" : 4
   }, {
-    pop : "parquet-scan",
-    @id : 6,
-    entries : [ {
-      path : "/tmp/nation.parquet"
+    "pop" : "parquet-scan",
+    "@id" : 6,
+    "entries" : [ {
+      "path" : "/tmp/nation.parquet"
     } ],
-    storageengine: {type: "parquet", dfsName: "file:///" },
-    ref : "_MAP",
-    fragmentPointer : 0
+    "storageengine" : {
+      "type" : "parquet",
+      "dfsName" : "file:///"
+    },
+    "ref" : "_MAP",
+    "fragmentPointer" : 0
   }, {
-    pop : "project",
-    @id : 7,
-    exprs : [ {
-      ref : "NAME",
-      expr : "_MAP.N_NAME"
+    "pop" : "project",
+    "@id" : 7,
+    "exprs" : [ {
+      "ref" : "NAME",
+      "expr" : "_MAP.N_NAME"
     }, {
-      ref : "REGIONKEY",
-      expr : "_MAP.N_REGIONKEY"
+      "ref" : "REGIONKEY",
+      "expr" : "_MAP.N_REGIONKEY"
     } ],
-    child : 6
+    "child" : 6
   }, {
-    pop : "sort",
-    @id : 8,
-    child : 7,
-    orderings : [ {
-      order : "ASC",
-      expr : "REGIONKEY"
+    "pop" : "sort",
+    "@id" : 8,
+    "child" : 7,
+    "orderings" : [ {
+      "order" : "ASC",
+      "expr" : "REGIONKEY"
     } ],
-    reverse : false
+    "reverse" : false
   }, {
-    pop : "selection-vector-remover",
-    @id : 9,
-    child : 8
+    "pop" : "selection-vector-remover",
+    "@id" : 9,
+    "child" : 8
   }, {
-    pop : "merge-join",
-    @id : 10,
-    left: 9,
-    right: 5,
-    conditions : [ {
-      relationship : "==",
-      left : "REGIONKEY",
-      right : "NAME0"
+    "pop" : "merge-join",
+    "@id" : 10,
+    "left" : 9,
+    "right" : 5,
+    "conditions" : [ {
+      "relationship" : "==",
+      "left" : "REGIONKEY",
+      "right" : "NAME0"
     } ]
   }, {
-    pop : "selection-vector-remover",
-    @id : 11,
-    child : 10
+    "pop" : "selection-vector-remover",
+    "@id" : 11,
+    "child" : 10
   }, {
-    pop : "project",
-    @id : 12,
-    exprs : [ {
-      ref : "NAME",
-      expr : "NAME"
+    "pop" : "project",
+    "@id" : 12,
+    "exprs" : [ {
+      "ref" : "NAME",
+      "expr" : "NAME"
     }, {
-      ref : "NAME0",
-      expr : "NAME0"
+      "ref" : "NAME0",
+      "expr" : "NAME0"
     } ],
-    child : 11
+    "child" : 11
   }, {
-    pop : "screen",
-    @id : 13,
-    child : 12
+    "pop" : "screen",
+    "@id" : 13,
+    "child" : 12
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/physical_json_scan_test1.json b/exec/java-exec/src/test/resources/physical_json_scan_test1.json
index 5013d6d..9019f5a 100644
--- a/exec/java-exec/src/test/resources/physical_json_scan_test1.json
+++ b/exec/java-exec/src/test/resources/physical_json_scan_test1.json
@@ -1,28 +1,30 @@
 {
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
+    }
+  },
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
     },
-    graph:[
-        {
-            @id:1,
-            pop:"fs-scan",
-            format: {type: "json"},
-            storage:{type: "file", connection: "file:///"},
-            files: [ "#{TEST_FILE}" ]
-        },
-        {
-            @id: 2,
-            child: 1,
-            pop: "union-exchange"
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "enabled" : true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "@id" : 2,
+    "child" : 1,
+    "pop" : "union-exchange"
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/physical_order_varbinary.json b/exec/java-exec/src/test/resources/physical_order_varbinary.json
index 3493278..ef17d00 100644
--- a/exec/java-exec/src/test/resources/physical_order_varbinary.json
+++ b/exec/java-exec/src/test/resources/physical_order_varbinary.json
@@ -1,48 +1,51 @@
 {
-  head : {
-    type : "APACHE_DRILL_PHYSICAL",
-    version : 1,
-    generator : {
-      type : "optiq",
-      info : "na"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : 1,
+    "generator" : {
+      "type" : "optiq",
+      "info" : "na"
     }
   },
-  graph : [ {
-    pop : "parquet-scan",
-    @id : 1,
-    entries : [ {
-      path : "/tmp/nation.parquet"
+  "graph" : [ {
+    "pop" : "parquet-scan",
+    "@id" : 1,
+    "entries" : [ {
+      "path" : "/tmp/nation.parquet"
     } ],
-    storageengine: {type: "parquet", dfsName: "file:///" },
-    ref : "_MAP",
-    fragmentPointer : 0
+    "storageengine" : {
+      "type" : "parquet",
+      "dfsName" : "file:///"
+    },
+    "ref" : "_MAP",
+    "fragmentPointer" : 0
   }, {
-    pop : "project",
-    @id : 2,
-    exprs : [ {
-      ref : "REGIONKEY",
-      expr : "_MAP.N_REGIONKEY"
+    "pop" : "project",
+    "@id" : 2,
+    "exprs" : [ {
+      "ref" : "REGIONKEY",
+      "expr" : "_MAP.N_REGIONKEY"
     }, {
-      ref : "NAME",
-      expr : "_MAP.N_NAME"
+      "ref" : "NAME",
+      "expr" : "_MAP.N_NAME"
     } ],
-    child : 1
+    "child" : 1
   }, {
-    pop : "sort",
-    @id : 3,
-    child : 2,
-    orderings : [ {
-      order : "ASC",
-      expr : "NAME"
+    "pop" : "sort",
+    "@id" : 3,
+    "child" : 2,
+    "orderings" : [ {
+      "order" : "ASC",
+      "expr" : "NAME"
     } ],
-    reverse : false
+    "reverse" : false
   }, {
-    pop : "selection-vector-remover",
-    @id : 4,
-    child : 3
+    "pop" : "selection-vector-remover",
+    "@id" : 4,
+    "child" : 3
   }, {
-    pop : "screen",
-    @id : 5,
-    child : 4
+    "pop" : "screen",
+    "@id" : 5,
+    "child" : 4
   } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/record/vector/test_all_date_literals.json b/exec/java-exec/src/test/resources/record/vector/test_all_date_literals.json
index 657b016..e5bd788 100644
--- a/exec/java-exec/src/test/resources/record/vector/test_all_date_literals.json
+++ b/exec/java-exec/src/test/resources/record/vector/test_all_date_literals.json
@@ -8,29 +8,38 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [
-              {ref: "Date", expr: "cast(datetype(2008, 2, 27) as varchar(100))"},
-              {ref: "TimeStamp", expr: "cast(timestamptype(2008, 2, 27, 1, 2, 3, 0) as varchar(100))"},
-              {ref: "Time", expr: "cast(timetype(10,11,13, 999) as varchar(100))"},
-              {ref: "Interval", expr: "cast(intervaltype(2, 2, 3, 0, 1, 3, 89) as varchar(100))"}
-            ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled": true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "Date",
+      "expr" : "cast(datetype(2008, 2, 27) as varchar(100))"
+    }, {
+      "ref" : "TimeStamp",
+      "expr" : "cast(timestamptype(2008, 2, 27, 1, 2, 3, 0) as varchar(100))"
+    }, {
+      "ref" : "Time",
+      "expr" : "cast(timetype(10,11,13, 999) as varchar(100))"
+    }, {
+      "ref" : "Interval",
+      "expr" : "cast(intervaltype(2, 2, 3, 0, 1, 3, 89) as varchar(100))"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/record/vector/test_date.json b/exec/java-exec/src/test/resources/record/vector/test_date.json
index f07dcb3..cbdfab0 100644
--- a/exec/java-exec/src/test/resources/record/vector/test_date.json
+++ b/exec/java-exec/src/test/resources/record/vector/test_date.json
@@ -8,27 +8,29 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [ {
-              ref: "Datecast",
-              expr: "cast((cast(stringdate as date)) as varchar(100))"
-            } ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled" : true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "Datecast",
+      "expr" : "cast((cast(stringdate as date)) as varchar(100))"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/record/vector/test_date_add.json b/exec/java-exec/src/test/resources/record/vector/test_date_add.json
index fcef323..d0efd13 100644
--- a/exec/java-exec/src/test/resources/record/vector/test_date_add.json
+++ b/exec/java-exec/src/test/resources/record/vector/test_date_add.json
@@ -8,27 +8,32 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [
-              {ref: "DateIntervalAdd", expr: "cast(date_add(datetype(2008, 2, 27), intervaltype(0, 1, 0, 0, 0, 0, 0)) as varchar(100))"},
-              {ref: "DateIntervalAdd1", expr: "cast((date_add((cast('2008-2-27' as date)), (cast('P1M' as interval)))) as varchar(100))"}
-            ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled" : true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "DateIntervalAdd",
+      "expr" : "cast(date_add(datetype(2008, 2, 27), intervaltype(0, 1, 0, 0, 0, 0, 0)) as varchar(100))"
+    }, {
+      "ref" : "DateIntervalAdd1",
+      "expr" : "cast((date_add((cast('2008-2-27' as date)), (cast('P1M' as interval)))) as varchar(100))"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/record/vector/test_interval.json b/exec/java-exec/src/test/resources/record/vector/test_interval.json
index 1b45269..332d0d0 100644
--- a/exec/java-exec/src/test/resources/record/vector/test_interval.json
+++ b/exec/java-exec/src/test/resources/record/vector/test_interval.json
@@ -8,35 +8,35 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [ {
-              ref: "IntervalCast",
-              expr: "cast((cast(stringinterval as interval)) as varchar(100))"
-            },
-            {
-              ref: "IntervalYearCast",
-              expr: "cast((cast(stringinterval as intervalyear)) as varchar(100))"
-            },
-            {
-              ref: "IntervalDayCast",
-              expr: "cast((cast(stringinterval as intervalday)) as varchar(100))"
-            } ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled" : true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "IntervalCast",
+      "expr" : "cast((cast(stringinterval as interval)) as varchar(100))"
+    }, {
+      "ref" : "IntervalYearCast",
+      "expr" : "cast((cast(stringinterval as intervalyear)) as varchar(100))"
+    }, {
+      "ref" : "IntervalDayCast",
+      "expr" : "cast((cast(stringinterval as intervalday)) as varchar(100))"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/record/vector/test_sort_date.json b/exec/java-exec/src/test/resources/record/vector/test_sort_date.json
index e6b049b..e611709 100644
--- a/exec/java-exec/src/test/resources/record/vector/test_sort_date.json
+++ b/exec/java-exec/src/test/resources/record/vector/test_sort_date.json
@@ -8,49 +8,48 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [ {
-              ref: "Datecast",
-              expr: "(cast(stringdate as date))"
-            } ]
-        },
-        {
-            @id:3,
-            child: 2,
-            pop:"external-sort",
-            orderings: [
-              {expr: "Datecast"}
-            ]
-        },
-        {
-            @id:4,
-            child: 3,
-            pop:"selection-vector-remover"
-        },
-        {
-            pop:"project",
-            @id:5,
-            child: 4,
-            exprs: [ {
-              ref: "SortedDateStr",
-              expr: "(cast(Datecast as varchar(100)))"
-            } ]
-        },
-        {
-            @id: 6,
-            child: 5,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled" : true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "Datecast",
+      "expr" : "(cast(stringdate as date))"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "external-sort",
+    "orderings" : [ {
+      "expr" : "Datecast"
+    } ]
+  }, {
+    "@id" : 4,
+    "child" : 3,
+    "pop" : "selection-vector-remover"
+  }, {
+    "pop" : "project",
+    "@id" : 5,
+    "child" : 4,
+    "exprs" : [ {
+      "ref" : "SortedDateStr",
+      "expr" : "(cast(Datecast as varchar(100)))"
+    } ]
+  }, {
+    "@id" : 6,
+    "child" : 5,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/record/vector/test_timestamp.json b/exec/java-exec/src/test/resources/record/vector/test_timestamp.json
index 219fcfa..e64318a 100644
--- a/exec/java-exec/src/test/resources/record/vector/test_timestamp.json
+++ b/exec/java-exec/src/test/resources/record/vector/test_timestamp.json
@@ -8,27 +8,29 @@
     "type" : "APACHE_DRILL_PHYSICAL",
     "resultMode" : "EXEC"
   },
-  graph:[
-  {
-      @id:1,
-      pop:"fs-scan",
-      format: {type: "json"},
-      storage:{type: "file", connection: "classpath:///"},
-      files:["#{TEST_FILE}"]
-  },
-        {
-            pop:"project",
-            @id:2,
-            child: 1,
-            exprs: [ {
-              ref: "Datecast",
-              expr: "cast((cast(stringdate as timestamp)) as varchar(100))"
-            } ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "classpath:///",
+      "enabled" : true
+    },
+    "files" : [ "#{TEST_FILE}" ]
+  }, {
+    "pop" : "project",
+    "@id" : 2,
+    "child" : 1,
+    "exprs" : [ {
+      "ref" : "Datecast",
+      "expr" : "cast((cast(stringdate as timestamp)) as varchar(100))"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/scan_screen_logical.json b/exec/java-exec/src/test/resources/scan_screen_logical.json
index 005251b..1cf3800 100644
--- a/exec/java-exec/src/test/resources/scan_screen_logical.json
+++ b/exec/java-exec/src/test/resources/scan_screen_logical.json
@@ -1,47 +1,38 @@
 {
-  head:{
-    type:"APACHE_DRILL_LOGICAL",
-    version:"1",
-    generator:{
-      type:"manual",
-      info:"na"
+  "head" : {
+    "type" : "APACHE_DRILL_LOGICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual",
+      "info" : "na"
     }
   },
   "storage" : {
-  "mock" : {
-      "type" : "mock"
-  }
-},
-
-  query:[
-    {
-      @id:"1",
-      op:"scan",
-      memo:"initial_scan",
-      storageengine:"mock",
-      selection: [
-       {
-            records : 100,
-            types : [
-
-                    {
-                        name : "superhero_name",
-                        type : "VARCHAR",
-                        mode : "REQUIRED"
-
-                    }
-             ]
-         }
-      ]
-    },
-    {
-      @id:"2",
-      input: 1,
-      op: "store",
-      memo: "output sink",
-      target: {
-        file: "console:///stdout"
-      }
+    "mock" : {
+      "type" : "mock",
+      "enabled" : true
     }
-  ]
-}
+  },
+  "query" : [ {
+    "@id" : "1",
+    "op" : "scan",
+    "memo" : "initial_scan",
+    "storageengine" : "mock",
+    "selection" : [ {
+      "records" : 100,
+      "types" : [ {
+        "name" : "superhero_name",
+        "type" : "VARCHAR",
+        "mode" : "REQUIRED"
+      } ]
+    } ]
+  }, {
+    "@id" : "2",
+    "input" : 1,
+    "op" : "store",
+    "memo" : "output sink",
+    "target" : {
+      "file" : "console:///stdout"
+    }
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/sender/broadcast_exchange.json b/exec/java-exec/src/test/resources/sender/broadcast_exchange.json
index 1288736..15ce027 100644
--- a/exec/java-exec/src/test/resources/sender/broadcast_exchange.json
+++ b/exec/java-exec/src/test/resources/sender/broadcast_exchange.json
@@ -1,43 +1,53 @@
 {
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
+    }
+  },
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
     },
-    graph:[
-           {
-              @id:1,
-              pop:"fs-scan",
-              format: {type: "json"},
-              storage:{type: "file", connection: "file:///"},
-              files:["#{LEFT_FILE}"]
-            },
-            {
-              @id:2,
-              pop:"fs-scan",
-              format: {type: "json"},
-              storage:{type: "file", connection: "file:///"},
-              files:["#{RIGHT_FILE}"]
-            },
-        {
-            @id: 3,
-            child: 1,
-            pop: "broadcast-exchange"
-        },
-        {
-            pop : "merge-join",
-            @id : 4,
-            left: 3,
-            right: 2,
-            joinType: "LEFT",
-            conditions: [ { relationship: "==", left: "a", right: "aa" } ]
-          },
-        {
-            @id: 5,
-            child: 4,
-            pop: "screen"
-        }
-    ]
-}
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "enabled" : true
+    },
+    "files" : [ "#{LEFT_FILE}" ]
+  }, {
+    "@id" : 2,
+    "pop" : "fs-scan",
+    "format" : {
+      "type" : "json"
+    },
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "enabled" : true
+    },
+    "files" : [ "#{RIGHT_FILE}" ]
+  }, {
+    "@id" : 3,
+    "child" : 1,
+    "pop" : "broadcast-exchange"
+  }, {
+    "pop" : "merge-join",
+    "@id" : 4,
+    "left" : 3,
+    "right" : 2,
+    "joinType" : "LEFT",
+    "conditions" : [ {
+      "relationship" : "==",
+      "left" : "a",
+      "right" : "aa"
+    } ]
+  }, {
+    "@id" : 5,
+    "child" : 4,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/sender/hash_exchange2.json b/exec/java-exec/src/test/resources/sender/hash_exchange2.json
index 844dea7..bbc2202 100644
--- a/exec/java-exec/src/test/resources/sender/hash_exchange2.json
+++ b/exec/java-exec/src/test/resources/sender/hash_exchange2.json
@@ -1,47 +1,43 @@
 {
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
+    }
+  },
+  "graph" : [ {
+    "pop" : "parquet-scan",
+    "@id" : 1,
+    "entries" : [ {
+      "path" : "/Users/sphillips/tpc-h/supplier"
+    } ],
+    "storageengine" : {
+      "type" : "parquet",
+      "dfsName" : "file:///"
     },
-    graph:[
-    {pop : "parquet-scan",
-        @id : 1,
-        entries : [ {
-          path : "/Users/sphillips/tpc-h/supplier"
-        } ],
-        storageengine : {
-          type : "parquet",
-          dfsName : "file:///"
-        },
-        ref : "_MAP",
-        fragmentPointer : 0
-      },
-      {
-                  @id:2,
-                  child: 1,
-                  pop:"project",
-                  exprs: [
-                    { ref: "suppkey", expr:"_MAP.S_SUPPKEY"}
-                  ]
-              },
-         {
-            @id: 3,
-            child: 2,
-            pop: "hash-to-random-exchange",
-            expr: "hash(suppkey)"
-        },
-        {
-            @id: 4,
-            child: 3,
-            pop: "union-exchange"
-        },
-        {
-            @id: 5,
-            child: 4,
-            pop: "screen"
-        }
-    ]
+    "ref" : "_MAP",
+    "fragmentPointer" : 0
+  }, {
+    "@id" : 2,
+    "child" : 1,
+    "pop" : "project",
+    "exprs" : [ {
+      "ref" : "suppkey",
+      "expr" : "_MAP.S_SUPPKEY"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "hash-to-random-exchange",
+    "expr" : "hash(suppkey)"
+  }, {
+    "@id" : 4,
+    "child" : 3,
+    "pop" : "union-exchange"
+  }, {
+    "@id" : 5,
+    "child" : 4,
+    "pop" : "screen"
+  } ]
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/store/json/project_pushdown_json_physical_plan.json b/exec/java-exec/src/test/resources/store/json/project_pushdown_json_physical_plan.json
index b665207..cd22994 100644
--- a/exec/java-exec/src/test/resources/store/json/project_pushdown_json_physical_plan.json
+++ b/exec/java-exec/src/test/resources/store/json/project_pushdown_json_physical_plan.json
@@ -36,7 +36,7 @@
     "format" : {
       "type" : "json"
     },
-    "columns" : [ "`field_1`", "`field_3`.`inner_1`", "`field_3`.`inner_2`", "`field_4`.`inner_1`", "`non_existent_at_root`", "`non_existent`.`nested`.`field`"],
+    "columns" : [ "`field_1`", "`field_3`.`inner_1`", "`field_3`.`inner_2`", "`field_4`.`inner_1`", "`non_existent_at_root`", "`non_existent`.`nested`.`field`" ],
     "selectionRoot" : "/store/json/schema_change_int_to_string.json",
     "cost" : {
       "memoryCost" : 0,
@@ -53,4 +53,4 @@
       "outputRowCount" : 1.0
     }
   } ]
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/store/text/test.json b/exec/java-exec/src/test/resources/store/text/test.json
index 78ac516..3c41b69 100644
--- a/exec/java-exec/src/test/resources/store/text/test.json
+++ b/exec/java-exec/src/test/resources/store/text/test.json
@@ -1,40 +1,37 @@
 {
-    head:{
-        type:"APACHE_DRILL_PHYSICAL",
-        version:"1",
-        generator:{
-            type:"manual"
-        }
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
+    }
+  },
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "fs-scan",
+    "files" : [ "#{DATA_FILE}" ],
+    "storage" : {
+      "type" : "named",
+      "name" : "dfs"
     },
-    graph:[
-        {
-            @id:1,
-            pop:"fs-scan",
-            files:[
-              "#{DATA_FILE}"
-            ],
-            storage : {
-              type : "named",
-              name: "dfs"
-            },
-            format: {
-              type: "named",
-              name: "csv"
-            }
-        },
-        {
-            @id:2,
-            child: 1,
-            pop:"project",
-            exprs: [
-              { ref: "col1", expr:"columns[0]" },
-              { ref: "col2", expr:"columns[1]" }
-            ]
-        },
-        {
-            @id: 3,
-            child: 2,
-            pop: "screen"
-        }
-    ]
-}
+    "format" : {
+      "type" : "named",
+      "name" : "csv"
+    }
+  }, {
+    "@id" : 2,
+    "child" : 1,
+    "pop" : "project",
+    "exprs" : [ {
+      "ref" : "col1",
+      "expr" : "columns[0]"
+    }, {
+      "ref" : "col2",
+      "expr" : "columns[1]"
+    } ]
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/writer/simple_csv_writer.json b/exec/java-exec/src/test/resources/writer/simple_csv_writer.json
index b3c215a..19b6a78 100644
--- a/exec/java-exec/src/test/resources/writer/simple_csv_writer.json
+++ b/exec/java-exec/src/test/resources/writer/simple_csv_writer.json
@@ -1,79 +1,97 @@
 {
-  head:{
-    type:"APACHE_DRILL_PHYSICAL",
-    version:"1",
-    generator:{
-      type:"manual"
+  "head" : {
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "version" : "1",
+    "generator" : {
+      "type" : "manual"
     }
   },
-  graph:[
-    {
-       @id:1,
-       pop:"mock-scan",
-       url: "http://apache.org",
-       entries:[
-         {records: 66000, types: [
-           {name: "red", type: "INT", mode: "REQUIRED"},
-           {name: "green", type: "BIGINT", mode: "OPTIONAL"},
-           {name: "blue", type: "VARCHAR", mode: "REQUIRED"}
-         ]},
-         {records: 66000, types: [
-           {name: "blue", type: "BIT", mode: "REQUIRED"},
-           {name: "green", type: "DECIMAL18", mode: "REQUIRED"},
-           {name: "red", type: "FLOAT8", mode: "OPTIONAL"}
-         ]}
-       ]
+  "graph" : [ {
+    "@id" : 1,
+    "pop" : "mock-scan",
+    "url" : "http://apache.org",
+    "entries" : [ {
+      "records" : 66000,
+      "types" : [ {
+        "name" : "red",
+        "type" : "INT",
+        "mode" : "REQUIRED"
+      }, {
+        "name" : "green",
+        "type" : "BIGINT",
+        "mode" : "OPTIONAL"
+      }, {
+        "name" : "blue",
+        "type" : "VARCHAR",
+        "mode" : "REQUIRED"
+      } ]
     }, {
-      @id: 2,
-      child: 1,
-      pop: "fs-writer",
-      "location" : "%TEST_DIR%",
-      "storage" : {
-        "type" : "file",
-        "connection" : "file:///",
-        "workspaces" : {
-          "root" : {
-            "location" : "%ROOT_DIR%",
-            "writable" : false
-          },
-          "tmp" : {
-            "location" : "%TMP_DIR%",
-            "writable" : true
-          }
+      "records" : 66000,
+      "types" : [ {
+        "name" : "blue",
+        "type" : "BIT",
+        "mode" : "REQUIRED"
+      }, {
+        "name" : "green",
+        "type" : "DECIMAL18",
+        "mode" : "REQUIRED"
+      }, {
+        "name" : "red",
+        "type" : "FLOAT8",
+        "mode" : "OPTIONAL"
+      } ]
+    } ]
+  }, {
+    "@id" : 2,
+    "child" : 1,
+    "pop" : "fs-writer",
+    "location" : "%TEST_DIR%",
+    "storage" : {
+      "type" : "file",
+      "connection" : "file:///",
+      "enabled" : true,
+      "workspaces" : {
+        "root" : {
+          "location" : "%ROOT_DIR%",
+          "writable" : false
         },
-        "formats" : {
-          "psv" : {
-            "type" : "text",
-            "extensions" : [ "tbl" ],
-            "delimiter" : "|"
-          },
-          "csv" : {
-            "type" : "text",
-            "extensions" : [ "csv" ],
-            "delimiter" : ","
-          },
-          "tsv" : {
-            "type" : "text",
-            "extensions" : [ "tsv" ],
-            "delimiter" : "\t"
-          },
-          "parquet" : {
-            "type" : "parquet"
-          },
-          "json" : {
-            "type" : "json"
-          }
+        "tmp" : {
+          "location" : "%TMP_DIR%",
+          "writable" : true
         }
       },
-      "format" : {
-        "type" : "text",
-        "extensions" : [ "csv" ],
-        "delimiter" : ","
+      "formats" : {
+        "psv" : {
+          "type" : "text",
+          "extensions" : [ "tbl" ],
+          "delimiter" : "|"
+        },
+        "csv" : {
+          "type" : "text",
+          "extensions" : [ "csv" ],
+          "delimiter" : ","
+        },
+        "tsv" : {
+          "type" : "text",
+          "extensions" : [ "tsv" ],
+          "delimiter" : "\t"
+        },
+        "parquet" : {
+          "type" : "parquet"
+        },
+        "json" : {
+          "type" : "json"
+        }
       }
-    }, {
-      @id: 3,
-      child: 2,
-      pop: "screen"
+    },
+    "format" : {
+      "type" : "text",
+      "extensions" : [ "csv" ],
+      "delimiter" : ","
     }
-  ]
-}
+  }, {
+    "@id" : 3,
+    "child" : 2,
+    "pop" : "screen"
+  } ]
+}
\ No newline at end of file


[drill] 03/05: DRILL-7204: Add proper validation when creating plugin

Posted by sa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sachouche pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 20b58ab54249525a564a41e43cc8b8fe1db9a6bc
Author: Anton Gozhiy <an...@gmail.com>
AuthorDate: Wed May 15 13:05:56 2019 +0300

    DRILL-7204: Add proper validation when creating plugin
    
    - Added validation for an empty plugin name.
    - Added an URL encoding for pluing name, so plugins with special characters can be accessed without issues.
    - Replaced alerts with modal windows.
    - Added a confirmation dialog when disabling a plugin on Update page.
---
 .../drill/exec/server/rest/StorageResources.java   |  4 ++
 .../src/main/resources/rest/confirmationModals.ftl | 54 ++++++++++++++++++++++
 .../src/main/resources/rest/storage/list.ftl       | 21 +++++++--
 .../src/main/resources/rest/storage/update.ftl     | 29 ++++++++----
 4 files changed, 94 insertions(+), 14 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java
index a80a5b1..7bce8f9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StorageResources.java
@@ -204,6 +204,10 @@ public class StorageResources {
   @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
   @Produces(MediaType.APPLICATION_JSON)
   public JsonResult createOrUpdatePlugin(@FormParam("name") String name, @FormParam("config") String storagePluginConfig) {
+    name = name.trim();
+    if (name.isEmpty()) {
+      return message("Error (a storage name cannot be empty)");
+    }
     try {
       mapper.configure(JsonParser.Feature.ALLOW_COMMENTS, true);
       StoragePluginConfig config = mapper.readValue(new StringReader(storagePluginConfig), StoragePluginConfig.class);
diff --git a/exec/java-exec/src/main/resources/rest/confirmationModals.ftl b/exec/java-exec/src/main/resources/rest/confirmationModals.ftl
new file mode 100644
index 0000000..86f59a4
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/confirmationModals.ftl
@@ -0,0 +1,54 @@
+<#--
+
+    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.
+
+-->
+
+<!--
+  Confirmation Modal to use across templates.
+  By default, modal is hidden and expected to be populated and activated by relevant JavaScripts
+-->
+<div class="modal fade" id="confirmationModal" role="dialog" aria-labelledby="configuration">
+  <div class="modal-dialog">
+    <!-- Modal content-->
+    <div class="modal-content">
+      <div class="modal-header">
+        <button type="button" class="close closeX" data-dismiss="modal"><span class="glyphicon glyphicon-remove"></span></button>
+        <h3 class="modal-title" id="confirmation" style="color:orange" align="center"><span class="glyphicon glyphicon-alert">&#xe209;</span><span id="modalHeader" style="font-family:Helvetica Neue,Helvetica,Arial,sans-serif;white-space:pre">  Warning</span></h3>
+      </div>
+      <div class="modal-body" id="modalBody" style="font-size:125%">
+      ~ConfirmationMessage~
+      </div>
+      <div class="modal-footer">
+        <button id="confirmationOk" type="button" class="btn btn-success" data-dismiss="modal" style="width:20%">Confirm</button>
+        <button id="confirmationCancel" type="button" class="btn btn-primary" data-dismiss="modal" style="width:20%">Cancel</button>
+      </div>
+    </div>
+  </div>
+</div>
+
+<script>
+    //Populate the confirmation modal with the right message params and show
+    function showConfirmationDialog(confirmationMessage, okCallback) {
+      modalBody.innerHTML = confirmationMessage;
+      //Show dialog
+      $('#confirmationModal').modal('show');
+      $('#confirmationOk').unbind('click')
+          .click(okCallback);
+      $('#confirmationCancel').focus();
+    }
+</script>
diff --git a/exec/java-exec/src/main/resources/rest/storage/list.ftl b/exec/java-exec/src/main/resources/rest/storage/list.ftl
index c821a9d..f79485c 100644
--- a/exec/java-exec/src/main/resources/rest/storage/list.ftl
+++ b/exec/java-exec/src/main/resources/rest/storage/list.ftl
@@ -32,6 +32,8 @@
   <div class="page-header">
   </div>
 
+  <#include "*/confirmationModals.ftl">
+
   <h4 class="col-xs-6">Plugin Management</h4>
   <table style="margin: 10px" class="table">
     <tbody>
@@ -61,7 +63,7 @@
                 ${plugin.getName()}
               </td>
               <td style="border:none;">
-                <button type="button" class="btn btn-primary" onclick="location.href='/storage/${plugin.getName()}'">
+                <button type="button" class="btn btn-primary" onclick="doUpdate('${plugin.getName()}')">
                   Update
                 </button>
                 <button type="button" class="btn btn-warning" onclick="doEnable('${plugin.getName()}', false)">
@@ -90,7 +92,7 @@
                 ${plugin.getName()}
               </td>
               <td style="border:none;">
-                <button type="button" class="btn btn-primary" onclick="location.href='/storage/${plugin.getName()}'">
+                <button type="button" class="btn btn-primary" onclick="doUpdate('${plugin.getName()}')">
                   Update
                 </button>
                 <button type="button" class="btn btn-success" onclick="doEnable('${plugin.getName()}', true)">
@@ -200,13 +202,22 @@
 
   <script>
     function doEnable(name, flag) {
-      if (flag || confirm(name + ' plugin will be disabled')) {
-        $.get("/storage/" + name + "/enable/" + flag, function() {
+      if (flag) {
+        proceed();
+      } else {
+        showConfirmationDialog('"' + name + '"' + ' plugin will be disabled. Proceed?', proceed);
+      }
+      function proceed() {
+        $.get("/storage/" + encodeURIComponent(name) + "/enable/" + flag, function() {
           location.reload();
         });
       }
     }
 
+    function doUpdate(name) {
+      window.location.href = "/storage/" + encodeURIComponent(name);
+    }
+
     function doCreate() {
       $("#createForm").ajaxForm({
         dataType: 'json',
@@ -271,7 +282,7 @@
           }
           url = '/storage/' + pluginGroup + '/plugins/export/' + format;
         } else {
-          url = '/storage/' + exportInstance + '/export/' + format;
+          url = '/storage/' + encodeURIComponent(exportInstance) + '/export/' + format;
         }
         window.open(url);
       });
diff --git a/exec/java-exec/src/main/resources/rest/storage/update.ftl b/exec/java-exec/src/main/resources/rest/storage/update.ftl
index 3e827f7..d077ea0 100644
--- a/exec/java-exec/src/main/resources/rest/storage/update.ftl
+++ b/exec/java-exec/src/main/resources/rest/storage/update.ftl
@@ -55,6 +55,8 @@
   <div id="message" class="hidden alert alert-info">
   </div>
 
+  <#include "*/confirmationModals.ftl">
+
   <#-- Modal window-->
   <div class="modal fade" id="pluginsModal" tabindex="-1" role="dialog" aria-labelledby="exportPlugin" aria-hidden="true">
     <div class="modal-dialog modal-sm" role="document">
@@ -106,18 +108,27 @@
       textarea.val(editor.getSession().getValue());
     });
 
-    $.get("/storage/${model.getName()}.json", function(data) {
+    $.get("/storage/" + encodeURIComponent("${model.getName()}") + ".json", function(data) {
       $("#config").val(JSON.stringify(data.config, null, 2));
       editor.getSession().setValue( JSON.stringify(data.config, null, 2) );
     });
 
 
     $("#enabled").click(function() {
-      $.get("/storage/${model.getName()}/enable/<#if model.enabled()>false<#else>true</#if>", function(data) {
-        $("#message").removeClass("hidden").text(data.result).alert();
-        setTimeout(function() { location.reload(); }, 800);
-      });
+      const enabled = ${model.enabled()?c};
+      if (enabled) {
+        showConfirmationDialog('"${model.getName()}"' + ' plugin will be disabled. Proceed?', proceed);
+      } else {
+        proceed();
+      }
+      function proceed() {
+        $.get("/storage/" + encodeURIComponent("${model.getName()}") + "/enable/<#if model.enabled()>false<#else>true</#if>", function(data) {
+          $("#message").removeClass("hidden").text(data.result).alert();
+          setTimeout(function() { location.reload(); }, 800);
+        });
+      }
     });
+
     function doUpdate() {
       $("#updateForm").ajaxForm({
         dataType: 'json',
@@ -126,9 +137,9 @@
     }
 
     function deleteFunction() {
-      if (confirm("Are you sure?")) {
-        $.get("/storage/${model.getName()}/delete", serverMessage);
-      }
+      showConfirmationDialog('"${model.getName()}"' + ' plugin will be deleted. Proceed?', function() {
+        $.get("/storage/" + encodeURIComponent("${model.getName()}") + "/delete", serverMessage);
+      });
     }
 
     // Modal window management
@@ -146,7 +157,7 @@
           format = 'conf';
         }
 
-        let url = '/storage/' + exportInstance + '/export/' + format;
+        let url = '/storage/' + encodeURIComponent(exportInstance) + '/export/' + format;
         window.open(url);
       });
     })


[drill] 02/05: DRILL-7206:using primitive int list at right or full join case

Posted by sa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sachouche pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 7629664b1335b77d7a7de2cc26d6241e3db604cb
Author: weijie.tong <we...@alipay.com>
AuthorDate: Wed Apr 24 20:37:40 2019 +0800

    DRILL-7206:using primitive int list at right or full join case
---
 .../org/apache/drill/exec/physical/impl/common/HashPartition.java    | 3 ++-
 .../org/apache/drill/exec/physical/impl/join/HashJoinHelper.java     | 5 +++--
 .../apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java  | 4 ++--
 3 files changed, 7 insertions(+), 5 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
index 275cf16..be3c51e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.common;
 
+import com.carrotsearch.hppc.IntArrayList;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.commons.lang3.tuple.Pair;
@@ -399,7 +400,7 @@ public class HashPartition implements HashJoinMemoryCalculator.PartitionStat {
   public boolean setRecordMatched(int compositeIndex) {
     return hjHelper.setRecordMatched(compositeIndex);
   }
-  public List<Integer> getNextUnmatchedIndex() {
+  public IntArrayList getNextUnmatchedIndex() {
     return hjHelper.getNextUnmatchedIndex();
   }
   //
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinHelper.java
index 37d8dee..982ed7a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinHelper.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.join;
 
+import com.carrotsearch.hppc.IntArrayList;
 import io.netty.buffer.ByteBuf;
 
 import java.util.ArrayList;
@@ -163,8 +164,8 @@ public class HashJoinHelper {
     return info.getLinks().get(recordIdx);
   }
 
-  public List<Integer> getNextUnmatchedIndex() {
-    List<Integer> compositeIndexes = new ArrayList<>();
+  public IntArrayList getNextUnmatchedIndex() {
+    IntArrayList compositeIndexes = new IntArrayList();
 
     for (int i = 0; i < buildInfoList.size(); i++) {
       BuildInfo info = buildInfoList.get(i);
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 2836794..4699b3e 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
@@ -18,8 +18,8 @@
 package org.apache.drill.exec.physical.impl.join;
 
 import java.util.ArrayList;
-import java.util.List;
 
+import com.carrotsearch.hppc.IntArrayList;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.physical.config.HashJoinPOP;
 import org.apache.drill.exec.physical.impl.common.HashPartition;
@@ -72,7 +72,7 @@ public abstract class HashJoinProbeTemplate implements HashJoinProbe {
   private ProbeState probeState = ProbeState.PROBE_PROJECT;
 
   // For outer or right joins, this is a list of unmatched records that needs to be projected
-  private List<Integer> unmatchedBuildIndexes = null;
+  private IntArrayList unmatchedBuildIndexes = null;
 
   private  HashPartition partitions[];