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/07/20 03:57:51 UTC

[30/53] [abbrv] git commit: Merge branch 'master' into execwork. Fix conflicts.

Merge branch 'master' into execwork.  Fix conflicts.

Conflicts:
	sandbox/prototype/common/src/test/resources/logback.xml
	sandbox/prototype/pom.xml


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

Branch: refs/heads/master
Commit: 5a5d07f69c17ed4c0f63d17f27cd652830a5d03e
Parents: 1be6bd4 97eb07a
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed Jun 19 16:30:02 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed Jun 19 16:30:02 2013 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/JSONOptions.java    |   4 +
 .../drill/common/logical/data/Constant.java     |  52 +++++++++
 .../apache/drill/common/logical/data/Scan.java  |  14 +--
 .../common/logical/data/SourceOperator.java     |   9 ++
 .../data/visitors/AbstractLogicalVisitor.java   |  32 ++++--
 .../logical/data/visitors/LogicalVisitor.java   |  18 +++
 .../apache/drill/exec/ref/rops/ConstantROP.java | 112 +++++++++++++++++++
 .../drill/exec/ref/rse/JSONRecordReader.java    |   2 +-
 .../drill/exec/ref/rops/ConstantROPTest.java    |  95 ++++++++++++++++
 .../exec/ref/src/test/resources/constant.json   |   4 +
 .../exec/ref/src/test/resources/constant2.json  |  42 +++++++
 sandbox/prototype/pom.xml                       |  46 ++++----
 sandbox/prototype/sqlline                       |  16 +++
 13 files changed, 404 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
