You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ja...@apache.org on 2013/08/02 05:03:06 UTC

[06/13] git commit: Int and Long non-nullable comparison functions

Int and Long non-nullable comparison functions


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

Branch: refs/heads/master
Commit: 787041e3aca71c732dd0047b3b952c03940a9bfb
Parents: 3399184
Author: Steven Phillips <sp...@maprtech.com>
Authored: Wed Jul 31 15:29:16 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 1 19:24:43 2013 -0700

----------------------------------------------------------------------
 .../drill/common/expression/parser/ExprLexer.g  |   3 +-
 .../drill/common/expression/parser/ExprParser.g |   2 +-
 .../common/expression/fn/BooleanFunctions.java  |   3 +-
 .../exec/expr/fn/impl/ComparisonFunctions.java  | 188 ++++++++++
 .../physical/impl/TestComparisonFunctions.java  | 360 +++++++++++++++++++
 .../src/test/resources/functions/intEqual.json  |  35 ++
 .../resources/functions/intGreaterThan.json     |  35 ++
 .../functions/intGreaterThanEqual.json          |  35 ++
 .../test/resources/functions/intLessThan.json   |  35 ++
 .../resources/functions/intLessThanEqual.json   |  35 ++
 .../test/resources/functions/intNotEqual.json   |  35 ++
 .../src/test/resources/functions/longEqual.json |  35 ++
 .../resources/functions/longGreaterThan.json    |  35 ++
 .../functions/longGreaterThanEqual.json         |  35 ++
 .../test/resources/functions/longLessThan.json  |  35 ++
 .../resources/functions/longLessThanEqual.json  |  35 ++
 .../test/resources/functions/longNotEqual.json  |  35 ++
 sandbox/prototype/pom.xml                       |   3 +
 18 files changed, 976 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g b/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
index aa97b31..1500d07 100644
--- a/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
+++ b/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
@@ -39,7 +39,8 @@ When     : 'when';
 Or       : '||';
 And      : '&&';
 Equals   : '==';