index e309c60,0000000..8a185a4
mode 100644,000000..100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/JSONOptions.java
@@@ -1,111 -1,0 +1,115 @@@
 +/*******************************************************************************
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + * 
 + * http://www.apache.org/licenses/LICENSE-2.0
 + * 
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + ******************************************************************************/
 +package org.apache.drill.common;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import com.fasterxml.jackson.core.*;
 +import com.fasterxml.jackson.core.type.TypeReference;
 +import com.fasterxml.jackson.databind.ObjectMapper;
 +import org.apache.drill.common.JSONOptions.De;
 +import org.apache.drill.common.JSONOptions.Se;
 +import org.apache.drill.common.config.DrillConfig;
 +import org.apache.drill.common.exceptions.LogicalPlanParsingException;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.fasterxml.jackson.databind.DeserializationContext;
 +import com.fasterxml.jackson.databind.JsonNode;
 +import com.fasterxml.jackson.databind.SerializerProvider;
 +import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
 +import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 +import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
 +import com.fasterxml.jackson.databind.ser.std.StdSerializer;
 +
 +@JsonSerialize(using = Se.class)
 +@JsonDeserialize(using = De.class)
 +public class JSONOptions {
 +  
 +  final static Logger logger = LoggerFactory.getLogger(JSONOptions.class);
 +  
 +  private JsonNode root;
 +  private JsonLocation location;
 +  
 +  private JSONOptions(JsonNode n, JsonLocation location){
 +    this.root = n;
 +    this.location = location;
 +  }
 +  
 +  public <T> T getWith(DrillConfig config, Class<T> c){
 +    try {
 +      //logger.debug("Read tree {}", root);
 +      return config.getMapper().treeToValue(root, c);
 +    } catch (JsonProcessingException e) {
 +      throw new LogicalPlanParsingException(String.format("Failure while trying to convert late bound json options to type of %s. Reference was originally located at line %d, column %d.", c.getCanonicalName(), location.getLineNr(), location.getColumnNr()), e);
 +    }
 +  }
 +
 +  public <T> T getListWith(DrillConfig config, TypeReference<T> t) throws IOException {
 +      ObjectMapper mapper = config.getMapper();
 +      return mapper.treeAsTokens(root).readValueAs(t);
 +     // return mapper.treeToValue(root,  mapper.getTypeFactory().constructCollectionType(List.class, c));
 +  }
 +  
 +  public JsonNode path(String name){
 +    return root.path(name);
 +  }
++
++  public JsonNode getRoot(){
++      return root;
++  }
 +  
 +  public static class De extends StdDeserializer<JSONOptions> {
 +    
 +    public De() {
 +      super(JSONOptions.class);
 +      logger.debug("Creating Deserializer.");
 +    }
 +
 +    @Override
 +    public JSONOptions deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
 +        JsonProcessingException {
 +      JsonLocation l = jp.getTokenLocation();
 +//      logger.debug("Reading tree.");
 +      TreeNode n = jp.readValueAsTree();
 +//      logger.debug("Tree {}", n);
 +      if(n instanceof JsonNode){
 +        return new JSONOptions( (JsonNode) n, l); 
 +      }else{
 +        throw new IllegalArgumentException(String.format("Received something other than a JsonNode %s", n));
 +      }
 +    }
 +
 +  }
 +
 +  public static class Se extends StdSerializer<JSONOptions> {
 +
 +    public Se() {
 +      super(JSONOptions.class);
 +    }
 +
 +    @Override
 +    public void serialize(JSONOptions value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
 +        JsonGenerationException {
 +      jgen.writeTree(value.root);
 +    }
 +
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Constant.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Constant.java
index 0000000,84d5939..8a0024c
mode 000000,100644..100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Constant.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Constant.java
@@@ -1,0 -1,44 +1,52 @@@
+ /*******************************************************************************
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  ******************************************************************************/
+ package org.apache.drill.common.logical.data;
+ 
 -import com.google.common.base.Preconditions;
 -import org.apache.drill.common.expression.FieldReference;
 -import org.apache.drill.common.logical.JSONOptions;
++import java.util.Iterator;
++
++import org.apache.drill.common.JSONOptions;
++import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+ 
+ import com.fasterxml.jackson.annotation.JsonCreator;
+ import com.fasterxml.jackson.annotation.JsonProperty;
+ import com.fasterxml.jackson.annotation.JsonTypeName;
++import com.google.common.base.Preconditions;
+ 
+ @JsonTypeName("constant")
+ public class Constant extends SourceOperator{
+ 
+     private final JSONOptions content;
+ 
+     @JsonCreator
+     public Constant(@JsonProperty("content") JSONOptions content){
+         super();
+         this.content = content;
+         Preconditions.checkNotNull(content, "content attribute is required for source operator 'constant'.");
+     }
+ 
+     public JSONOptions getContent() {
+         return content;
+     }
+ 
++    @Override
++    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
++      return logicalVisitor.visitConstant(this, value);
++    }
++
++    
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
index 14f44b4,f22a5bc..d56d3f3
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
@@@ -17,16 -17,12 +17,14 @@@
   ******************************************************************************/
  package org.apache.drill.common.logical.data;
  
- import com.google.common.collect.Iterators;
 +import org.apache.drill.common.JSONOptions;
  import org.apache.drill.common.expression.FieldReference;
 -import org.apache.drill.common.logical.JSONOptions;
  
  import com.fasterxml.jackson.annotation.JsonCreator;
  import com.fasterxml.jackson.annotation.JsonProperty;
  import com.fasterxml.jackson.annotation.JsonTypeName;
 +import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
 +
- import java.util.Iterator;
  
  @JsonTypeName("scan")
  public class Scan extends SourceOperator{
@@@ -55,18 -51,7 +53,14 @@@
    public FieldReference getOutputReference() {
      return outputReference;
    }
 -	
 -  
 +
-     @Override
-     public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
-         return logicalVisitor.visitScan(this, value);
-     }
++  @Override
++  public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
++      return logicalVisitor.visitScan(this, value);
++  }
 +
-     @Override
-     public Iterator<LogicalOperator> iterator() {
-         return Iterators.emptyIterator();
-     }
 +
 +
 +
  
  }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/SourceOperator.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/SourceOperator.java
index 73373e9,73373e9..2ad94a3
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/SourceOperator.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/SourceOperator.java
@@@ -17,10 -17,10 +17,19 @@@
   ******************************************************************************/
  package org.apache.drill.common.logical.data;
  
++import java.util.Iterator;
++
++import com.google.common.collect.Iterators;
++
  
  /**
   * An operator that produces data without any parents.  (zero input operator)
   */
  public abstract class SourceOperator extends LogicalOperatorBase{
  
++  @Override
++  public Iterator<LogicalOperator> iterator() {
++      return Iterators.emptyIterator();
++  }
++  
  }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
index a293d0b,0000000..d8a1378
mode 100644,000000..100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
@@@ -1,94 -1,0 +1,112 @@@
++/*******************************************************************************
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ * 
++ * http://www.apache.org/licenses/LICENSE-2.0
++ * 
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ ******************************************************************************/
 +package org.apache.drill.common.logical.data.visitors;
 +
 +import org.apache.drill.common.logical.data.*;
 +
- /**
-  * Created with IntelliJ IDEA.
-  * User: jaltekruse
-  * Date: 6/10/13
-  * Time: 1:55 PM
-  * To change this template use File | Settings | File Templates.
-  */
++
 +public abstract class AbstractLogicalVisitor<T, X, E extends Throwable> implements LogicalVisitor<T, X, E> {
 +
 +    public T visitOp(LogicalOperator op, X value) throws E{
 +        throw new UnsupportedOperationException(String.format(
 +                "The LogicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
 +                .getClass().getCanonicalName(), op.getClass().getCanonicalName()));
 +    }
 +
 +    @Override
 +    public T visitScan(Scan scan, X value) throws E {
 +        return visitOp(scan, value);
 +    }
 +
 +    @Override
 +    public T visitStore(Store store, X value) throws E {
 +        return visitOp(store, value);
 +    }
 +
 +    @Override
 +    public T visitFilter(Filter filter, X value) throws E {
 +        return visitOp(filter, value);
 +    }
 +
 +    @Override
 +    public T visitFlatten(Flatten flatten, X value) throws E {
 +        return visitOp(flatten, value);
 +    }
 +
 +    @Override
 +    public T visitProject(Project project, X value) throws E {
 +        return visitOp(project, value);
 +    }
 +
 +    @Override
 +    public T visitOrder(Order order, X value) throws E {
 +        return visitOp(order, value);
 +    }
 +
 +    @Override
 +    public T visitJoin(Join join, X value) throws E {
 +        return visitOp(join, value);
 +    }
 +
 +    @Override
 +    public T visitLimit(Limit limit, X value) throws E {
 +        return visitOp(limit, value);
 +    }
 +
 +    @Override
 +    public T visitRunningAggregate(RunningAggregate runningAggregate, X value) throws E {
 +        return visitOp(runningAggregate, value);
 +    }
 +
 +    @Override
 +    public T visitSegment(Segment segment, X value) throws E {
 +        return visitOp(segment, value);
 +    }
 +
 +    @Override
 +    public T visitSequence(Sequence sequence, X value) throws E {
 +        return visitOp(sequence, value);
 +    }
 +
 +    @Override
 +    public T visitTransform(Transform transform, X value) throws E {
 +        return visitOp(transform, value);
 +    }
 +
 +    @Override
 +    public T visitUnion(Union union, X value) throws E {
 +        return visitOp(union, value);
 +    }
 +
 +    @Override
 +    public T visitCollapsingAggregate(CollapsingAggregate collapsingAggregate, X value) throws E {
 +        return visitOp(collapsingAggregate, value);
 +    }
 +
 +    @Override
 +    public T visitWindowFrame(WindowFrame windowFrame, X value) throws E {
 +        return visitOp(windowFrame, value);
 +    }
++
++    @Override
++    public T visitConstant(Constant constant, X value) throws E {
++       return visitOp(constant, value);
++    }
++    
++    
 +}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
index 1f08913,0000000..10e6ec2
mode 100644,000000..100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
@@@ -1,35 -1,0 +1,53 @@@
++/*******************************************************************************
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ * 
++ * http://www.apache.org/licenses/LICENSE-2.0
++ * 
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ ******************************************************************************/
 +package org.apache.drill.common.logical.data.visitors;
 +
 +
 +import org.apache.drill.common.graph.GraphVisitor;
 +import org.apache.drill.common.logical.data.*;
 +
 +/**
 + * Visitor class designed to traversal of a operator tree.  Basis for a number of operator manipulations including fragmentation and materialization.
 + * @param <RETURN> The class associated with the return of each visit method.
 + * @param <EXTRA> The class object associated with additional data required for a particular operator modification.
 + * @param <EXCEP> An optional exception class that can be thrown when a portion of a modification or traversal fails.  Must extend Throwable.  In the case where the visitor does not throw any caught exception, this can be set as RuntimeException.
 + */
 +public interface LogicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
 +    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LogicalVisitor.class);
 +
 +
 +    //public RETURN visitExchange(Exchange exchange, EXTRA value) throws EXCEP;
 +    public RETURN visitScan(Scan scan, EXTRA value) throws EXCEP;
 +    public RETURN visitStore(Store store, EXTRA value) throws EXCEP;
 +    public RETURN visitCollapsingAggregate(CollapsingAggregate collapsingAggregate, EXTRA value) throws EXCEP;
 +
 +    public RETURN visitFilter(Filter filter, EXTRA value) throws EXCEP;
 +    public RETURN visitFlatten(Flatten flatten, EXTRA value) throws EXCEP;
 +
 +    public RETURN visitProject(Project project, EXTRA value) throws EXCEP;
++    public RETURN visitConstant(Constant constant, EXTRA value) throws EXCEP;
 +    public RETURN visitOrder(Order order, EXTRA value) throws EXCEP;
 +    public RETURN visitJoin(Join join, EXTRA value) throws EXCEP;
 +    public RETURN visitLimit(Limit limit, EXTRA value) throws EXCEP;
 +    public RETURN visitRunningAggregate(RunningAggregate runningAggregate, EXTRA value) throws EXCEP;
 +    public RETURN visitSegment(Segment segment, EXTRA value) throws EXCEP;
 +    public RETURN visitSequence(Sequence sequence, EXTRA value) throws EXCEP;
 +    public RETURN visitTransform(Transform transform, EXTRA value) throws EXCEP;
 +    public RETURN visitUnion(Union union, EXTRA value) throws EXCEP;
 +    public RETURN visitWindowFrame(WindowFrame windowFrame, EXTRA value) throws EXCEP;
 +}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/ConstantROPTest.java
----------------------------------------------------------------------
diff --cc sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/ConstantROPTest.java
index 0000000,c7d4719..499b335
mode 000000,100644..100644
--- a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/ConstantROPTest.java
+++ b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/ConstantROPTest.java
@@@ -1,0 -1,97 +1,95 @@@
+ package org.apache.drill.exec.ref.rops;
+ 
+ import com.fasterxml.jackson.databind.ObjectMapper;
+ import com.google.common.base.Charsets;
+ import com.google.common.io.Files;
+ import org.apache.drill.common.config.DrillConfig;
+ import org.apache.drill.common.expression.SchemaPath;
+ import org.apache.drill.common.logical.LogicalPlan;
+ import org.apache.drill.common.logical.data.Constant;
+ import org.apache.drill.common.util.FileUtils;
+ import org.apache.drill.exec.ref.*;
+ import org.apache.drill.exec.ref.eval.BasicEvaluatorFactory;
+ import org.apache.drill.exec.ref.rse.RSERegistry;
+ import org.apache.drill.exec.ref.values.ScalarValues;
+ import org.junit.Test;
+ 
+ 
+ import java.io.File;
+ import java.util.Collection;
+ import java.util.Iterator;
+ 
+ /**
+  * Created with IntelliJ IDEA.
+  * User: jaltekruse
+  * Date: 6/4/13
+  * Time: 4:15 PM
+  * To change this template use File | Settings | File Templates.
+  */
+ public class ConstantROPTest {
+ 
+     @Test
+     public void testConstant(){
+ 
+         ObjectMapper map = DrillConfig.create().getMapper();
+         Constant con;
+         try{
+             con = map.readValue( FileUtils.getResourceAsString("/constant.json"), Constant.class);
+             if (con == null){
+                 System.out.println("constant is null");
+             }
+             System.out.println(con);
+ 
+             ConstantROP rop = new ConstantROP(con);
+ 
+             rop.setupIterators(new IteratorRegistry());
+             RecordIterator iter = rop.getIteratorInternal();
+             RecordPointer ptr = iter.getRecordPointer();
+ 
+             int i = 1;
+             while (iter.next() != RecordIterator.NextOutcome.NONE_LEFT){
+                 System.out.println(ptr);
+                 org.junit.Assert.assertEquals("Integer value in record " + i + " is incorrect.",
+                         ptr.getField(new SchemaPath("c1")), new ScalarValues.IntegerScalar(i));
+                 org.junit.Assert.assertEquals("String value in record " + i + " is incorrect.",
+                         ptr.getField(new SchemaPath("c2")), new ScalarValues.StringScalar("string " + i));
+                 i++;
+             }
+             org.junit.Assert.assertEquals("Incorrect number of records returned by 'constant' record iterator.", 3, i - 1);
+         } catch (Exception ex){ ex.printStackTrace(); }
+         System.out.println("end test");
+     }
+ 
+     // not sure if we want to keep this as a test and check the results. Now that the internals of the ConstantROP work
+     // it might now be worth running the reference intepreter with every build
+     @Test
+     public void testRefInterp(){
+ 
+         try{
+             DrillConfig config = DrillConfig.create();
+             final String jsonFile = "/constant2.json";
+             LogicalPlan plan = LogicalPlan.parse(config, FileUtils.getResourceAsString(jsonFile));
+             org.junit.Assert.assertEquals("Constant operator not read in properly or not recognized as a source operator.",
 -                    plan.getGraph().getSources().toString(), "[Constant [memo=null]]");
++                    plan.getGraph().getLeaves().toString(), "[Constant [memo=null]]");
+ 
+             org.junit.Assert.assertEquals("Edge between constant operator and sink not recognized.",
 -                    plan.getGraph().getSinks().toString(), "[Store [memo=output sink]]");
 -
 -            org.junit.Assert.assertEquals("Constant operator not read in properly or not recognized as a sink operator.",
 -                    plan.getGraph().getAdjList().getAllEdges().toString(), "[Edge [from=Node [val=Constant [memo=null]], to=Node [val=Store [memo=output sink]]]]");
++                    plan.getGraph().getRoots().toString(), "[Store [memo=output sink]]");
+ 
++            
+             IteratorRegistry ir = new IteratorRegistry();
+             ReferenceInterpreter i = new ReferenceInterpreter(plan, ir, new BasicEvaluatorFactory(ir), new RSERegistry(config));
+             i.setup();
+             Collection<RunOutcome> outcomes = i.run();
+ 
+             for(RunOutcome outcome : outcomes){
+                 System.out.println("============");
+                 System.out.println(outcome);
+                 if(outcome.outcome == RunOutcome.OutcomeType.FAILED && outcome.exception != null){
+                     outcome.exception.printStackTrace();
+                 }
+             }
+         } catch (Exception e) {
+             e.printStackTrace();  //To change body of catch statement use File | Settings | File Templates.
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5a5d07f6/sandbox/prototype/pom.xml
----------------------------------------------------------------------