-NEquals  : '!=';
+NEquals  : '<>';
+NEquals2 : '!=';
 GTEquals : '>=';
 LTEquals : '<=';
 Caret      : '^';

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g b/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
index 6cf041c..d2b46b8 100644
--- a/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
+++ b/sandbox/prototype/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
@@ -159,7 +159,7 @@ equExpr returns [LogicalExpression e]
 	  $e = registry.createByOp(exprs, p, cmps);
 	}
   :  r1=relExpr { exprs.add($r1.e); p = pos( $r1.start );
-    } ( cmpr= (Equals | NEquals ) r2=relExpr {exprs.add($r2.e); cmps.add($cmpr.text); })*
+    } ( cmpr= (Equals | NEquals | NEquals2 ) r2=relExpr {exprs.add($r2.e); cmps.add($cmpr.text); })*
   ;
 
 relExpr returns [LogicalExpression e]

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
index 13bc051..6b6a95f 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
@@ -36,7 +36,8 @@ public class BooleanFunctions implements CallProvider {
         FunctionDefinition.operator("and", new AllowedTypeList(2, Integer.MAX_VALUE, Types.REQUIRED_BIT), OutputTypeDeterminer.FIXED_BIT, "and", "&&"),
         FunctionDefinition.operator("greater than", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, ">"),
         FunctionDefinition.operator("less than", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, "<"),
-        FunctionDefinition.operator("equal", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, "==", "<>"),
+        FunctionDefinition.operator("equal", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, "=="),
+        FunctionDefinition.operator("not equal", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, "<>", "!="),
         FunctionDefinition.operator("greater than or equal to", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, ">="),
         FunctionDefinition.operator("less than or equal to", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, "<="), };
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ComparisonFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ComparisonFunctions.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ComparisonFunctions.java
new file mode 100644
index 0000000..0906c14
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ComparisonFunctions.java
@@ -0,0 +1,188 @@
+package org.apache.drill.exec.expr.fn.impl;
+
+import org.apache.drill.common.expression.ArgumentValidators;
+import org.apache.drill.common.expression.CallProvider;
+import org.apache.drill.common.expression.FunctionDefinition;
+import org.apache.drill.common.expression.OutputTypeDeterminer;
+import org.apache.drill.exec.expr.DrillFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.vector.IntHolder;
+import org.apache.drill.exec.vector.BigIntHolder;
+import org.apache.drill.exec.vector.BitHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+public class ComparisonFunctions {
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MathFunctions.class);
+
+    private ComparisonFunctions() {}
+
+    @FunctionTemplate(name = "equal", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class IntEqual implements DrillFunc {
+
+        @Param IntHolder left;
+        @Param IntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value == right.value) ? 1 : 0;
+        }
+    }
+
+    @FunctionTemplate(name = "equal", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class LongEqual implements DrillFunc {
+
+        @Param BigIntHolder left;
+        @Param BigIntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value == right.value) ? 1 : 0;
+        }
+    }
+
+    @FunctionTemplate(name = "not equal", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class IntNotEqual implements DrillFunc {
+
+        @Param IntHolder left;
+        @Param IntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value != right.value) ? 1 : 0;
+        }
+    }
+
+    @FunctionTemplate(name = "not equal", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class LongNotEqual implements DrillFunc {
+
+        @Param BigIntHolder left;
+        @Param BigIntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value != right.value) ? 1 : 0;
+        }
+    }
+
+    @FunctionTemplate(name = "greater than", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class IntGreaterThan implements DrillFunc {
+
+        @Param IntHolder left;
+        @Param IntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value > right.value) ? 1 : 0;
+        }
+    }
+
+    @FunctionTemplate(name = "greater than", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class LongGreaterThan implements DrillFunc {
+
+        @Param BigIntHolder left;
+        @Param BigIntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value > right.value) ? 1 : 0;
+        }
+    }
+
+    @FunctionTemplate(name = "greater than or equal to", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class IntGreaterThanEqual implements DrillFunc {
+
+        @Param IntHolder left;
+        @Param IntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value >= right.value) ? 1 : 0;
+        }
+    }
+
+    @FunctionTemplate(name = "greater than or equal to", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class LongGreaterThanEqual implements DrillFunc {
+
+        @Param BigIntHolder left;
+        @Param BigIntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value >= right.value) ? 1 : 0;
+        }
+    }
+
+    @FunctionTemplate(name = "less than", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class IntLessThan implements DrillFunc {
+
+        @Param IntHolder left;
+        @Param IntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value < right.value) ? 1 : 0;
+        }
+    }
+
+    @FunctionTemplate(name = "less than", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class LongLessThan implements DrillFunc {
+
+        @Param BigIntHolder left;
+        @Param BigIntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value < right.value) ? 1 : 0;
+        }
+    }
+
+    @FunctionTemplate(name = "less than or equal to", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class IntLessThanEqual implements DrillFunc {
+
+        @Param IntHolder left;
+        @Param IntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value <= right.value) ? 1 : 0;
+        }
+    }
+
+    @FunctionTemplate(name = "less than or equal to", scope = FunctionTemplate.FunctionScope.SIMPLE)
+    public static class LongLessThanEqual implements DrillFunc {
+
+        @Param BigIntHolder left;
+        @Param BigIntHolder right;
+        @Output BitHolder out;
+
+        public void setup(RecordBatch b) {}
+
+        public void eval() {
+            out.value = (left.value <= right.value) ? 1 : 0;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java
new file mode 100644
index 0000000..2e0e106
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestComparisonFunctions.java
@@ -0,0 +1,360 @@
+package org.apache.drill.exec.physical.impl;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import com.yammer.metrics.MetricRegistry;
+import mockit.Injectable;
+import mockit.NonStrictExpectations;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.rpc.user.UserServer;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.junit.After;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestComparisonFunctions {
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestComparisonFunctions.class);
+
+    DrillConfig c = DrillConfig.create();
+
+    @Test
+    public void testIntEqual(@Injectable final DrillbitContext bitContext,
+                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intEqual.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(100, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @Test
+    public void testLongEqual(@Injectable final DrillbitContext bitContext,
+                              @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longEqual.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(100, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @Test
+    public void testIntNotEqual(@Injectable final DrillbitContext bitContext,
+                              @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intNotEqual.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(0, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @Test
+    public void testLongNotEqual(@Injectable final DrillbitContext bitContext,
+                                @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longNotEqual.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(0, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @Test
+    public void testIntGreaterThan(@Injectable final DrillbitContext bitContext,
+                                   @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intGreaterThan.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(0, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @Test
+    public void testLongGreaterThan(@Injectable final DrillbitContext bitContext,
+                                   @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longGreaterThan.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(0, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @Test
+    public void testIntGreaterThanEqual(@Injectable final DrillbitContext bitContext,
+                                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intGreaterThanEqual.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(100, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @Test
+    public void testLongGreaterThanEqual(@Injectable final DrillbitContext bitContext,
+                                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longGreaterThanEqual.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(100, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @Test
+    public void testIntLessThan(@Injectable final DrillbitContext bitContext,
+                                   @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intLessThan.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(0, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @Test
+    public void testLongLessThan(@Injectable final DrillbitContext bitContext,
+                                @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longLessThan.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(0, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @Test
+    public void testIntLessThanEqual(@Injectable final DrillbitContext bitContext,
+                                        @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/intLessThanEqual.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(100, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @Test
+    public void testLongLessThanEqual(@Injectable final DrillbitContext bitContext,
+                                     @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+        new NonStrictExpectations(){{
+            bitContext.getMetrics(); result = new MetricRegistry("test");
+            bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+        }};
+
+        PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+        PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/functions/longLessThanEqual.json"), Charsets.UTF_8));
+        FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+        FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+        SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+        while(exec.next()){
+            assertEquals(100, exec.getSelectionVector2().getCount());
+        }
+
+        if(context.getFailureCause() != null){
+            throw context.getFailureCause();
+        }
+
+        assertTrue(!context.isFailed());
+
+    }
+
+    @After
+    public void tearDown() throws Exception{
+        // pause to get logger to catch up.
+        Thread.sleep(1000);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/intEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intEqual.json
new file mode 100644
index 0000000..88fb2eb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intEqual.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "blue == blue"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json
new file mode 100644
index 0000000..015bf36
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThan.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "blue > blue"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json
new file mode 100644
index 0000000..167d4ba
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intGreaterThanEqual.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "blue >= blue"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThan.json
new file mode 100644
index 0000000..473c08f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThan.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "blue < blue"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json
new file mode 100644
index 0000000..418e5f0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intLessThanEqual.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "blue <= blue"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json
new file mode 100644
index 0000000..f0e8578
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/intNotEqual.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "blue <> blue"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json
new file mode 100644
index 0000000..4b0e2b8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longEqual.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "red == red"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json
new file mode 100644
index 0000000..830c126
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThan.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "red > red"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json
new file mode 100644
index 0000000..56c0dd7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longGreaterThanEqual.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "red >= red"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThan.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThan.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThan.json
new file mode 100644
index 0000000..267a6c0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThan.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "red < red"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json
new file mode 100644
index 0000000..e06d865
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longLessThanEqual.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "red <= red"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/exec/java-exec/src/test/resources/functions/longNotEqual.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/functions/longNotEqual.json b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longNotEqual.json
new file mode 100644
index 0000000..71269a6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/functions/longNotEqual.json
@@ -0,0 +1,35 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "yellow", type: "FLOAT8", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            child: 1,
+            pop:"filter",
+            expr: "red <> red"
+        },
+        {
+            @id: 3,
+            child: 2,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/787041e3/sandbox/prototype/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/pom.xml b/sandbox/prototype/pom.xml
index 4d50916..52b2250 100644
--- a/sandbox/prototype/pom.xml
+++ b/sandbox/prototype/pom.xml
@@ -120,6 +120,9 @@
         <plugin>
           <artifactId>maven-surefire-plugin</artifactId>
           <version>2.15</version>
+          <configuration>
+            <argLine>-XX:MaxDirectMemorySize=4096M</argLine>
+          </configuration>
         </plugin>
         <!--This plugin's configuration is used to store Eclipse m2e settings 
           only. It has no influence on the Maven build itself. -->