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:58:03 UTC

[42/53] [abbrv] Types transition

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
new file mode 100644
index 0000000..bc44490
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -0,0 +1,128 @@
+/*******************************************************************************
+ * 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;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ExpressionValidator;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.IfExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.expression.visitors.SimpleExprVisitor;
+import org.apache.drill.exec.record.NullExpression;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
+
+import com.google.common.collect.Lists;
+
+public class ExpressionTreeMaterializer {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionTreeMaterializer.class);
+  
+  
+  
+  private ExpressionTreeMaterializer() {
+  };
+
+  public static LogicalExpression materialize(LogicalExpression expr, RecordBatch batch, ErrorCollector errorCollector) {
+    return expr.accept(new MaterializeVisitor(batch, errorCollector), null);
+  }
+
+  private static class MaterializeVisitor extends SimpleExprVisitor<LogicalExpression> {
+    private final ErrorCollector errorCollector;
+    private final RecordBatch batch;
+    private ExpressionValidator validator = new ExpressionValidator();
+
+    public MaterializeVisitor(RecordBatch batch, ErrorCollector errorCollector) {
+      this.batch = batch;
+      this.errorCollector = errorCollector;
+    }
+
+    private LogicalExpression validateNewExpr(LogicalExpression newExpr) {
+      newExpr.accept(validator, errorCollector);
+      return newExpr;
+    }
+
+    @Override
+    public LogicalExpression visitUnknown(LogicalExpression e, Void value) throws RuntimeException {
+      throw new UnsupportedOperationException(String.format("Expression tree materializer does not currently support materializing nodes of type %s.", e.getClass().getCanonicalName()));
+    }
+
+    @Override
+    public LogicalExpression visitFunctionCall(FunctionCall call) {
+      List<LogicalExpression> args = Lists.newArrayList();
+      for (int i = 0; i < call.args.size(); ++i) {
+        LogicalExpression newExpr = call.args.get(i).accept(this, null);
+        args.add(newExpr);
+      }
+
+      return validateNewExpr(new FunctionCall(call.getDefinition(), args, call.getPosition()));
+    }
+
+    @Override
+    public LogicalExpression visitIfExpression(IfExpression ifExpr) {
+      List<IfExpression.IfCondition> conditions = Lists.newArrayList(ifExpr.iterator());
+      LogicalExpression newElseExpr = ifExpr.elseExpression.accept(this, null);
+
+      for (int i = 0; i < conditions.size(); ++i) {
+        IfExpression.IfCondition condition = conditions.get(i);
+
+        LogicalExpression newCondition = condition.condition.accept(this, null);
+        LogicalExpression newExpr = condition.expression.accept(this, null);
+        conditions.set(i, new IfExpression.IfCondition(newCondition, newExpr));
+      }
+
+      return validateNewExpr(IfExpression.newBuilder().setElse(newElseExpr).addConditions(conditions).build());
+    }
+
+    @Override
+    public LogicalExpression visitSchemaPath(SchemaPath path) {
+      logger.debug("Visiting schema path {}", path);
+      TypedFieldId tfId = batch.getValueVector(path);
+      if (tfId == null) {
+        return NullExpression.INSTANCE;
+      } else {
+        return new ValueVectorReadExpression(tfId);
+      }
+    }
+
+    @Override
+    public LogicalExpression visitLongConstant(ValueExpressions.LongExpression intExpr) {
+      return intExpr;
+    }
+
+    @Override
+    public LogicalExpression visitDoubleConstant(ValueExpressions.DoubleExpression dExpr) {
+      return dExpr;
+    }
+
+    @Override
+    public LogicalExpression visitBooleanConstant(ValueExpressions.BooleanExpression e) {
+      return e;
+    }
+
+    @Override
+    public LogicalExpression visitQuotedStringConstant(ValueExpressions.QuotedString e) {
+      return e;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
new file mode 100644
index 0000000..813a32f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
@@ -0,0 +1,40 @@
+package org.apache.drill.exec.expr;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.StringWriter;
+import java.io.Writer;
+
+import com.google.common.base.Preconditions;
+import com.sun.codemodel.CodeWriter;
+import com.sun.codemodel.JPackage;
+
+public class SingleClassStringWriter extends CodeWriter{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleClassStringWriter.class);
+
+  private boolean used;
+  private StringWriter writer = new StringWriter();
+  
+  @Override
+  public OutputStream openBinary(JPackage pkg, String fileName) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  
+  @Override
+  public Writer openSource(JPackage pkg, String fileName) throws IOException {
+    Preconditions.checkArgument(!used, "The SingleClassStringWriter can only output once src file.");
+    used = true;
+    return writer;
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  public StringBuffer getCode(){
+    return writer.getBuffer();
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
new file mode 100644
index 0000000..c8d906d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
@@ -0,0 +1,41 @@
+package org.apache.drill.exec.expr;
+
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class ValueVectorReadExpression implements LogicalExpression{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVectorReadExpression.class);
+
+  private final MajorType type;
+  private final int fieldId;
+  
+  public ValueVectorReadExpression(TypedFieldId tfId) {
+    this.type = tfId.getType();
+    this.fieldId = tfId.getFieldId();
+  }
+
+  @Override
+  public MajorType getMajorType() {
+    return type;
+  }
+
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    return visitor.visitUnknown(this, value);
+  }
+
+  public int getFieldId() {
+    return fieldId;
+  }
+
+  @Override
+  public ExpressionPosition getPosition() {
+    return ExpressionPosition.UNKNOWN;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
new file mode 100644
index 0000000..96d9d05
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
@@ -0,0 +1,45 @@
+package org.apache.drill.exec.expr;
+
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class ValueVectorWriteExpression implements LogicalExpression {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVectorWriteExpression.class);
+
+  private final int fieldId;
+  private final LogicalExpression child;
+  
+  public ValueVectorWriteExpression(int fieldId, LogicalExpression child){
+    this.fieldId = fieldId;
+    this.child = child;
+  }
+  
+  public int getFieldId() {
+    return fieldId;
+  }
+
+  @Override
+  public MajorType getMajorType() {
+    return Types.NULL;
+  }
+
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    return visitor.visitUnknown(this, value);
+  }
+
+  @Override
+  public ExpressionPosition getPosition() {
+    return ExpressionPosition.UNKNOWN;
+  }
+
+  public LogicalExpression getChild() {
+    return child;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
new file mode 100644
index 0000000..9e32750
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
@@ -0,0 +1,25 @@
+package org.apache.drill.exec.expr.annotations;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE})
+public @interface FunctionTemplate {
+  
+  String name();
+  FunctionScope scope();
+  NullHandling nulls() default NullHandling.INTERNAL;
+  boolean isBinaryCommutative() default false;
+  
+  public static enum NullHandling {
+    INTERNAL, NULL_IF_NULL;
+  }
+  
+  public static enum FunctionScope{
+    SIMPLE, AGGREGATE, RUNNING;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
new file mode 100644
index 0000000..2d77dfc
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
@@ -0,0 +1,18 @@
+package org.apache.drill.exec.expr.annotations;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Describes the field will provide output from the given function.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.FIELD})
+public @interface Output {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Output.class);
+
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
new file mode 100644
index 0000000..9ba7611
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
@@ -0,0 +1,18 @@
+package org.apache.drill.exec.expr.annotations;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+/**
+ * Marker annotation to determine which fields should be included as parameters for the function.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.FIELD})
+public @interface Param {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Param.class);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
new file mode 100644
index 0000000..cc106a7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
@@ -0,0 +1,14 @@
+package org.apache.drill.exec.expr.annotations;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Describes the field will provide output from the given function.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.FIELD})
+public @interface WorkSpace {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
new file mode 100644
index 0000000..7276e7d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
@@ -0,0 +1,186 @@
+package org.apache.drill.exec.expr.fn;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.lang.reflect.Field;
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+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.expr.annotations.WorkSpace;
+import org.apache.drill.exec.expr.fn.FunctionHolder.ValueReference;
+import org.apache.drill.exec.expr.holders.ValueHolder;
+import org.codehaus.commons.compiler.CompileException;
+import org.codehaus.janino.Java;
+import org.codehaus.janino.Java.CompilationUnit;
+import org.codehaus.janino.Parser;
+import org.codehaus.janino.Scanner;
+import org.mortbay.util.IO;
+
+import com.beust.jcommander.internal.Lists;
+import com.google.common.io.InputSupplier;
+import com.google.common.io.Resources;
+
+/**
+ * Converts FunctionCalls to Java Expressions.
+ */
+public class FunctionConverter {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FunctionConverter.class);
+  
+  
+  public <T extends DrillFunc> FunctionHolder getHolder(Class<T> clazz){
+    FunctionTemplate template = clazz.getAnnotation(FunctionTemplate.class);
+    if(template == null){
+      return failure("Class does not declare FunctionTemplate annotation.", clazz);
+    }
+    
+    // start by getting field information.
+    List<ValueReference> params = Lists.newArrayList();
+    List<String> workspaceFields = Lists.newArrayList();
+    ValueReference outputField = null;
+    
+    
+    for(Field field : clazz.getDeclaredFields()){
+
+      Param param = field.getAnnotation(Param.class);
+      Output output = field.getAnnotation(Output.class);
+      WorkSpace workspace = field.getAnnotation(WorkSpace.class);
+      
+      int i =0;
+      if(param != null) i++;
+      if(output != null) i++;
+      if(workspace != null) i++;
+      if(i == 0){
+        return failure("The field must be either a @Param, @Output or @WorkSpace field.", clazz, field);
+      }else if(i > 1){
+        return failure("The field must be only one of @Param, @Output or @WorkSpace.  It currently has more than one of these annotations.", clazz, field);
+      }
+
+      
+      
+      if(param != null || output != null){
+        
+        // check that param and output are value holders.
+        if(!ValueHolder.class.isAssignableFrom(field.getType())){
+          return failure(String.format("The field doesn't holds value of type %s which does not implement the ValueHolder interface.  All fields of type @Param or @Output must extend this interface..", field.getType()), clazz, field);
+        }
+        
+        // get the type field from the value holder.
+        MajorType type = null;
+        try{
+          type = getStaticFieldValue("TYPE", field.getType(), MajorType.class);
+        }catch(Exception e){
+          return failure("Failure while trying to access the ValueHolder's TYPE static variable.  All ValueHolders must contain a static TYPE variable that defines their MajorType.", e, clazz, field.getName());
+        }
+        
+        
+        ValueReference p = new ValueReference(type, field.getName());
+        if(param != null){
+          params.add(p);
+        }else{ 
+          if(outputField != null){
+            return failure("You've declared more than one @Output field.  You must declare one and only @Output field per Function class.", clazz, field);
+          }else{
+            outputField = p; 
+            
+          }
+           
+        }
+        
+      }else{
+        // workspace work.
+        workspaceFields.add(field.getName());
+      }
+      
+    }
+    
+    
+    if(!workspaceFields.isEmpty()) return failure("This function declares one or more workspace fields.  However, those have not yet been implemented.", clazz);
+    if(outputField == null)  return failure("This function declares zero output fields.  A function must declare one output field.", clazz);
+    
+    // get function body.     
+   
+    CompilationUnit cu;
+    try {
+      cu = getClassBody(clazz);
+    } catch (CompileException | IOException e) {
+      return failure("Failure while getting class body.", e, clazz);
+    }
+    
+    Map<String, String> methods = MethodGrabbingVisitor.getMethods(cu, clazz);
+
+    // return holder
+    ValueReference[] ps = params.toArray(new ValueReference[params.size()]);
+    FunctionHolder fh = new FunctionHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.name(), ps, outputField, methods);
+    return fh;
+  }
+  
+  
+  
+  private Java.CompilationUnit getClassBody(Class<?> c) throws CompileException, IOException{
+    String path = c.getName();
+    path = path.replaceFirst("\\$.*", "");
+    path = path.replace(".", File.separator);
+    path = "/" + path + ".java";
+    URL u = Resources.getResource(FunctionConverter.class, path);
+    InputSupplier<InputStream> supplier = Resources.newInputStreamSupplier(u);
+    try(InputStream is = supplier.getInput()){
+      if(is == null){
+        throw new IOException(String.format("Failure trying to located source code for Class %s, tried to read on classpath location %s", c.getName(), path));
+      }
+      String body = IO.toString(is);
+      
+      //TODO: Hack to remove annotations so Janino doesn't choke.  Need to reconsider this problem...
+      body = body.replaceAll("@(?:Output|Param|FunctionTemplate\\([^\\\\]*?\\))", "");
+      return new Parser(new Scanner(null, new StringReader(body))).parseCompilationUnit();
+    }
+    
+  }
+  
+  @SuppressWarnings("unchecked")
+  private <T> T getStaticFieldValue(String fieldName, Class<?> valueType, Class<T> c) throws NoSuchFieldException, SecurityException, IllegalArgumentException, IllegalAccessException{
+      Field f = valueType.getDeclaredField(fieldName);
+      Object val = f.get(null);
+      return (T) val;
+  }
+  
+  private static FunctionHolder failure(String message, Throwable t, Class<?> clazz, String fieldName){
+    logger.warn("Failure loading function class {}, field {}. " + message, clazz.getName(), fieldName, t);
+    return null;
+  }  
+  
+  private FunctionHolder failure(String message, Class<?> clazz, String fieldName){
+    logger.warn("Failure loading function class {}, field {}. " + message, clazz.getName(), fieldName);
+    return null;
+  }
+
+  private FunctionHolder failure(String message, Class<?> clazz){
+    logger.warn("Failure loading function class {}. " + message, clazz.getName());
+    return null;
+  }
+
+  private FunctionHolder failure(String message, Throwable t, Class<?> clazz){
+    logger.warn("Failure loading function class {}. " + message, t, clazz.getName());
+    return null;
+  }
+  
+  private FunctionHolder failure(String message, Class<?> clazz, Field field){
+    return failure(message, clazz, field.getName());
+  }
+  
+  public static void main(String[] args) throws Exception{
+    
+    URL u = Resources.getResource(FunctionConverter.class, "/org/apache/drill/exec/expr/fn/impl/MathFunctions.java");
+    InputStream is = Resources.newInputStreamSupplier(u).getInput();
+    String s = IO.toString(is);
+    System.out.println(s);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
new file mode 100644
index 0000000..460169f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
@@ -0,0 +1,158 @@
+package org.apache.drill.exec.expr.fn;
+
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+
+import com.sun.codemodel.JBlock;
+import com.sun.codemodel.JConditional;
+import com.sun.codemodel.JExpr;
+import com.sun.codemodel.JExpression;
+import com.sun.codemodel.JMod;
+import com.sun.codemodel.JVar;
+
+public class FunctionHolder {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FunctionImplementationRegistry.class);
+  
+  private FunctionTemplate.FunctionScope scope;
+  private FunctionTemplate.NullHandling nullHandling;
+  private boolean isBinaryCommutative;
+  private String functionName;
+  private String evalBody;
+  private String addBody;
+  private String setupBody;
+  private ValueReference[] parameters;
+  private ValueReference returnValue;
+  
+  public FunctionHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, String functionName, ValueReference[] parameters, ValueReference returnValue, Map<String, String> methods) {
+    super();
+    this.scope = scope;
+    this.nullHandling = nullHandling;
+    this.isBinaryCommutative = isBinaryCommutative;
+    this.functionName = functionName;
+    this.setupBody = methods.get("setup");
+    this.addBody = methods.get("add");
+    this.evalBody = methods.get("eval");
+    this.parameters = parameters;
+    this.returnValue = returnValue;
+  }
+
+  public HoldingContainer generateEvalBody(CodeGenerator g, HoldingContainer[] inputVariables){
+    
+    g.getBlock().directStatement(String.format("//---- start of eval portion of %s function. ----//", functionName));
+    
+    JBlock sub = new JBlock(true, true);
+    
+    
+    
+    HoldingContainer out = null;
+
+    // add outside null handling if it is defined.
+    if(nullHandling == NullHandling.NULL_IF_NULL){
+      JExpression e = null;
+      for(HoldingContainer v : inputVariables){
+        if(v.isOptional()){
+          if(e == null){
+            e = v.getIsSet();
+          }else{
+            e = e.mul(v.getIsSet());
+          }
+        }
+      }
+      
+      if(e != null){
+        // if at least one expression must be checked, set up the conditional.
+        returnValue.type = returnValue.type.toBuilder().setMode(DataMode.OPTIONAL).build();
+        out = g.declare(returnValue.type, false);
+        e = e.eq(JExpr.lit(0));
+        JConditional jc = sub._if(e);
+        jc._then().assign(out.getIsSet(), JExpr.lit(0));
+        sub = jc._else();
+      }
+    }
+    
+    if(out == null) out = g.declare(returnValue.type);
+    
+    // add the subblock after the out declaration.
+    g.getBlock().add(sub);
+    
+    // locally name external blocks.
+    
+    // internal out value.
+    JVar internalOutput = sub.decl(JMod.FINAL, g.getHolderType(returnValue.type), returnValue.name, JExpr._new(g.getHolderType(returnValue.type)));
+    
+    for(int i =0; i < inputVariables.length; i++){
+      
+      ValueReference parameter = parameters[i];
+      HoldingContainer inputVariable = inputVariables[i];
+      sub.decl(JMod.FINAL, inputVariable.getHolder().type(), parameter.name, inputVariable.getHolder());  
+    }
+    
+    // add function body.
+    sub.directStatement(evalBody);
+    
+    sub.assign(out.getHolder(), internalOutput);
+
+    g.getBlock().directStatement(String.format("//---- end of eval portion of %s function. ----//\n", functionName));
+    return out;
+  }
+  
+  public boolean matches(FunctionCall call){
+    if(!softCompare(call.getMajorType(), returnValue.type)) return false;
+    if(call.args.size() != parameters.length) return false;
+    for(int i =0; i < parameters.length; i++){
+      ValueReference param = parameters[i];
+      LogicalExpression arg = call.args.get(i);
+      if(!softCompare(param.type, arg.getMajorType())) return false;
+    }
+    
+    return true;
+  }
+  
+  private boolean softCompare(MajorType a, MajorType b){
+    return Types.softEquals(a, b, nullHandling == NullHandling.NULL_IF_NULL);
+  }
+  
+  public String getFunctionName() {
+    return functionName;
+  }
+
+  public static class ValueReference{
+    MajorType type;
+    String name;
+    public ValueReference(MajorType type, String name) {
+      super();
+      this.type = type;
+      this.name = name;
+    }
+    @Override
+    public String toString() {
+      return "ValueReference [type=" + type + ", name=" + name + "]";
+    }
+    
+    
+  }
+
+  @Override
+  public String toString() {
+    final int maxLen = 10;
+    return "FunctionHolder [scope=" + scope + ", isBinaryCommutative=" + isBinaryCommutative + ", functionName="
+        + functionName + ", evalBody=" + evalBody + ", addBody=" + addBody + ", setupBody=" + setupBody
+        + ", parameters="
+        + (parameters != null ? Arrays.asList(parameters).subList(0, Math.min(parameters.length, maxLen)) : null)
+        + ", returnValue=" + returnValue + "]";
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
new file mode 100644
index 0000000..5130f2b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
@@ -0,0 +1,43 @@
+package org.apache.drill.exec.expr.fn;
+
+import java.util.Set;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.util.PathScanner;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.expr.DrillFunc;
+
+import com.google.common.collect.ArrayListMultimap;
+
+public class FunctionImplementationRegistry {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FunctionImplementationRegistry.class);
+  
+  private ArrayListMultimap<String, FunctionHolder> methods = ArrayListMultimap.create();
+  
+  public FunctionImplementationRegistry(DrillConfig config){
+    FunctionConverter converter = new FunctionConverter();
+    Set<Class<? extends DrillFunc>> providerClasses = PathScanner.scanForImplementations(DrillFunc.class, config.getStringList(ExecConstants.FUNCTION_PACKAGES));
+    for (Class<? extends DrillFunc> clazz : providerClasses) {
+      FunctionHolder holder = converter.getHolder(clazz);
+      if(holder != null){
+        methods.put(holder.getFunctionName(), holder);
+        logger.debug("Registering function {}", holder);
+      }else{
+        logger.debug("Unable to initialize function for class {}", clazz.getName());
+      }
+    }
+  }
+  
+  public FunctionHolder getFunction(FunctionCall call){
+    for(FunctionHolder h : methods.get(call.getDefinition().getName())){
+      if(h.matches(call)){
+        return h;
+      }
+    }
+    throw new UnsupportedOperationException(String.format("Unable to find matching function implementation for call %s.", call));
+  }
+
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
new file mode 100644
index 0000000..22b9046
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
@@ -0,0 +1,52 @@
+package org.apache.drill.exec.expr.fn;
+
+import java.io.StringWriter;
+import java.util.Map;
+
+import org.codehaus.janino.Java;
+import org.codehaus.janino.Java.ClassDeclaration;
+import org.codehaus.janino.Java.MethodDeclarator;
+import org.codehaus.janino.util.Traverser;
+
+import com.beust.jcommander.internal.Maps;
+
+public class MethodGrabbingVisitor{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MethodGrabbingVisitor.class);
+  
+  private Class<?> c;
+  private Map<String, String> methods = Maps.newHashMap();
+  private ClassFinder classFinder = new ClassFinder();
+  private boolean captureMethods = false;
+  
+  private MethodGrabbingVisitor(Class<?> c) {
+    super();
+    this.c = c;
+  }
+  
+  public class ClassFinder extends Traverser{
+
+    @Override
+    public void traverseClassDeclaration(ClassDeclaration cd) {
+      captureMethods = c.getName().equals(cd.getClassName());
+      super.traverseClassDeclaration(cd);
+    }
+
+    @Override
+    public void traverseMethodDeclarator(MethodDeclarator md) {
+      if(captureMethods){
+        StringWriter writer = new StringWriter();
+        ModifiedUnparseVisitor v = new ModifiedUnparseVisitor(writer);
+        md.accept(v);
+        methods.put(md.name, writer.getBuffer().toString());  
+      }
+    } 
+  }
+
+  
+  public static Map<String, String> getMethods(Java.CompilationUnit cu, Class<?> c){
+    MethodGrabbingVisitor visitor = new MethodGrabbingVisitor(c);
+    cu.getPackageMemberTypeDeclarations()[0].accept(visitor.classFinder.comprehensiveVisitor());
+    return visitor.methods;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
new file mode 100644
index 0000000..72b2008
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
@@ -0,0 +1,891 @@
+package org.apache.drill.exec.expr.fn;
+
+
+/*
+ * 
+ * Modified so that we can avoid printing some things.
+ * 
+ * Janino - An embedded Java[TM] compiler
+ *
+ * Copyright (c) 2001-2010, Arno Unkrig
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without modification, are permitted provided that the
+ * following conditions are met:
+ *
+ *    1. Redistributions of source code must retain the above copyright notice, this list of conditions and the
+ *       following disclaimer.
+ *    2. Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the
+ *       following disclaimer in the documentation and/or other materials provided with the distribution.
+ *    3. The name of the author may not be used to endorse or promote products derived from this software without
+ *       specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+ * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
+ * THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.io.Writer;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.codehaus.janino.JaninoRuntimeException;
+import org.codehaus.janino.Java;
+import org.codehaus.janino.Mod;
+import org.codehaus.janino.Parser;
+import org.codehaus.janino.Scanner;
+import org.codehaus.janino.Visitor.ComprehensiveVisitor;
+import org.codehaus.janino.util.AutoIndentWriter;
+
+/**
+ * A visitor that unparses (un-compiles) an AST to a {@link Writer}. See
+ * {@link #main(String[])} for a usage example.
+ */
+public class ModifiedUnparseVisitor implements ComprehensiveVisitor {
+    protected final AutoIndentWriter aiw;
+    protected final PrintWriter      pw;
+
+    /**
+     * Testing of parsing/unparsing.
+     * <p>
+     * Reads compilation units from the files named on the command line
+     * and unparses them to {@link System#out}.
+     */
+    public static void main(String[] args) throws Exception {
+        Writer w = new BufferedWriter(new OutputStreamWriter(System.out));
+        for (int i = 0; i < args.length; ++i) {
+            String fileName = args[i];
+
+            // Parse each compilation unit.
+            FileReader r = new FileReader(fileName);
+            Java.CompilationUnit cu;
+            try {
+                cu = new Parser(new Scanner(fileName, r)).parseCompilationUnit();
+            } finally {
+                r.close();
+            }
+
+            // Unparse each compilation unit.
+            ModifiedUnparseVisitor.unparse(cu, w);
+        }
+        w.flush();
+    }
+
+    /**
+     * Unparse the given {@link Java.CompilationUnit} to the given {@link Writer}.
+     */
+    public static void unparse(Java.CompilationUnit cu, Writer w) {
+        ModifiedUnparseVisitor uv = new ModifiedUnparseVisitor(w);
+        uv.unparseCompilationUnit(cu);
+        uv.close();
+    }
+
+    public ModifiedUnparseVisitor(Writer w) {
+        this.aiw = new AutoIndentWriter(w);
+        this.pw = new PrintWriter(this.aiw, true);
+    }
+
+    /**
+     * Flushes all generated code and closes the {@link Writer} that was passed
+     * to {@link #ModifiedUnparseVisitor(Writer)}.
+     */
+    public void close() {
+        this.pw.close();
+    }
+
+    public void unparseCompilationUnit(Java.CompilationUnit cu) {
+        if (cu.optionalPackageDeclaration != null) {
+            this.pw.println();
+            this.pw.println("package " + cu.optionalPackageDeclaration.packageName + ';');
+        }
+        if (!cu.importDeclarations.isEmpty()) {
+            this.pw.println();
+            for (Iterator it = cu.importDeclarations.iterator(); it.hasNext();) {
+                ((Java.CompilationUnit.ImportDeclaration) it.next()).accept(this);
+            }
+        }
+        for (Iterator it = cu.packageMemberTypeDeclarations.iterator(); it.hasNext();) {
+            this.pw.println();
+            this.unparseTypeDeclaration((Java.PackageMemberTypeDeclaration) it.next());
+            this.pw.println();
+        }
+    }
+
+    public void visitSingleTypeImportDeclaration(Java.CompilationUnit.SingleTypeImportDeclaration stid) {
+        this.pw.println("import " + Java.join(stid.identifiers, ".") + ';');
+    }
+    public void visitTypeImportOnDemandDeclaration(Java.CompilationUnit.TypeImportOnDemandDeclaration tiodd) {
+        this.pw.println("import " + Java.join(tiodd.identifiers, ".") + ".*;");
+    }
+    public void visitSingleStaticImportDeclaration(Java.CompilationUnit.SingleStaticImportDeclaration ssid) {
+        this.pw.println("import static " + Java.join(ssid.identifiers, ".") + ';');
+    }
+    public void visitStaticImportOnDemandDeclaration(Java.CompilationUnit.StaticImportOnDemandDeclaration siodd) {
+        this.pw.println("import static " + Java.join(siodd.identifiers, ".") + ".*;");
+    }
+
+    public void visitLocalClassDeclaration(Java.LocalClassDeclaration lcd) {
+        this.unparseNamedClassDeclaration(lcd);
+    }
+    public void visitMemberClassDeclaration(Java.MemberClassDeclaration mcd) {
+        this.unparseNamedClassDeclaration(mcd);
+    }
+    public void visitMemberInterfaceDeclaration(Java.MemberInterfaceDeclaration mid) {
+        this.unparseInterfaceDeclaration(mid);
+    }
+    public void visitPackageMemberClassDeclaration(Java.PackageMemberClassDeclaration pmcd) {
+        this.unparseNamedClassDeclaration(pmcd);
+    }
+    public void visitPackageMemberInterfaceDeclaration(Java.PackageMemberInterfaceDeclaration pmid) {
+        this.unparseInterfaceDeclaration(pmid);
+    }
+    public void visitConstructorDeclarator(Java.ConstructorDeclarator cd) {
+        this.unparseDocComment(cd);
+        this.unparseModifiers(cd.modifiers);
+        Java.ClassDeclaration declaringClass = cd.getDeclaringClass();
+        this.pw.print(
+            declaringClass instanceof Java.NamedClassDeclaration
+            ? ((Java.NamedClassDeclaration) declaringClass).name
+            : "UNNAMED"
+        );
+        this.unparseFunctionDeclaratorRest(cd);
+        this.pw.print(' ');
+        if (cd.optionalConstructorInvocation != null) {
+            this.pw.println('{');
+            this.pw.print(AutoIndentWriter.INDENT);
+            this.unparseBlockStatement(cd.optionalConstructorInvocation);
+            this.pw.println(';');
+
+            if (!cd.optionalStatements.isEmpty()) {
+                this.pw.println();
+                this.unparseStatements(cd.optionalStatements);
+            }
+            this.pw.print(AutoIndentWriter.UNINDENT + "}");
+        } else
+        if (cd.optionalStatements.isEmpty()) {
+            this.pw.print("{}");
+        } else
+        {
+            this.pw.println('{');
+            this.pw.print(AutoIndentWriter.INDENT);
+            this.unparseStatements(cd.optionalStatements);
+            this.pw.print(AutoIndentWriter.UNINDENT + "}");
+        }
+    }
+    public void visitMethodDeclarator(Java.MethodDeclarator md) {
+        if (md.optionalStatements == null) {
+            this.pw.print(';');
+        } else
+        if (md.optionalStatements.isEmpty()) {
+            this.pw.print(" ");
+        } else
+        {
+            this.pw.print(AutoIndentWriter.INDENT);
+            this.unparseStatements(md.optionalStatements);
+            this.pw.print(AutoIndentWriter.UNINDENT);
+        }
+    }
+    public void visitFieldDeclaration(Java.FieldDeclaration fd) {
+        this.unparseDocComment(fd);
+        this.unparseModifiers(fd.modifiers);
+        this.unparseType(fd.type);
+        this.pw.print(' ');
+        for (int i = 0; i < fd.variableDeclarators.length; ++i) {
+            if (i > 0) this.pw.print(", ");
+            this.unparseVariableDeclarator(fd.variableDeclarators[i]);
+        }
+        this.pw.print(';');
+    }
+    public void visitInitializer(Java.Initializer i) {
+        if (i.statiC) this.pw.print("static ");
+        this.unparseBlockStatement(i.block);
+    }
+    public void visitBlock(Java.Block b) {
+        if (b.statements.isEmpty()) {
+            this.pw.print("{}");
+            return;
+        }
+        this.pw.println('{');
+        this.pw.print(AutoIndentWriter.INDENT);
+        this.unparseStatements(b.statements);
+        this.pw.print(AutoIndentWriter.UNINDENT + "}");
+    }
+
+    private void unparseStatements(List statements) {
+        int state = -1;
+        for (Iterator it = statements.iterator(); it.hasNext();) {
+            Java.BlockStatement bs = (Java.BlockStatement) it.next();
+            int x = (
+                bs instanceof Java.Block                             ? 1 :
+                bs instanceof Java.LocalClassDeclarationStatement    ? 2 :
+                bs instanceof Java.LocalVariableDeclarationStatement ? 3 :
+                bs instanceof Java.SynchronizedStatement             ? 4 :
+                99
+            );
+            if (state != -1 && state != x) this.pw.println(AutoIndentWriter.CLEAR_TABULATORS);
+            state = x;
+
+            this.unparseBlockStatement(bs);
+            this.pw.println();
+        }
+    }
+    public void visitBreakStatement(Java.BreakStatement bs) {
+        this.pw.print("break");
+        if (bs.optionalLabel != null) this.pw.print(' ' + bs.optionalLabel);
+        this.pw.print(';');
+    }
+    public void visitContinueStatement(Java.ContinueStatement cs) {
+        this.pw.print("continue");
+        if (cs.optionalLabel != null) this.pw.print(' ' + cs.optionalLabel);
+        this.pw.print(';');
+    }
+    public void visitDoStatement(Java.DoStatement ds) {
+        this.pw.print("do ");
+        this.unparseBlockStatement(ds.body);
+        this.pw.print("while (");
+        this.unparse(ds.condition);
+        this.pw.print(");");
+    }
+    public void visitEmptyStatement(Java.EmptyStatement es) {
+        this.pw.print(';');
+    }
+    public void visitExpressionStatement(Java.ExpressionStatement es) {
+        this.unparse(es.rvalue);
+        this.pw.print(';');
+    }
+    public void visitForStatement(Java.ForStatement fs) {
+        this.pw.print("for (");
+        if (fs.optionalInit != null) {
+            this.unparseBlockStatement(fs.optionalInit);
+        } else {
+            this.pw.print(';');
+        }
+        if (fs.optionalCondition != null) {
+            this.pw.print(' ');
+            this.unparse(fs.optionalCondition);
+        }
+        this.pw.print(';');
+        if (fs.optionalUpdate != null) {
+            this.pw.print(' ');
+            for (int i = 0; i < fs.optionalUpdate.length; ++i) {
+                if (i > 0) this.pw.print(", ");
+                this.unparse(fs.optionalUpdate[i]);
+            }
+        }
+        this.pw.print(") ");
+        this.unparseBlockStatement(fs.body);
+    }
+    public void visitIfStatement(Java.IfStatement is) {
+        this.pw.print("if (");
+        this.unparse(is.condition);
+        this.pw.print(") ");
+        this.unparseBlockStatement(is.thenStatement);
+        if (is.optionalElseStatement != null) {
+            this.pw.println(" else");
+            this.unparseBlockStatement(is.optionalElseStatement);
+        }
+    }
+    public void visitLabeledStatement(Java.LabeledStatement ls) {
+        this.pw.println(ls.label + ':');
+        this.unparseBlockStatement(ls.body);
+    }
+    public void visitLocalClassDeclarationStatement(Java.LocalClassDeclarationStatement lcds) {
+        this.unparseTypeDeclaration(lcds.lcd);
+    }
+    public void visitLocalVariableDeclarationStatement(Java.LocalVariableDeclarationStatement lvds) {
+        this.unparseModifiers(lvds.modifiers);
+        this.unparseType(lvds.type);
+        this.pw.print(' ');
+        this.pw.print(AutoIndentWriter.TABULATOR);
+        this.unparseVariableDeclarator(lvds.variableDeclarators[0]);
+        for (int i = 1; i < lvds.variableDeclarators.length; ++i) {
+            this.pw.print(", ");
+            this.unparseVariableDeclarator(lvds.variableDeclarators[i]);
+        }
+        this.pw.print(';');
+    }
+    public void visitReturnStatement(Java.ReturnStatement rs) {
+        this.pw.print("return");
+        if (rs.optionalReturnValue != null) {
+            this.pw.print(' ');
+            this.unparse(rs.optionalReturnValue);
+        }
+        this.pw.print(';');
+    }
+    public void visitSwitchStatement(Java.SwitchStatement ss) {
+        this.pw.print("switch (");
+        this.unparse(ss.condition);
+        this.pw.println(") {");
+        for (Iterator it = ss.sbsgs.iterator(); it.hasNext();) {
+            Java.SwitchStatement.SwitchBlockStatementGroup sbgs = (
+                (Java.SwitchStatement.SwitchBlockStatementGroup) it.next()
+            );
+            this.pw.print(AutoIndentWriter.UNINDENT);
+            try {
+                for (Iterator it2 = sbgs.caseLabels.iterator(); it2.hasNext();) {
+                    Java.Rvalue rv = (Java.Rvalue) it2.next();
+                    this.pw.print("case ");
+                    this.unparse(rv);
+                    this.pw.println(':');
+                }
+                if (sbgs.hasDefaultLabel) this.pw.println("default:");
+            } finally {
+                this.pw.print(AutoIndentWriter.INDENT);
+            }
+            for (Iterator it2 = sbgs.blockStatements.iterator(); it2.hasNext();) {
+                this.unparseBlockStatement((Java.BlockStatement) it2.next());
+                this.pw.println();
+            }
+        }
+        this.pw.print('}');
+    }
+    public void visitSynchronizedStatement(Java.SynchronizedStatement ss) {
+        this.pw.print("synchronized (");
+        this.unparse(ss.expression);
+        this.pw.print(") ");
+        this.unparseBlockStatement(ss.body);
+    }
+    public void visitThrowStatement(Java.ThrowStatement ts) {
+        this.pw.print("throw ");
+        this.unparse(ts.expression);
+        this.pw.print(';');
+    }
+    public void visitTryStatement(Java.TryStatement ts) {
+        this.pw.print("try ");
+        this.unparseBlockStatement(ts.body);
+        for (Iterator it = ts.catchClauses.iterator(); it.hasNext();) {
+            Java.CatchClause cc = (Java.CatchClause) it.next();
+            this.pw.print(" catch (");
+            this.unparseFormalParameter(cc.caughtException);
+            this.pw.print(") ");
+            this.unparseBlockStatement(cc.body);
+        }
+        if (ts.optionalFinally != null) {
+            this.pw.print(" finally ");
+            this.unparseBlockStatement(ts.optionalFinally);
+        }
+    }
+    public void visitWhileStatement(Java.WhileStatement ws) {
+        this.pw.print("while (");
+        this.unparse(ws.condition);
+        this.pw.print(") ");
+        this.unparseBlockStatement(ws.body);
+    }
+    public void unparseVariableDeclarator(Java.VariableDeclarator vd) {
+        this.pw.print(vd.name);
+        for (int i = 0; i < vd.brackets; ++i) this.pw.print("[]");
+        if (vd.optionalInitializer != null) {
+            this.pw.print(" = ");
+            this.unparseArrayInitializerOrRvalue(vd.optionalInitializer);
+        }
+    }
+    public void unparseFormalParameter(Java.FunctionDeclarator.FormalParameter fp) {
+        if (fp.finaL) this.pw.print("final ");
+        this.unparseType(fp.type);
+        this.pw.print(" " + AutoIndentWriter.TABULATOR + fp.name);
+    }
+    public void visitMethodInvocation(Java.MethodInvocation mi) {
+        if (mi.optionalTarget != null) {
+            this.unparseLhs(mi.optionalTarget, ".");
+            this.pw.print('.');
+        }
+        this.pw.print(mi.methodName);
+        this.unparseFunctionInvocationArguments(mi.arguments);
+    }
+    public void visitAlternateConstructorInvocation(Java.AlternateConstructorInvocation aci) {
+        this.pw.print("this");
+        this.unparseFunctionInvocationArguments(aci.arguments);
+    }
+    public void visitSuperConstructorInvocation(Java.SuperConstructorInvocation sci) {
+        if (sci.optionalQualification != null) {
+            this.unparseLhs(sci.optionalQualification, ".");
+            this.pw.print('.');
+        }
+        this.pw.print("super");
+        this.unparseFunctionInvocationArguments(sci.arguments);
+    }
+    public void visitNewClassInstance(Java.NewClassInstance nci) {
+        if (nci.optionalQualification != null) {
+            this.unparseLhs(nci.optionalQualification, ".");
+            this.pw.print('.');
+        }
+        this.pw.print("new " + nci.type.toString());
+        this.unparseFunctionInvocationArguments(nci.arguments);
+    }
+    public void visitAssignment(Java.Assignment a) {
+        this.unparseLhs(a.lhs, a.operator);
+        this.pw.print(' ' + a.operator + ' ');
+        this.unparseRhs(a.rhs, a.operator);
+    }
+    public void visitAmbiguousName(Java.AmbiguousName an) { this.pw.print(an.toString()); }
+    public void visitArrayAccessExpression(Java.ArrayAccessExpression aae) {
+        this.unparseLhs(aae.lhs, "[ ]");
+        this.pw.print('[');
+        this.unparse(aae.index);
+        this.pw.print(']');
+    }
+    public void visitArrayLength(Java.ArrayLength al) {
+        this.unparseLhs(al.lhs, ".");
+        this.pw.print(".length");
+    }
+    public void visitArrayType(Java.ArrayType at) {
+        this.unparseType(at.componentType);
+        this.pw.print("[]");
+    }
+    public void visitBasicType(Java.BasicType bt) {
+        this.pw.print(bt.toString());
+    }
+    public void visitBinaryOperation(Java.BinaryOperation bo) {
+        this.unparseLhs(bo.lhs, bo.op);
+        this.pw.print(' ' + bo.op + ' ');
+        this.unparseRhs(bo.rhs, bo.op);
+    }
+    public void visitCast(Java.Cast c) {
+        this.pw.print('(');
+        this.unparseType(c.targetType);
+        this.pw.print(") ");
+        this.unparseRhs(c.value, "cast");
+    }
+    public void visitClassLiteral(Java.ClassLiteral cl) {
+        this.unparseType(cl.type);
+        this.pw.print(".class");
+    }
+    public void visitConditionalExpression(Java.ConditionalExpression ce) {
+        this.unparseLhs(ce.lhs, "?:");
+        this.pw.print(" ? ");
+        this.unparseLhs(ce.mhs, "?:");
+        this.pw.print(" : ");
+        this.unparseRhs(ce.rhs, "?:");
+    }
+    public void visitCrement(Java.Crement c) {
+        if (c.pre) {
+            this.pw.print(c.operator);
+            this.unparseUnaryOperation(c.operand, c.operator + "x");
+        } else
+        {
+            this.unparseUnaryOperation(c.operand, "x" + c.operator);
+            this.pw.print(c.operator);
+        }
+    }
+    public void visitFieldAccess(Java.FieldAccess fa) {
+        this.unparseLhs(fa.lhs, ".");
+        this.pw.print('.' + fa.field.getName());
+    }
+    public void visitFieldAccessExpression(Java.FieldAccessExpression fae) {
+        this.unparseLhs(fae.lhs, ".");
+        this.pw.print('.' + fae.fieldName);
+    }
+    public void visitSuperclassFieldAccessExpression(Java.SuperclassFieldAccessExpression scfae) {
+        if (scfae.optionalQualification != null) {
+            this.unparseType(scfae.optionalQualification);
+            this.pw.print(".super." + scfae.fieldName);
+        } else
+        {
+            this.pw.print("super." + scfae.fieldName);
+        }
+    }
+    public void visitInstanceof(Java.Instanceof io) {
+        this.unparseLhs(io.lhs, "instanceof");
+        this.pw.print(" instanceof ");
+        this.unparseType(io.rhs);
+    }
+    public void visitLiteral(Java.Literal l) { this.pw.print(l.toString()); }
+    public void visitLocalVariableAccess(Java.LocalVariableAccess lva) { this.pw.print(lva.toString()); }
+    public void visitNewArray(Java.NewArray na) {
+        this.pw.print("new ");
+        this.unparseType(na.type);
+        for (int i = 0; i < na.dimExprs.length; ++i) {
+            this.pw.print('[');
+            this.unparse(na.dimExprs[i]);
+            this.pw.print(']');
+        }
+        for (int i = 0; i < na.dims; ++i) {
+            this.pw.print("[]");
+        }
+    }
+    public void visitNewInitializedArray(Java.NewInitializedArray nai) {
+        this.pw.print("new ");
+        this.unparseType(nai.arrayType);
+        this.pw.print(" ");
+        this.unparseArrayInitializerOrRvalue(nai.arrayInitializer);
+    }
+    public void visitPackage(Java.Package p) { this.pw.print(p.toString()); }
+    public void visitParameterAccess(Java.ParameterAccess pa) { this.pw.print(pa.toString()); }
+    public void visitQualifiedThisReference(Java.QualifiedThisReference qtr) {
+        this.unparseType(qtr.qualification);
+        this.pw.print(".this");
+    }
+    public void visitReferenceType(Java.ReferenceType rt) { this.pw.print(rt.toString()); }
+    public void visitRvalueMemberType(Java.RvalueMemberType rmt) { this.pw.print(rmt.toString()); }
+    public void visitSimpleType(Java.SimpleType st) { this.pw.print(st.toString()); }
+    public void visitSuperclassMethodInvocation(Java.SuperclassMethodInvocation smi) {
+        this.pw.print("super." + smi.methodName);
+        this.unparseFunctionInvocationArguments(smi.arguments);
+    }
+    public void visitThisReference(Java.ThisReference tr) {
+        this.pw.print("this");
+    }
+    public void visitUnaryOperation(Java.UnaryOperation uo) {
+        this.pw.print(uo.operator);
+        this.unparseUnaryOperation(uo.operand, uo.operator + "x");
+    }
+    public void visitParenthesizedExpression(Java.ParenthesizedExpression pe) {
+        this.pw.print('(');
+        this.unparse(pe.value);
+        this.pw.print(')');
+    }
+
+    // Helpers
+
+    private void unparseBlockStatement(Java.BlockStatement blockStatement) {
+        blockStatement.accept(this);
+    }
+
+    private void unparseTypeDeclaration(Java.TypeDeclaration typeDeclaration) {
+        typeDeclaration.accept(this);
+    }
+
+    private void unparseType(Java.Type type) {
+        ((Java.Atom) type).accept(this);
+    }
+
+    private void unparse(Java.Atom operand) {
+        operand.accept(this);
+    }
+
+    /**
+     * Iff the <code>operand</code> is unnatural for the <code>unaryOperator</code>, enclose the
+     * <code>operand</code> in parentheses. Example: "a+b" is an unnatural operand for unary "!x".
+     *
+     * @param unaryOperator ++x --x +x -x ~x !x x++ x--
+     */
+    private void unparseUnaryOperation(Java.Rvalue operand, String unaryOperator) {
+        int cmp = ModifiedUnparseVisitor.comparePrecedence(unaryOperator, operand);
+        this.unparse(operand, cmp < 0);
+    }
+
+    /**
+     * Iff the <code>lhs</code> is unnatural for the <code>binaryOperator</code>, enclose the
+     * <code>lhs</code> in parentheses. Example: "a+b" is an unnatural lhs for operator "*".
+     *
+     * @param binaryOperator = +=... ?: || && | ^ & == != < > <= >= instanceof << >> >>> + - * / % cast
+     */
+
+    private void unparseLhs(Java.Atom lhs, String binaryOperator) {
+        int cmp = ModifiedUnparseVisitor.comparePrecedence(binaryOperator, lhs);
+        this.unparse(lhs, cmp < 0 || (cmp == 0 && ModifiedUnparseVisitor.isLeftAssociate(binaryOperator)));
+    }
+
+
+    /**
+     * Iff the <code>rhs</code> is unnatural for the <code>binaryOperator</code>, enclose the
+     * <code>rhs</code> in parentheses. Example: "a+b" is an unnatural rhs for operator "*".
+     */
+    private void unparseRhs(Java.Rvalue rhs, String binaryOperator) {
+        int cmp = ModifiedUnparseVisitor.comparePrecedence(binaryOperator, rhs);
+        this.unparse(rhs, cmp < 0 || (cmp == 0 && ModifiedUnparseVisitor.isRightAssociate(binaryOperator)));
+    }
+
+    private void unparse(Java.Atom operand, boolean natural) {
+        if (!natural) this.pw.print("((( ");
+        this.unparse(operand);
+        if (!natural) this.pw.print(" )))");
+    }
+
+    /**
+     * Return true iff operator is right associative e.g. <code>a = b = c</code> evaluates as
+     * <code>a = (b = c)</code>.
+     *
+     * @return Return true iff operator is right associative
+     */
+    private static boolean isRightAssociate(String op) {
+        return ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS.contains(op);
+    }
+
+    /**
+     * Return true iff operator is left associative e.g. <code>a - b - c</code> evaluates as
+     * <code>(a - b) - c</code>.
+     *
+     * @return Return true iff operator is left associative
+     */
+    private static boolean isLeftAssociate(String op) {
+        return ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS.contains(op);
+    }
+
+    /**
+     * Returns a value
+     * <ul>
+     *   <li>&lt; 0 iff the <code>operator</code> has lower precedence than the <code>operand</code>
+     *   <li>==; 0 iff the <code>operator</code> has equal precedence than the <code>operand</code>
+     *   <li>&gt; 0 iff the <code>operator</code> has higher precedence than the <code>operand</code>
+     * </ul>
+     */
+    private static int comparePrecedence(String operator, Java.Atom operand) {
+        if (operand instanceof Java.BinaryOperation) {
+            return (
+                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
+                - ModifiedUnparseVisitor.getOperatorPrecedence(((Java.BinaryOperation) operand).op)
+            );
+        } else
+        if (operand instanceof Java.UnaryOperation) {
+            return (
+                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
+                - ModifiedUnparseVisitor.getOperatorPrecedence(((Java.UnaryOperation) operand).operator + "x")
+            );
+        } else
+        if (operand instanceof Java.ConditionalExpression) {
+            return ModifiedUnparseVisitor.getOperatorPrecedence(operator) - ModifiedUnparseVisitor.getOperatorPrecedence("?:");
+        } else
+        if (operand instanceof Java.Instanceof) {
+            return ModifiedUnparseVisitor.getOperatorPrecedence(operator) - ModifiedUnparseVisitor.getOperatorPrecedence("instanceof");
+        } else
+        if (operand instanceof Java.Cast) {
+            return ModifiedUnparseVisitor.getOperatorPrecedence(operator) - ModifiedUnparseVisitor.getOperatorPrecedence("cast");
+        } else
+        if (operand instanceof Java.MethodInvocation || operand instanceof Java.FieldAccess) {
+            return ModifiedUnparseVisitor.getOperatorPrecedence(operator) - ModifiedUnparseVisitor.getOperatorPrecedence(".");
+        } else
+        if (operand instanceof Java.NewArray) {
+            return ModifiedUnparseVisitor.getOperatorPrecedence(operator) - ModifiedUnparseVisitor.getOperatorPrecedence("new");
+        } else
+        if (operand instanceof Java.Crement) {
+            Java.Crement c = (Java.Crement) operand;
+            return (
+                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
+                - ModifiedUnparseVisitor.getOperatorPrecedence(c.pre ? c.operator + "x" : "x" + c.operator)
+            );
+        } else
+        {
+            // All other rvalues (e.g. literal) have higher precedence than any operator.
+            return -1;
+        }
+    }
+    private static int getOperatorPrecedence(String operator) {
+        return ((Integer) ModifiedUnparseVisitor.OPERATOR_PRECEDENCE.get(operator)).intValue();
+    }
+    private static final Set LEFT_ASSOCIATIVE_OPERATORS = new HashSet();
+    private static final Set RIGHT_ASSOCIATIVE_OPERATORS = new HashSet();
+    private static final Set UNARY_OPERATORS = new HashSet();
+    private static final Map OPERATOR_PRECEDENCE = new HashMap();
+    static {
+        Object[] ops = {
+            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "=", "*=", "/=", "%=", "+=", "-=", "<<=", ">>=", ">>>=",
+                                                        "&=", "^=", "|=",
+            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "?:",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "||",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "&&",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "|",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "^",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "&",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "==", "!=",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "<", ">", "<=", ">=", "instanceof",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "<<", ">>", ">>>",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "+", "-",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "*", "/", "%",
+            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "cast",
+            ModifiedUnparseVisitor.UNARY_OPERATORS,             "++x", "--x", "+x", "-x", "~x", "!x",
+            ModifiedUnparseVisitor.UNARY_OPERATORS,             "x++", "x--",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "new",
+            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  ".", "[ ]",
+        };
+        int precedence = 0;
+        LOOP1: for (int i = 0;;) {
+            Set s = (Set) ops[i++];
+            Integer pi = new Integer(++precedence);
+            for (;;) {
+                if (i == ops.length) break LOOP1;
+                if (!(ops[i] instanceof String)) break;
+                String op = (String) ops[i++];
+                s.add(op);
+                ModifiedUnparseVisitor.OPERATOR_PRECEDENCE.put(op, pi);
+            }
+        }
+    }
+
+    private void unparseNamedClassDeclaration(Java.NamedClassDeclaration ncd) {
+        this.unparseDocComment(ncd);
+        this.unparseModifiers(ncd.getModifiers());
+        this.pw.print("class " + ncd.name);
+        if (ncd.optionalExtendedType != null) {
+            this.pw.print(" extends ");
+            this.unparseType(ncd.optionalExtendedType);
+        }
+        if (ncd.implementedTypes.length > 0) this.pw.print(" implements " + Java.join(ncd.implementedTypes, ", "));
+        this.pw.println(" {");
+        this.pw.print(AutoIndentWriter.INDENT);
+        this.unparseClassDeclarationBody(ncd);
+        this.pw.print(AutoIndentWriter.UNINDENT + "}");
+    }
+    private void unparseArrayInitializerOrRvalue(Java.ArrayInitializerOrRvalue aiorv) {
+        if (aiorv instanceof Java.Rvalue) {
+            this.unparse((Java.Rvalue) aiorv);
+        } else
+        if (aiorv instanceof Java.ArrayInitializer) {
+            Java.ArrayInitializer ai = (Java.ArrayInitializer) aiorv;
+            if (ai.values.length == 0) {
+                this.pw.print("{}");
+            } else
+            {
+                this.pw.print("{ ");
+                this.unparseArrayInitializerOrRvalue(ai.values[0]);
+                for (int i = 1; i < ai.values.length; ++i) {
+                    this.pw.print(", ");
+                    this.unparseArrayInitializerOrRvalue(ai.values[i]);
+                }
+                this.pw.print(" }");
+            }
+        } else
+        {
+            throw new JaninoRuntimeException(
+                "Unexpected array initializer or rvalue class "
+                + aiorv.getClass().getName()
+            );
+        }
+    }
+
+    public void visitAnonymousClassDeclaration(Java.AnonymousClassDeclaration acd) {
+        this.unparseType(acd.baseType);
+        this.pw.println(" {");
+        this.pw.print(AutoIndentWriter.INDENT);
+        this.unparseClassDeclarationBody(acd);
+        this.pw.print(AutoIndentWriter.UNINDENT + "}");
+    }
+    public void visitNewAnonymousClassInstance(Java.NewAnonymousClassInstance naci) {
+        if (naci.optionalQualification != null) {
+            this.unparseLhs(naci.optionalQualification, ".");
+            this.pw.print('.');
+        }
+        this.pw.print("new " + naci.anonymousClassDeclaration.baseType.toString() + '(');
+        for (int i = 0; i < naci.arguments.length; ++i) {
+            if (i > 0) this.pw.print(", ");
+            this.unparse(naci.arguments[i]);
+        }
+        this.pw.println(") {");
+        this.pw.print(AutoIndentWriter.INDENT);
+        this.unparseClassDeclarationBody(naci.anonymousClassDeclaration);
+        this.pw.print(AutoIndentWriter.UNINDENT + "}");
+    }
+    // Multi-line!
+    private void unparseClassDeclarationBody(Java.ClassDeclaration cd) {
+        for (Iterator it = cd.constructors.iterator(); it.hasNext();) {
+            this.pw.println();
+            ((Java.ConstructorDeclarator) it.next()).accept(this);
+            this.pw.println();
+        }
+        this.unparseAbstractTypeDeclarationBody(cd);
+        for (Iterator it = cd.variableDeclaratorsAndInitializers.iterator(); it.hasNext();) {
+            this.pw.println();
+            ((Java.TypeBodyDeclaration) it.next()).accept(this);
+            this.pw.println();
+        }
+    }
+    private void unparseInterfaceDeclaration(Java.InterfaceDeclaration id) {
+        this.unparseDocComment(id);
+        this.unparseModifiers(id.getModifiers());
+        //make sure we print "interface", even if it wasn't in the modifiers
+        if ((id.getModifiers() & Mod.INTERFACE) == 0) {
+            this.pw.print("interface ");
+        }
+        this.pw.print(id.name);
+        if (id.extendedTypes.length > 0) this.pw.print(" extends " + Java.join(id.extendedTypes, ", "));
+        this.pw.println(" {");
+        this.pw.print(AutoIndentWriter.INDENT);
+        this.unparseAbstractTypeDeclarationBody(id);
+        for (Iterator it = id.constantDeclarations.iterator(); it.hasNext();) {
+            ((Java.TypeBodyDeclaration) it.next()).accept(this);
+            this.pw.println();
+        }
+        this.pw.print(AutoIndentWriter.UNINDENT + "}");
+    }
+    // Multi-line!
+    private void unparseAbstractTypeDeclarationBody(Java.AbstractTypeDeclaration atd) {
+        for (Iterator it = atd.getMethodDeclarations().iterator(); it.hasNext();) {
+            this.pw.println();
+            ((Java.MethodDeclarator) it.next()).accept(this);
+            this.pw.println();
+        }
+        for (Iterator it = atd.getMemberTypeDeclarations().iterator(); it.hasNext();) {
+            this.pw.println();
+            ((Java.TypeBodyDeclaration) it.next()).accept(this);
+            this.pw.println();
+        }
+    }
+    private void unparseFunctionDeclaratorRest(Java.FunctionDeclarator fd) {
+        boolean big = fd.formalParameters.length >= 4;
+        this.pw.print('(');
+        if (big) { this.pw.println(); this.pw.print(AutoIndentWriter.INDENT); }
+        for (int i = 0; i < fd.formalParameters.length; ++i) {
+            if (i > 0) {
+                if (big) {
+                    this.pw.println(',');
+                } else
+                {
+                    this.pw.print(", ");
+                }
+            }
+            this.unparseFormalParameter(fd.formalParameters[i]);
+        }
+        if (big) { this.pw.println(); this.pw.print(AutoIndentWriter.UNINDENT); }
+        this.pw.print(')');
+        if (fd.thrownExceptions.length > 0) this.pw.print(" throws " + Java.join(fd.thrownExceptions, ", "));
+    }
+    private void unparseDocComment(Java.DocCommentable dc) {
+        String optionalDocComment = dc.getDocComment();
+        if (optionalDocComment != null) {
+            this.pw.print("/**");
+            BufferedReader br = new BufferedReader(new StringReader(optionalDocComment));
+            for (;;) {
+                String line;
+                try {
+                    line = br.readLine();
+                } catch (IOException e) {
+                    throw new JaninoRuntimeException();
+                }
+                if (line == null) break;
+                this.pw.println(line);
+                this.pw.print(" *");
+            }
+            this.pw.println("/");
+        }
+    }
+    private void unparseModifiers(short modifiers) {
+        if (modifiers != 0) {
+            this.pw.print(Mod.shortToString(modifiers) + ' ');
+        }
+    }
+    private void unparseFunctionInvocationArguments(Java.Rvalue[] arguments) {
+        boolean big = arguments.length >= 5;
+        this.pw.print('(');
+        if (big) { this.pw.println(); this.pw.print(AutoIndentWriter.INDENT); }
+        for (int i = 0; i < arguments.length; ++i) {
+            if (i > 0) {
+                if (big) {
+                    this.pw.println(',');
+                } else
+                {
+                    this.pw.print(", ");
+                }
+            }
+            this.unparse(arguments[i]);
+        }
+        if (big) { this.pw.println(); this.pw.print(AutoIndentWriter.UNINDENT); }
+        this.pw.print(')');
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
new file mode 100644
index 0000000..702a0ad
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
@@ -0,0 +1,48 @@
+package org.apache.drill.exec.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.IntHolder;
+import org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.LongHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+public class MathFunctions{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MathFunctions.class);
+  
+  private MathFunctions(){}
+  
+  @FunctionTemplate(name = "add", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class Add1 implements DrillFunc{
+    
+    @Param IntHolder left;
+    @Param IntHolder right;
+    @Output IntHolder out;
+
+    public void setup(RecordBatch b){}
+    
+    public void eval(){
+      out.value = left.value + right.value;
+    }
+
+  }
+  
+  @FunctionTemplate(name = "add", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class LongAdd1 implements DrillFunc{
+    
+    @Param LongHolder left;
+    @Param LongHolder right;
+    @Output LongHolder out;
+
+    public void setup(RecordBatch b){}
+    
+    public void eval(){
+      out.value = left.value + right.value;
+    }
+
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
new file mode 100644
index 0000000..0ef2e04
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
@@ -0,0 +1,4 @@
+package org.apache.drill.exec.expr.holders;
+
+public interface ValueHolder {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
new file mode 100644
index 0000000..adf97fe
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
@@ -0,0 +1,44 @@
+package org.apache.drill.exec.expr.holders;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+
+public class ValueHolderImplmenetations {
+  
+  private ValueHolderImplmenetations(){}
+  
+  public final static class BooleanHolder implements ValueHolder{
+    public static final MajorType TYPE = Types.required(MinorType.BOOLEAN); 
+    public int value;
+  }
+  
+  public final static class NullableBooleanHolder implements ValueHolder {
+    public static final MajorType TYPE = Types.optional(MinorType.BOOLEAN);
+    public int value;
+    public int isSet;
+  }
+  
+  public final static class IntHolder implements ValueHolder{
+    public static final MajorType TYPE = Types.required(MinorType.INT); 
+    public int value;
+  }
+  
+  public final static class NullableIntHolder implements ValueHolder {
+    public static final MajorType TYPE = Types.optional(MinorType.INT);
+    public int value;
+    public int isSet;
+  }
+
+  public final static class LongHolder implements ValueHolder {
+    public static final MajorType TYPE = Types.required(MinorType.BIGINT);
+    public long value;
+  }
+  
+  public final static class NullableLongHolder implements ValueHolder {
+    public static final MajorType TYPE = Types.optional(MinorType.BIGINT);
+    public long value;
+    public int isSet;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
new file mode 100644
index 0000000..554c7d3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
@@ -0,0 +1,12 @@
+package org.apache.drill.exec.expr.holders;
+
+import io.netty.buffer.ByteBuf;
+
+public class VarBinaryHolder {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarBinaryHolder.class);
+  
+  public ByteBuf buffer;
+  public int start;
+  public int length;
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index 33707a0..1dd7de8 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -17,7 +17,15 @@
  ******************************************************************************/
 package org.apache.drill.exec.ops;
 
+import java.io.IOException;
+
 import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.compile.ClassTransformer;
+import org.apache.drill.exec.compile.QueryClassLoader;
+import org.apache.drill.exec.compile.TemplateClassDefinition;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
 import org.apache.drill.exec.physical.impl.FilteringRecordBatchTransformer;
@@ -54,8 +62,13 @@ public class FragmentContext {
   private final IncomingBuffers buffers;
   private volatile Throwable failureCause;
   private volatile boolean failed = false;
+  private final FunctionImplementationRegistry funcRegistry;
+  private final QueryClassLoader loader;
+  private final ClassTransformer transformer;
   
-  public FragmentContext(DrillbitContext dbContext, FragmentHandle handle, UserClientConnection connection, IncomingBuffers buffers) {
+  public FragmentContext(DrillbitContext dbContext, FragmentHandle handle, UserClientConnection connection, IncomingBuffers buffers, FunctionImplementationRegistry funcRegistry) {
+    this.loader = new QueryClassLoader(true);
+    this.transformer = new ClassTransformer();
     this.fragmentTime = dbContext.getMetrics().timer(METRIC_TIMER_FRAGMENT_TIME);
     this.batchesCompleted = new SingleThreadNestedCounter(dbContext, METRIC_BATCHES_COMPLETED);
     this.recordsCompleted = new SingleThreadNestedCounter(dbContext, METRIC_RECORDS_COMPLETED);
@@ -64,6 +77,7 @@ public class FragmentContext {
     this.connection = connection;
     this.handle = handle;
     this.buffers = buffers;
+    this.funcRegistry = funcRegistry;
   }
 
   public void fail(Throwable cause) {
@@ -89,6 +103,10 @@ public class FragmentContext {
     return context.getAllocator();
   }
 
+  public <T> T getImplementationClass(TemplateClassDefinition<T, Void> templateDefinition, CodeGenerator cg) throws ClassTransformationException, IOException{
+    return transformer.getImplementationClass(this.loader, templateDefinition, cg.generate(), null);
+  }
+  
   public FilteringRecordBatchTransformer getFilteringExpression(LogicalExpression expr){
     return null;
   }
@@ -119,5 +137,11 @@ public class FragmentContext {
     return failed;
   }
   
+  public FunctionImplementationRegistry getFunctionRegistry(){
+    return funcRegistry;
+  }
   
+  public QueryClassLoader getClassLoader(){
+    return loader;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
index 98ef05f..dcaf823 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
@@ -1,28 +1,28 @@
 package org.apache.drill.exec.opt;
 
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.drill.common.JSONOptions;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
 import org.apache.drill.common.PlanProperties;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.logical.data.*;
+import org.apache.drill.common.logical.data.Project;
+import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.common.logical.data.SinkOperator;
+import org.apache.drill.common.logical.data.Store;
 import org.apache.drill.common.logical.data.visitors.AbstractLogicalVisitor;
-import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.exception.OptimizerException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.MockScanPOP;
-import org.apache.drill.exec.physical.config.MockStorePOP;
 import org.apache.drill.exec.physical.config.Screen;
-import org.apache.drill.exec.proto.SchemaDefProtos;
-import org.apache.drill.exec.server.DrillbitContext;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
+import com.fasterxml.jackson.core.type.TypeReference;
 
 /**
  * Created with IntelliJ IDEA.
@@ -99,8 +99,8 @@ public class BasicOptimizer extends Optimizer{
                 }
                 else{
                     myObjects = new ArrayList<>();
-                    MockScanPOP.MockColumn[] cols = { new MockScanPOP.MockColumn("blah", SchemaDefProtos.MinorType.INT, SchemaDefProtos.DataMode.REQUIRED,4,4,4),
-                            new MockScanPOP.MockColumn("blah_2", SchemaDefProtos.MinorType.INT, SchemaDefProtos.DataMode.REQUIRED,4,4,4)};
+                    MockScanPOP.MockColumn[] cols = { new MockScanPOP.MockColumn("blah", MinorType.INT, DataMode.REQUIRED,4,4,4),
+                            new MockScanPOP.MockColumn("blah_2", MinorType.INT, DataMode.REQUIRED,4,4,4)};
                     myObjects.add(new MockScanPOP.MockScanEntry(50, cols));
                 }
             } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
index db3390a..4e9eb4c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
@@ -17,7 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical;
 
-import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.common.types.TypeProtos.MajorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -26,17 +26,17 @@ public class RecordField {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordField.class);
 
   
-  private DataType type;
+  private MajorType type;
   private ValueMode mode;
   
   @JsonCreator
-  public RecordField(@JsonProperty("type") DataType type, @JsonProperty("mode") ValueMode mode) {
+  public RecordField(@JsonProperty("type") MajorType type, @JsonProperty("mode") ValueMode mode) {
     super();
     this.type = type;
     this.mode = mode;
   }
 
-  public DataType getType() {
+  public MajorType getType() {
     return type;
   }
 


Re: [42/53] [abbrv] Types transition

Posted by Ted Dunning <te...@gmail.com>.
On Fri, Jul 19, 2013 at 6:58 PM, <ja...@apache.org> wrote:

>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
> new file mode 100644
> index 0000000..bc44490
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
> @@ -0,0 +1,128 @@
>
> +/*******************************************************************************
> + * 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;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.expression.ErrorCollector;
> +import org.apache.drill.common.expression.ExpressionValidator;
> +import org.apache.drill.common.expression.FunctionCall;
> +import org.apache.drill.common.expression.IfExpression;
> +import org.apache.drill.common.expression.LogicalExpression;
> +import org.apache.drill.common.expression.SchemaPath;
> +import org.apache.drill.common.expression.ValueExpressions;
> +import org.apache.drill.common.expression.visitors.SimpleExprVisitor;
> +import org.apache.drill.exec.record.NullExpression;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
> +
> +import com.google.common.collect.Lists;
> +
> +public class ExpressionTreeMaterializer {
> +
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ExpressionTreeMaterializer.class);
> +
> +
> +
> +  private ExpressionTreeMaterializer() {
> +  };
> +
> +  public static LogicalExpression materialize(LogicalExpression expr,
> RecordBatch batch, ErrorCollector errorCollector) {
> +    return expr.accept(new MaterializeVisitor(batch, errorCollector),
> null);
> +  }
> +
> +  private static class MaterializeVisitor extends
> SimpleExprVisitor<LogicalExpression> {
> +    private final ErrorCollector errorCollector;
> +    private final RecordBatch batch;
> +    private ExpressionValidator validator = new ExpressionValidator();
> +
> +    public MaterializeVisitor(RecordBatch batch, ErrorCollector
> errorCollector) {
> +      this.batch = batch;
> +      this.errorCollector = errorCollector;
> +    }
> +
> +    private LogicalExpression validateNewExpr(LogicalExpression newExpr) {
> +      newExpr.accept(validator, errorCollector);
> +      return newExpr;
> +    }
> +
> +    @Override
> +    public LogicalExpression visitUnknown(LogicalExpression e, Void
> value) throws RuntimeException {
> +      throw new UnsupportedOperationException(String.format("Expression
> tree materializer does not currently support materializing nodes of type
> %s.", e.getClass().getCanonicalName()));
> +    }
> +
> +    @Override
> +    public LogicalExpression visitFunctionCall(FunctionCall call) {
> +      List<LogicalExpression> args = Lists.newArrayList();
> +      for (int i = 0; i < call.args.size(); ++i) {
> +        LogicalExpression newExpr = call.args.get(i).accept(this, null);
> +        args.add(newExpr);
> +      }
> +
> +      return validateNewExpr(new FunctionCall(call.getDefinition(), args,
> call.getPosition()));
> +    }
> +
> +    @Override
> +    public LogicalExpression visitIfExpression(IfExpression ifExpr) {
> +      List<IfExpression.IfCondition> conditions =
> Lists.newArrayList(ifExpr.iterator());
> +      LogicalExpression newElseExpr = ifExpr.elseExpression.accept(this,
> null);
> +
> +      for (int i = 0; i < conditions.size(); ++i) {
> +        IfExpression.IfCondition condition = conditions.get(i);
> +
> +        LogicalExpression newCondition = condition.condition.accept(this,
> null);
> +        LogicalExpression newExpr = condition.expression.accept(this,
> null);
> +        conditions.set(i, new IfExpression.IfCondition(newCondition,
> newExpr));
> +      }
> +
> +      return
> validateNewExpr(IfExpression.newBuilder().setElse(newElseExpr).addConditions(conditions).build());
> +    }
> +
> +    @Override
> +    public LogicalExpression visitSchemaPath(SchemaPath path) {
> +      logger.debug("Visiting schema path {}", path);
> +      TypedFieldId tfId = batch.getValueVector(path);
> +      if (tfId == null) {
> +        return NullExpression.INSTANCE;
> +      } else {
> +        return new ValueVectorReadExpression(tfId);
> +      }
> +    }
> +
> +    @Override
> +    public LogicalExpression
> visitLongConstant(ValueExpressions.LongExpression intExpr) {
> +      return intExpr;
> +    }
> +
> +    @Override
> +    public LogicalExpression
> visitDoubleConstant(ValueExpressions.DoubleExpression dExpr) {
> +      return dExpr;
> +    }
> +
> +    @Override
> +    public LogicalExpression
> visitBooleanConstant(ValueExpressions.BooleanExpression e) {
> +      return e;
> +    }
> +
> +    @Override
> +    public LogicalExpression
> visitQuotedStringConstant(ValueExpressions.QuotedString e) {
> +      return e;
> +    }
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
> new file mode 100644
> index 0000000..813a32f
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/SingleClassStringWriter.java
> @@ -0,0 +1,40 @@
> +package org.apache.drill.exec.expr;
> +
> +import java.io.IOException;
> +import java.io.OutputStream;
> +import java.io.StringWriter;
> +import java.io.Writer;
> +
> +import com.google.common.base.Preconditions;
> +import com.sun.codemodel.CodeWriter;
> +import com.sun.codemodel.JPackage;
> +
> +public class SingleClassStringWriter extends CodeWriter{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(SingleClassStringWriter.class);
> +
> +  private boolean used;
> +  private StringWriter writer = new StringWriter();
> +
> +  @Override
> +  public OutputStream openBinary(JPackage pkg, String fileName) throws
> IOException {
> +    throw new UnsupportedOperationException();
> +  }
> +
> +
> +  @Override
> +  public Writer openSource(JPackage pkg, String fileName) throws
> IOException {
> +    Preconditions.checkArgument(!used, "The SingleClassStringWriter can
> only output once src file.");
> +    used = true;
> +    return writer;
> +  }
> +
> +  @Override
> +  public void close() throws IOException {
> +  }
> +
> +  public StringBuffer getCode(){
> +    return writer.getBuffer();
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
> new file mode 100644
> index 0000000..c8d906d
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
> @@ -0,0 +1,41 @@
> +package org.apache.drill.exec.expr;
> +
> +import org.apache.drill.common.expression.ExpressionPosition;
> +import org.apache.drill.common.expression.LogicalExpression;
> +import org.apache.drill.common.expression.visitors.ExprVisitor;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.exec.record.RecordBatch.TypedFieldId;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +
> +public class ValueVectorReadExpression implements LogicalExpression{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ValueVectorReadExpression.class);
> +
> +  private final MajorType type;
> +  private final int fieldId;
> +
> +  public ValueVectorReadExpression(TypedFieldId tfId) {
> +    this.type = tfId.getType();
> +    this.fieldId = tfId.getFieldId();
> +  }
> +
> +  @Override
> +  public MajorType getMajorType() {
> +    return type;
> +  }
> +
> +  @Override
> +  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E>
> visitor, V value) throws E {
> +    return visitor.visitUnknown(this, value);
> +  }
> +
> +  public int getFieldId() {
> +    return fieldId;
> +  }
> +
> +  @Override
> +  public ExpressionPosition getPosition() {
> +    return ExpressionPosition.UNKNOWN;
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
> new file mode 100644
> index 0000000..96d9d05
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
> @@ -0,0 +1,45 @@
> +package org.apache.drill.exec.expr;
> +
> +import org.apache.drill.common.expression.ExpressionPosition;
> +import org.apache.drill.common.expression.LogicalExpression;
> +import org.apache.drill.common.expression.visitors.ExprVisitor;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.Types;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +
> +public class ValueVectorWriteExpression implements LogicalExpression {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ValueVectorWriteExpression.class);
> +
> +  private final int fieldId;
> +  private final LogicalExpression child;
> +
> +  public ValueVectorWriteExpression(int fieldId, LogicalExpression child){
> +    this.fieldId = fieldId;
> +    this.child = child;
> +  }
> +
> +  public int getFieldId() {
> +    return fieldId;
> +  }
> +
> +  @Override
> +  public MajorType getMajorType() {
> +    return Types.NULL;
> +  }
> +
> +  @Override
> +  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E>
> visitor, V value) throws E {
> +    return visitor.visitUnknown(this, value);
> +  }
> +
> +  @Override
> +  public ExpressionPosition getPosition() {
> +    return ExpressionPosition.UNKNOWN;
> +  }
> +
> +  public LogicalExpression getChild() {
> +    return child;
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
> new file mode 100644
> index 0000000..9e32750
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
> @@ -0,0 +1,25 @@
> +package org.apache.drill.exec.expr.annotations;
> +
> +import java.lang.annotation.ElementType;
> +import java.lang.annotation.Retention;
> +import java.lang.annotation.RetentionPolicy;
> +import java.lang.annotation.Target;
> +
> +
> +@Retention(RetentionPolicy.RUNTIME)
> +@Target({ElementType.TYPE})
> +public @interface FunctionTemplate {
> +
> +  String name();
> +  FunctionScope scope();
> +  NullHandling nulls() default NullHandling.INTERNAL;
> +  boolean isBinaryCommutative() default false;
> +
> +  public static enum NullHandling {
> +    INTERNAL, NULL_IF_NULL;
> +  }
> +
> +  public static enum FunctionScope{
> +    SIMPLE, AGGREGATE, RUNNING;
> +  }
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
> new file mode 100644
> index 0000000..2d77dfc
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Output.java
> @@ -0,0 +1,18 @@
> +package org.apache.drill.exec.expr.annotations;
> +
> +import java.lang.annotation.ElementType;
> +import java.lang.annotation.Retention;
> +import java.lang.annotation.RetentionPolicy;
> +import java.lang.annotation.Target;
> +
> +/**
> + * Describes the field will provide output from the given function.
> + */
> +@Retention(RetentionPolicy.RUNTIME)
> +@Target({ElementType.FIELD})
> +public @interface Output {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(Output.class);
> +
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
> new file mode 100644
> index 0000000..9ba7611
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/Param.java
> @@ -0,0 +1,18 @@
> +package org.apache.drill.exec.expr.annotations;
> +
> +import java.lang.annotation.ElementType;
> +import java.lang.annotation.Retention;
> +import java.lang.annotation.RetentionPolicy;
> +import java.lang.annotation.Target;
> +
> +import org.apache.drill.common.types.TypeProtos.DataMode;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +
> +/**
> + * Marker annotation to determine which fields should be included as
> parameters for the function.
> + */
> +@Retention(RetentionPolicy.RUNTIME)
> +@Target({ElementType.FIELD})
> +public @interface Param {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(Param.class);
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> new file mode 100644
> index 0000000..cc106a7
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/WorkSpace.java
> @@ -0,0 +1,14 @@
> +package org.apache.drill.exec.expr.annotations;
> +
> +import java.lang.annotation.ElementType;
> +import java.lang.annotation.Retention;
> +import java.lang.annotation.RetentionPolicy;
> +import java.lang.annotation.Target;
> +
> +/**
> + * Describes the field will provide output from the given function.
> + */
> +@Retention(RetentionPolicy.RUNTIME)
> +@Target({ElementType.FIELD})
> +public @interface WorkSpace {
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
> new file mode 100644
> index 0000000..7276e7d
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
> @@ -0,0 +1,186 @@
> +package org.apache.drill.exec.expr.fn;
> +
> +import java.io.File;
> +import java.io.IOException;
> +import java.io.InputStream;
> +import java.io.StringReader;
> +import java.lang.reflect.Field;
> +import java.net.URL;
> +import java.util.List;
> +import java.util.Map;
> +
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +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.expr.annotations.WorkSpace;
> +import org.apache.drill.exec.expr.fn.FunctionHolder.ValueReference;
> +import org.apache.drill.exec.expr.holders.ValueHolder;
> +import org.codehaus.commons.compiler.CompileException;
> +import org.codehaus.janino.Java;
> +import org.codehaus.janino.Java.CompilationUnit;
> +import org.codehaus.janino.Parser;
> +import org.codehaus.janino.Scanner;
> +import org.mortbay.util.IO;
> +
> +import com.beust.jcommander.internal.Lists;
> +import com.google.common.io.InputSupplier;
> +import com.google.common.io.Resources;
> +
> +/**
> + * Converts FunctionCalls to Java Expressions.
> + */
> +public class FunctionConverter {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FunctionConverter.class);
> +
> +
> +  public <T extends DrillFunc> FunctionHolder getHolder(Class<T> clazz){
> +    FunctionTemplate template =
> clazz.getAnnotation(FunctionTemplate.class);
> +    if(template == null){
> +      return failure("Class does not declare FunctionTemplate
> annotation.", clazz);
> +    }
> +
> +    // start by getting field information.
> +    List<ValueReference> params = Lists.newArrayList();
> +    List<String> workspaceFields = Lists.newArrayList();
> +    ValueReference outputField = null;
> +
> +
> +    for(Field field : clazz.getDeclaredFields()){
> +
> +      Param param = field.getAnnotation(Param.class);
> +      Output output = field.getAnnotation(Output.class);
> +      WorkSpace workspace = field.getAnnotation(WorkSpace.class);
> +
> +      int i =0;
> +      if(param != null) i++;
> +      if(output != null) i++;
> +      if(workspace != null) i++;
> +      if(i == 0){
> +        return failure("The field must be either a @Param, @Output or
> @WorkSpace field.", clazz, field);
> +      }else if(i > 1){
> +        return failure("The field must be only one of @Param, @Output or
> @WorkSpace.  It currently has more than one of these annotations.", clazz,
> field);
> +      }
> +
> +
> +
> +      if(param != null || output != null){
> +
> +        // check that param and output are value holders.
> +        if(!ValueHolder.class.isAssignableFrom(field.getType())){
> +          return failure(String.format("The field doesn't holds value of
> type %s which does not implement the ValueHolder interface.  All fields of
> type @Param or @Output must extend this interface..", field.getType()),
> clazz, field);
> +        }
> +
> +        // get the type field from the value holder.
> +        MajorType type = null;
> +        try{
> +          type = getStaticFieldValue("TYPE", field.getType(),
> MajorType.class);
> +        }catch(Exception e){
> +          return failure("Failure while trying to access the
> ValueHolder's TYPE static variable.  All ValueHolders must contain a static
> TYPE variable that defines their MajorType.", e, clazz, field.getName());
> +        }
> +
> +
> +        ValueReference p = new ValueReference(type, field.getName());
> +        if(param != null){
> +          params.add(p);
> +        }else{
> +          if(outputField != null){
> +            return failure("You've declared more than one @Output field.
>  You must declare one and only @Output field per Function class.", clazz,
> field);
> +          }else{
> +            outputField = p;
> +
> +          }
> +
> +        }
> +
> +      }else{
> +        // workspace work.
> +        workspaceFields.add(field.getName());
> +      }
> +
> +    }
> +
> +
> +    if(!workspaceFields.isEmpty()) return failure("This function declares
> one or more workspace fields.  However, those have not yet been
> implemented.", clazz);
> +    if(outputField == null)  return failure("This function declares zero
> output fields.  A function must declare one output field.", clazz);
> +
> +    // get function body.
> +
> +    CompilationUnit cu;
> +    try {
> +      cu = getClassBody(clazz);
> +    } catch (CompileException | IOException e) {
> +      return failure("Failure while getting class body.", e, clazz);
> +    }
> +
> +    Map<String, String> methods = MethodGrabbingVisitor.getMethods(cu,
> clazz);
> +
> +    // return holder
> +    ValueReference[] ps = params.toArray(new
> ValueReference[params.size()]);
> +    FunctionHolder fh = new FunctionHolder(template.scope(),
> template.nulls(), template.isBinaryCommutative(), template.name(), ps,
> outputField, methods);
> +    return fh;
> +  }
> +
> +
> +
> +  private Java.CompilationUnit getClassBody(Class<?> c) throws
> CompileException, IOException{
> +    String path = c.getName();
> +    path = path.replaceFirst("\\$.*", "");
> +    path = path.replace(".", File.separator);
> +    path = "/" + path + ".java";
> +    URL u = Resources.getResource(FunctionConverter.class, path);
> +    InputSupplier<InputStream> supplier =
> Resources.newInputStreamSupplier(u);
> +    try(InputStream is = supplier.getInput()){
> +      if(is == null){
> +        throw new IOException(String.format("Failure trying to located
> source code for Class %s, tried to read on classpath location %s",
> c.getName(), path));
> +      }
> +      String body = IO.toString(is);
> +
> +      //TODO: Hack to remove annotations so Janino doesn't choke.  Need
> to reconsider this problem...
> +      body =
> body.replaceAll("@(?:Output|Param|FunctionTemplate\\([^\\\\]*?\\))", "");
> +      return new Parser(new Scanner(null, new
> StringReader(body))).parseCompilationUnit();
> +    }
> +
> +  }
> +
> +  @SuppressWarnings("unchecked")
> +  private <T> T getStaticFieldValue(String fieldName, Class<?> valueType,
> Class<T> c) throws NoSuchFieldException, SecurityException,
> IllegalArgumentException, IllegalAccessException{
> +      Field f = valueType.getDeclaredField(fieldName);
> +      Object val = f.get(null);
> +      return (T) val;
> +  }
> +
> +  private static FunctionHolder failure(String message, Throwable t,
> Class<?> clazz, String fieldName){
> +    logger.warn("Failure loading function class {}, field {}. " +
> message, clazz.getName(), fieldName, t);
> +    return null;
> +  }
> +
> +  private FunctionHolder failure(String message, Class<?> clazz, String
> fieldName){
> +    logger.warn("Failure loading function class {}, field {}. " +
> message, clazz.getName(), fieldName);
> +    return null;
> +  }
> +
> +  private FunctionHolder failure(String message, Class<?> clazz){
> +    logger.warn("Failure loading function class {}. " + message,
> clazz.getName());
> +    return null;
> +  }
> +
> +  private FunctionHolder failure(String message, Throwable t, Class<?>
> clazz){
> +    logger.warn("Failure loading function class {}. " + message, t,
> clazz.getName());
> +    return null;
> +  }
> +
> +  private FunctionHolder failure(String message, Class<?> clazz, Field
> field){
> +    return failure(message, clazz, field.getName());
> +  }
> +
> +  public static void main(String[] args) throws Exception{
> +
> +    URL u = Resources.getResource(FunctionConverter.class,
> "/org/apache/drill/exec/expr/fn/impl/MathFunctions.java");
> +    InputStream is = Resources.newInputStreamSupplier(u).getInput();
> +    String s = IO.toString(is);
> +    System.out.println(s);
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
> new file mode 100644
> index 0000000..460169f
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
> @@ -0,0 +1,158 @@
> +package org.apache.drill.exec.expr.fn;
> +
> +import java.util.Arrays;
> +import java.util.Map;
> +
> +import org.apache.drill.common.expression.FunctionCall;
> +import org.apache.drill.common.expression.LogicalExpression;
> +import org.apache.drill.common.types.TypeProtos.DataMode;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.Types;
> +import org.apache.drill.exec.expr.CodeGenerator;
> +import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
> +import org.apache.drill.exec.expr.annotations.FunctionTemplate;
> +import
> org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
> +import
> org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
> +
> +import com.sun.codemodel.JBlock;
> +import com.sun.codemodel.JConditional;
> +import com.sun.codemodel.JExpr;
> +import com.sun.codemodel.JExpression;
> +import com.sun.codemodel.JMod;
> +import com.sun.codemodel.JVar;
> +
> +public class FunctionHolder {
> +
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FunctionImplementationRegistry.class);
> +
> +  private FunctionTemplate.FunctionScope scope;
> +  private FunctionTemplate.NullHandling nullHandling;
> +  private boolean isBinaryCommutative;
> +  private String functionName;
> +  private String evalBody;
> +  private String addBody;
> +  private String setupBody;
> +  private ValueReference[] parameters;
> +  private ValueReference returnValue;
> +
> +  public FunctionHolder(FunctionScope scope, NullHandling nullHandling,
> boolean isBinaryCommutative, String functionName, ValueReference[]
> parameters, ValueReference returnValue, Map<String, String> methods) {
> +    super();
> +    this.scope = scope;
> +    this.nullHandling = nullHandling;
> +    this.isBinaryCommutative = isBinaryCommutative;
> +    this.functionName = functionName;
> +    this.setupBody = methods.get("setup");
> +    this.addBody = methods.get("add");
> +    this.evalBody = methods.get("eval");
> +    this.parameters = parameters;
> +    this.returnValue = returnValue;
> +  }
> +
> +  public HoldingContainer generateEvalBody(CodeGenerator g,
> HoldingContainer[] inputVariables){
> +
> +    g.getBlock().directStatement(String.format("//---- start of eval
> portion of %s function. ----//", functionName));
> +
> +    JBlock sub = new JBlock(true, true);
> +
> +
> +
> +    HoldingContainer out = null;
> +
> +    // add outside null handling if it is defined.
> +    if(nullHandling == NullHandling.NULL_IF_NULL){
> +      JExpression e = null;
> +      for(HoldingContainer v : inputVariables){
> +        if(v.isOptional()){
> +          if(e == null){
> +            e = v.getIsSet();
> +          }else{
> +            e = e.mul(v.getIsSet());
> +          }
> +        }
> +      }
> +
> +      if(e != null){
> +        // if at least one expression must be checked, set up the
> conditional.
> +        returnValue.type =
> returnValue.type.toBuilder().setMode(DataMode.OPTIONAL).build();
> +        out = g.declare(returnValue.type, false);
> +        e = e.eq(JExpr.lit(0));
> +        JConditional jc = sub._if(e);
> +        jc._then().assign(out.getIsSet(), JExpr.lit(0));
> +        sub = jc._else();
> +      }
> +    }
> +
> +    if(out == null) out = g.declare(returnValue.type);
> +
> +    // add the subblock after the out declaration.
> +    g.getBlock().add(sub);
> +
> +    // locally name external blocks.
> +
> +    // internal out value.
> +    JVar internalOutput = sub.decl(JMod.FINAL,
> g.getHolderType(returnValue.type), returnValue.name,
> JExpr._new(g.getHolderType(returnValue.type)));
> +
> +    for(int i =0; i < inputVariables.length; i++){
> +
> +      ValueReference parameter = parameters[i];
> +      HoldingContainer inputVariable = inputVariables[i];
> +      sub.decl(JMod.FINAL, inputVariable.getHolder().type(),
> parameter.name, inputVariable.getHolder());
> +    }
> +
> +    // add function body.
> +    sub.directStatement(evalBody);
> +
> +    sub.assign(out.getHolder(), internalOutput);
> +
> +    g.getBlock().directStatement(String.format("//---- end of eval
> portion of %s function. ----//\n", functionName));
> +    return out;
> +  }
> +
> +  public boolean matches(FunctionCall call){
> +    if(!softCompare(call.getMajorType(), returnValue.type)) return false;
> +    if(call.args.size() != parameters.length) return false;
> +    for(int i =0; i < parameters.length; i++){
> +      ValueReference param = parameters[i];
> +      LogicalExpression arg = call.args.get(i);
> +      if(!softCompare(param.type, arg.getMajorType())) return false;
> +    }
> +
> +    return true;
> +  }
> +
> +  private boolean softCompare(MajorType a, MajorType b){
> +    return Types.softEquals(a, b, nullHandling ==
> NullHandling.NULL_IF_NULL);
> +  }
> +
> +  public String getFunctionName() {
> +    return functionName;
> +  }
> +
> +  public static class ValueReference{
> +    MajorType type;
> +    String name;
> +    public ValueReference(MajorType type, String name) {
> +      super();
> +      this.type = type;
> +      this.name = name;
> +    }
> +    @Override
> +    public String toString() {
> +      return "ValueReference [type=" + type + ", name=" + name + "]";
> +    }
> +
> +
> +  }
> +
> +  @Override
> +  public String toString() {
> +    final int maxLen = 10;
> +    return "FunctionHolder [scope=" + scope + ", isBinaryCommutative=" +
> isBinaryCommutative + ", functionName="
> +        + functionName + ", evalBody=" + evalBody + ", addBody=" +
> addBody + ", setupBody=" + setupBody
> +        + ", parameters="
> +        + (parameters != null ? Arrays.asList(parameters).subList(0,
> Math.min(parameters.length, maxLen)) : null)
> +        + ", returnValue=" + returnValue + "]";
> +  }
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
> new file mode 100644
> index 0000000..5130f2b
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
> @@ -0,0 +1,43 @@
> +package org.apache.drill.exec.expr.fn;
> +
> +import java.util.Set;
> +
> +import org.apache.drill.common.config.DrillConfig;
> +import org.apache.drill.common.expression.FunctionCall;
> +import org.apache.drill.common.util.PathScanner;
> +import org.apache.drill.exec.ExecConstants;
> +import org.apache.drill.exec.expr.DrillFunc;
> +
> +import com.google.common.collect.ArrayListMultimap;
> +
> +public class FunctionImplementationRegistry {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(FunctionImplementationRegistry.class);
> +
> +  private ArrayListMultimap<String, FunctionHolder> methods =
> ArrayListMultimap.create();
> +
> +  public FunctionImplementationRegistry(DrillConfig config){
> +    FunctionConverter converter = new FunctionConverter();
> +    Set<Class<? extends DrillFunc>> providerClasses =
> PathScanner.scanForImplementations(DrillFunc.class,
> config.getStringList(ExecConstants.FUNCTION_PACKAGES));
> +    for (Class<? extends DrillFunc> clazz : providerClasses) {
> +      FunctionHolder holder = converter.getHolder(clazz);
> +      if(holder != null){
> +        methods.put(holder.getFunctionName(), holder);
> +        logger.debug("Registering function {}", holder);
> +      }else{
> +        logger.debug("Unable to initialize function for class {}",
> clazz.getName());
> +      }
> +    }
> +  }
> +
> +  public FunctionHolder getFunction(FunctionCall call){
> +    for(FunctionHolder h : methods.get(call.getDefinition().getName())){
> +      if(h.matches(call)){
> +        return h;
> +      }
> +    }
> +    throw new UnsupportedOperationException(String.format("Unable to find
> matching function implementation for call %s.", call));
> +  }
> +
> +
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> new file mode 100644
> index 0000000..22b9046
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
> @@ -0,0 +1,52 @@
> +package org.apache.drill.exec.expr.fn;
> +
> +import java.io.StringWriter;
> +import java.util.Map;
> +
> +import org.codehaus.janino.Java;
> +import org.codehaus.janino.Java.ClassDeclaration;
> +import org.codehaus.janino.Java.MethodDeclarator;
> +import org.codehaus.janino.util.Traverser;
> +
> +import com.beust.jcommander.internal.Maps;
> +
> +public class MethodGrabbingVisitor{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(MethodGrabbingVisitor.class);
> +
> +  private Class<?> c;
> +  private Map<String, String> methods = Maps.newHashMap();
> +  private ClassFinder classFinder = new ClassFinder();
> +  private boolean captureMethods = false;
> +
> +  private MethodGrabbingVisitor(Class<?> c) {
> +    super();
> +    this.c = c;
> +  }
> +
> +  public class ClassFinder extends Traverser{
> +
> +    @Override
> +    public void traverseClassDeclaration(ClassDeclaration cd) {
> +      captureMethods = c.getName().equals(cd.getClassName());
> +      super.traverseClassDeclaration(cd);
> +    }
> +
> +    @Override
> +    public void traverseMethodDeclarator(MethodDeclarator md) {
> +      if(captureMethods){
> +        StringWriter writer = new StringWriter();
> +        ModifiedUnparseVisitor v = new ModifiedUnparseVisitor(writer);
> +        md.accept(v);
> +        methods.put(md.name, writer.getBuffer().toString());
> +      }
> +    }
> +  }
> +
> +
> +  public static Map<String, String> getMethods(Java.CompilationUnit cu,
> Class<?> c){
> +    MethodGrabbingVisitor visitor = new MethodGrabbingVisitor(c);
> +
>  cu.getPackageMemberTypeDeclarations()[0].accept(visitor.classFinder.comprehensiveVisitor());
> +    return visitor.methods;
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
> new file mode 100644
> index 0000000..72b2008
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparseVisitor.java
> @@ -0,0 +1,891 @@
> +package org.apache.drill.exec.expr.fn;
> +
> +
> +/*
> + *
> + * Modified so that we can avoid printing some things.
> + *
> + * Janino - An embedded Java[TM] compiler
> + *
> + * Copyright (c) 2001-2010, Arno Unkrig
> + * All rights reserved.
> + *
> + * Redistribution and use in source and binary forms, with or without
> modification, are permitted provided that the
> + * following conditions are met:
> + *
> + *    1. Redistributions of source code must retain the above copyright
> notice, this list of conditions and the
> + *       following disclaimer.
> + *    2. Redistributions in binary form must reproduce the above
> copyright notice, this list of conditions and the
> + *       following disclaimer in the documentation and/or other materials
> provided with the distribution.
> + *    3. The name of the author may not be used to endorse or promote
> products derived from this software without
> + *       specific prior written permission.
> + *
> + * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
> IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
> + * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
> PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
> + * THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
> EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
> + * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS
> OF USE, DATA, OR PROFITS; OR BUSINESS
> + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER
> IN CONTRACT, STRICT LIABILITY, OR TORT
> + * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
> OF THIS SOFTWARE, EVEN IF ADVISED OF THE
> + * POSSIBILITY OF SUCH DAMAGE.
> + */
> +
> +
> +import java.io.BufferedReader;
> +import java.io.BufferedWriter;
> +import java.io.FileReader;
> +import java.io.IOException;
> +import java.io.OutputStreamWriter;
> +import java.io.PrintWriter;
> +import java.io.StringReader;
> +import java.io.Writer;
> +import java.util.HashMap;
> +import java.util.HashSet;
> +import java.util.Iterator;
> +import java.util.List;
> +import java.util.Map;
> +import java.util.Set;
> +
> +import org.codehaus.janino.JaninoRuntimeException;
> +import org.codehaus.janino.Java;
> +import org.codehaus.janino.Mod;
> +import org.codehaus.janino.Parser;
> +import org.codehaus.janino.Scanner;
> +import org.codehaus.janino.Visitor.ComprehensiveVisitor;
> +import org.codehaus.janino.util.AutoIndentWriter;
> +
> +/**
> + * A visitor that unparses (un-compiles) an AST to a {@link Writer}. See
> + * {@link #main(String[])} for a usage example.
> + */
> +public class ModifiedUnparseVisitor implements ComprehensiveVisitor {
> +    protected final AutoIndentWriter aiw;
> +    protected final PrintWriter      pw;
> +
> +    /**
> +     * Testing of parsing/unparsing.
> +     * <p>
> +     * Reads compilation units from the files named on the command line
> +     * and unparses them to {@link System#out}.
> +     */
> +    public static void main(String[] args) throws Exception {
> +        Writer w = new BufferedWriter(new OutputStreamWriter(System.out));
> +        for (int i = 0; i < args.length; ++i) {
> +            String fileName = args[i];
> +
> +            // Parse each compilation unit.
> +            FileReader r = new FileReader(fileName);
> +            Java.CompilationUnit cu;
> +            try {
> +                cu = new Parser(new Scanner(fileName,
> r)).parseCompilationUnit();
> +            } finally {
> +                r.close();
> +            }
> +
> +            // Unparse each compilation unit.
> +            ModifiedUnparseVisitor.unparse(cu, w);
> +        }
> +        w.flush();
> +    }
> +
> +    /**
> +     * Unparse the given {@link Java.CompilationUnit} to the given {@link
> Writer}.
> +     */
> +    public static void unparse(Java.CompilationUnit cu, Writer w) {
> +        ModifiedUnparseVisitor uv = new ModifiedUnparseVisitor(w);
> +        uv.unparseCompilationUnit(cu);
> +        uv.close();
> +    }
> +
> +    public ModifiedUnparseVisitor(Writer w) {
> +        this.aiw = new AutoIndentWriter(w);
> +        this.pw = new PrintWriter(this.aiw, true);
> +    }
> +
> +    /**
> +     * Flushes all generated code and closes the {@link Writer} that was
> passed
> +     * to {@link #ModifiedUnparseVisitor(Writer)}.
> +     */
> +    public void close() {
> +        this.pw.close();
> +    }
> +
> +    public void unparseCompilationUnit(Java.CompilationUnit cu) {
> +        if (cu.optionalPackageDeclaration != null) {
> +            this.pw.println();
> +            this.pw.println("package " +
> cu.optionalPackageDeclaration.packageName + ';');
> +        }
> +        if (!cu.importDeclarations.isEmpty()) {
> +            this.pw.println();
> +            for (Iterator it = cu.importDeclarations.iterator();
> it.hasNext();) {
> +                ((Java.CompilationUnit.ImportDeclaration)
> it.next()).accept(this);
> +            }
> +        }
> +        for (Iterator it = cu.packageMemberTypeDeclarations.iterator();
> it.hasNext();) {
> +            this.pw.println();
> +
>  this.unparseTypeDeclaration((Java.PackageMemberTypeDeclaration) it.next());
> +            this.pw.println();
> +        }
> +    }
> +
> +    public void
> visitSingleTypeImportDeclaration(Java.CompilationUnit.SingleTypeImportDeclaration
> stid) {
> +        this.pw.println("import " + Java.join(stid.identifiers, ".") +
> ';');
> +    }
> +    public void
> visitTypeImportOnDemandDeclaration(Java.CompilationUnit.TypeImportOnDemandDeclaration
> tiodd) {
> +        this.pw.println("import " + Java.join(tiodd.identifiers, ".") +
> ".*;");
> +    }
> +    public void
> visitSingleStaticImportDeclaration(Java.CompilationUnit.SingleStaticImportDeclaration
> ssid) {
> +        this.pw.println("import static " + Java.join(ssid.identifiers,
> ".") + ';');
> +    }
> +    public void
> visitStaticImportOnDemandDeclaration(Java.CompilationUnit.StaticImportOnDemandDeclaration
> siodd) {
> +        this.pw.println("import static " + Java.join(siodd.identifiers,
> ".") + ".*;");
> +    }
> +
> +    public void visitLocalClassDeclaration(Java.LocalClassDeclaration
> lcd) {
> +        this.unparseNamedClassDeclaration(lcd);
> +    }
> +    public void visitMemberClassDeclaration(Java.MemberClassDeclaration
> mcd) {
> +        this.unparseNamedClassDeclaration(mcd);
> +    }
> +    public void
> visitMemberInterfaceDeclaration(Java.MemberInterfaceDeclaration mid) {
> +        this.unparseInterfaceDeclaration(mid);
> +    }
> +    public void
> visitPackageMemberClassDeclaration(Java.PackageMemberClassDeclaration pmcd)
> {
> +        this.unparseNamedClassDeclaration(pmcd);
> +    }
> +    public void
> visitPackageMemberInterfaceDeclaration(Java.PackageMemberInterfaceDeclaration
> pmid) {
> +        this.unparseInterfaceDeclaration(pmid);
> +    }
> +    public void visitConstructorDeclarator(Java.ConstructorDeclarator cd)
> {
> +        this.unparseDocComment(cd);
> +        this.unparseModifiers(cd.modifiers);
> +        Java.ClassDeclaration declaringClass = cd.getDeclaringClass();
> +        this.pw.print(
> +            declaringClass instanceof Java.NamedClassDeclaration
> +            ? ((Java.NamedClassDeclaration) declaringClass).name
> +            : "UNNAMED"
> +        );
> +        this.unparseFunctionDeclaratorRest(cd);
> +        this.pw.print(' ');
> +        if (cd.optionalConstructorInvocation != null) {
> +            this.pw.println('{');
> +            this.pw.print(AutoIndentWriter.INDENT);
> +            this.unparseBlockStatement(cd.optionalConstructorInvocation);
> +            this.pw.println(';');
> +
> +            if (!cd.optionalStatements.isEmpty()) {
> +                this.pw.println();
> +                this.unparseStatements(cd.optionalStatements);
> +            }
> +            this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +        } else
> +        if (cd.optionalStatements.isEmpty()) {
> +            this.pw.print("{}");
> +        } else
> +        {
> +            this.pw.println('{');
> +            this.pw.print(AutoIndentWriter.INDENT);
> +            this.unparseStatements(cd.optionalStatements);
> +            this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +        }
> +    }
> +    public void visitMethodDeclarator(Java.MethodDeclarator md) {
> +        if (md.optionalStatements == null) {
> +            this.pw.print(';');
> +        } else
> +        if (md.optionalStatements.isEmpty()) {
> +            this.pw.print(" ");
> +        } else
> +        {
> +            this.pw.print(AutoIndentWriter.INDENT);
> +            this.unparseStatements(md.optionalStatements);
> +            this.pw.print(AutoIndentWriter.UNINDENT);
> +        }
> +    }
> +    public void visitFieldDeclaration(Java.FieldDeclaration fd) {
> +        this.unparseDocComment(fd);
> +        this.unparseModifiers(fd.modifiers);
> +        this.unparseType(fd.type);
> +        this.pw.print(' ');
> +        for (int i = 0; i < fd.variableDeclarators.length; ++i) {
> +            if (i > 0) this.pw.print(", ");
> +            this.unparseVariableDeclarator(fd.variableDeclarators[i]);
> +        }
> +        this.pw.print(';');
> +    }
> +    public void visitInitializer(Java.Initializer i) {
> +        if (i.statiC) this.pw.print("static ");
> +        this.unparseBlockStatement(i.block);
> +    }
> +    public void visitBlock(Java.Block b) {
> +        if (b.statements.isEmpty()) {
> +            this.pw.print("{}");
> +            return;
> +        }
> +        this.pw.println('{');
> +        this.pw.print(AutoIndentWriter.INDENT);
> +        this.unparseStatements(b.statements);
> +        this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +    }
> +
> +    private void unparseStatements(List statements) {
> +        int state = -1;
> +        for (Iterator it = statements.iterator(); it.hasNext();) {
> +            Java.BlockStatement bs = (Java.BlockStatement) it.next();
> +            int x = (
> +                bs instanceof Java.Block                             ? 1 :
> +                bs instanceof Java.LocalClassDeclarationStatement    ? 2 :
> +                bs instanceof Java.LocalVariableDeclarationStatement ? 3 :
> +                bs instanceof Java.SynchronizedStatement             ? 4 :
> +                99
> +            );
> +            if (state != -1 && state != x)
> this.pw.println(AutoIndentWriter.CLEAR_TABULATORS);
> +            state = x;
> +
> +            this.unparseBlockStatement(bs);
> +            this.pw.println();
> +        }
> +    }
> +    public void visitBreakStatement(Java.BreakStatement bs) {
> +        this.pw.print("break");
> +        if (bs.optionalLabel != null) this.pw.print(' ' +
> bs.optionalLabel);
> +        this.pw.print(';');
> +    }
> +    public void visitContinueStatement(Java.ContinueStatement cs) {
> +        this.pw.print("continue");
> +        if (cs.optionalLabel != null) this.pw.print(' ' +
> cs.optionalLabel);
> +        this.pw.print(';');
> +    }
> +    public void visitDoStatement(Java.DoStatement ds) {
> +        this.pw.print("do ");
> +        this.unparseBlockStatement(ds.body);
> +        this.pw.print("while (");
> +        this.unparse(ds.condition);
> +        this.pw.print(");");
> +    }
> +    public void visitEmptyStatement(Java.EmptyStatement es) {
> +        this.pw.print(';');
> +    }
> +    public void visitExpressionStatement(Java.ExpressionStatement es) {
> +        this.unparse(es.rvalue);
> +        this.pw.print(';');
> +    }
> +    public void visitForStatement(Java.ForStatement fs) {
> +        this.pw.print("for (");
> +        if (fs.optionalInit != null) {
> +            this.unparseBlockStatement(fs.optionalInit);
> +        } else {
> +            this.pw.print(';');
> +        }
> +        if (fs.optionalCondition != null) {
> +            this.pw.print(' ');
> +            this.unparse(fs.optionalCondition);
> +        }
> +        this.pw.print(';');
> +        if (fs.optionalUpdate != null) {
> +            this.pw.print(' ');
> +            for (int i = 0; i < fs.optionalUpdate.length; ++i) {
> +                if (i > 0) this.pw.print(", ");
> +                this.unparse(fs.optionalUpdate[i]);
> +            }
> +        }
> +        this.pw.print(") ");
> +        this.unparseBlockStatement(fs.body);
> +    }
> +    public void visitIfStatement(Java.IfStatement is) {
> +        this.pw.print("if (");
> +        this.unparse(is.condition);
> +        this.pw.print(") ");
> +        this.unparseBlockStatement(is.thenStatement);
> +        if (is.optionalElseStatement != null) {
> +            this.pw.println(" else");
> +            this.unparseBlockStatement(is.optionalElseStatement);
> +        }
> +    }
> +    public void visitLabeledStatement(Java.LabeledStatement ls) {
> +        this.pw.println(ls.label + ':');
> +        this.unparseBlockStatement(ls.body);
> +    }
> +    public void
> visitLocalClassDeclarationStatement(Java.LocalClassDeclarationStatement
> lcds) {
> +        this.unparseTypeDeclaration(lcds.lcd);
> +    }
> +    public void
> visitLocalVariableDeclarationStatement(Java.LocalVariableDeclarationStatement
> lvds) {
> +        this.unparseModifiers(lvds.modifiers);
> +        this.unparseType(lvds.type);
> +        this.pw.print(' ');
> +        this.pw.print(AutoIndentWriter.TABULATOR);
> +        this.unparseVariableDeclarator(lvds.variableDeclarators[0]);
> +        for (int i = 1; i < lvds.variableDeclarators.length; ++i) {
> +            this.pw.print(", ");
> +            this.unparseVariableDeclarator(lvds.variableDeclarators[i]);
> +        }
> +        this.pw.print(';');
> +    }
> +    public void visitReturnStatement(Java.ReturnStatement rs) {
> +        this.pw.print("return");
> +        if (rs.optionalReturnValue != null) {
> +            this.pw.print(' ');
> +            this.unparse(rs.optionalReturnValue);
> +        }
> +        this.pw.print(';');
> +    }
> +    public void visitSwitchStatement(Java.SwitchStatement ss) {
> +        this.pw.print("switch (");
> +        this.unparse(ss.condition);
> +        this.pw.println(") {");
> +        for (Iterator it = ss.sbsgs.iterator(); it.hasNext();) {
> +            Java.SwitchStatement.SwitchBlockStatementGroup sbgs = (
> +                (Java.SwitchStatement.SwitchBlockStatementGroup) it.next()
> +            );
> +            this.pw.print(AutoIndentWriter.UNINDENT);
> +            try {
> +                for (Iterator it2 = sbgs.caseLabels.iterator();
> it2.hasNext();) {
> +                    Java.Rvalue rv = (Java.Rvalue) it2.next();
> +                    this.pw.print("case ");
> +                    this.unparse(rv);
> +                    this.pw.println(':');
> +                }
> +                if (sbgs.hasDefaultLabel) this.pw.println("default:");
> +            } finally {
> +                this.pw.print(AutoIndentWriter.INDENT);
> +            }
> +            for (Iterator it2 = sbgs.blockStatements.iterator();
> it2.hasNext();) {
> +                this.unparseBlockStatement((Java.BlockStatement)
> it2.next());
> +                this.pw.println();
> +            }
> +        }
> +        this.pw.print('}');
> +    }
> +    public void visitSynchronizedStatement(Java.SynchronizedStatement ss)
> {
> +        this.pw.print("synchronized (");
> +        this.unparse(ss.expression);
> +        this.pw.print(") ");
> +        this.unparseBlockStatement(ss.body);
> +    }
> +    public void visitThrowStatement(Java.ThrowStatement ts) {
> +        this.pw.print("throw ");
> +        this.unparse(ts.expression);
> +        this.pw.print(';');
> +    }
> +    public void visitTryStatement(Java.TryStatement ts) {
> +        this.pw.print("try ");
> +        this.unparseBlockStatement(ts.body);
> +        for (Iterator it = ts.catchClauses.iterator(); it.hasNext();) {
> +            Java.CatchClause cc = (Java.CatchClause) it.next();
> +            this.pw.print(" catch (");
> +            this.unparseFormalParameter(cc.caughtException);
> +            this.pw.print(") ");
> +            this.unparseBlockStatement(cc.body);
> +        }
> +        if (ts.optionalFinally != null) {
> +            this.pw.print(" finally ");
> +            this.unparseBlockStatement(ts.optionalFinally);
> +        }
> +    }
> +    public void visitWhileStatement(Java.WhileStatement ws) {
> +        this.pw.print("while (");
> +        this.unparse(ws.condition);
> +        this.pw.print(") ");
> +        this.unparseBlockStatement(ws.body);
> +    }
> +    public void unparseVariableDeclarator(Java.VariableDeclarator vd) {
> +        this.pw.print(vd.name);
> +        for (int i = 0; i < vd.brackets; ++i) this.pw.print("[]");
> +        if (vd.optionalInitializer != null) {
> +            this.pw.print(" = ");
> +            this.unparseArrayInitializerOrRvalue(vd.optionalInitializer);
> +        }
> +    }
> +    public void
> unparseFormalParameter(Java.FunctionDeclarator.FormalParameter fp) {
> +        if (fp.finaL) this.pw.print("final ");
> +        this.unparseType(fp.type);
> +        this.pw.print(" " + AutoIndentWriter.TABULATOR + fp.name);
> +    }
> +    public void visitMethodInvocation(Java.MethodInvocation mi) {
> +        if (mi.optionalTarget != null) {
> +            this.unparseLhs(mi.optionalTarget, ".");
> +            this.pw.print('.');
> +        }
> +        this.pw.print(mi.methodName);
> +        this.unparseFunctionInvocationArguments(mi.arguments);
> +    }
> +    public void
> visitAlternateConstructorInvocation(Java.AlternateConstructorInvocation
> aci) {
> +        this.pw.print("this");
> +        this.unparseFunctionInvocationArguments(aci.arguments);
> +    }
> +    public void
> visitSuperConstructorInvocation(Java.SuperConstructorInvocation sci) {
> +        if (sci.optionalQualification != null) {
> +            this.unparseLhs(sci.optionalQualification, ".");
> +            this.pw.print('.');
> +        }
> +        this.pw.print("super");
> +        this.unparseFunctionInvocationArguments(sci.arguments);
> +    }
> +    public void visitNewClassInstance(Java.NewClassInstance nci) {
> +        if (nci.optionalQualification != null) {
> +            this.unparseLhs(nci.optionalQualification, ".");
> +            this.pw.print('.');
> +        }
> +        this.pw.print("new " + nci.type.toString());
> +        this.unparseFunctionInvocationArguments(nci.arguments);
> +    }
> +    public void visitAssignment(Java.Assignment a) {
> +        this.unparseLhs(a.lhs, a.operator);
> +        this.pw.print(' ' + a.operator + ' ');
> +        this.unparseRhs(a.rhs, a.operator);
> +    }
> +    public void visitAmbiguousName(Java.AmbiguousName an) {
> this.pw.print(an.toString()); }
> +    public void visitArrayAccessExpression(Java.ArrayAccessExpression
> aae) {
> +        this.unparseLhs(aae.lhs, "[ ]");
> +        this.pw.print('[');
> +        this.unparse(aae.index);
> +        this.pw.print(']');
> +    }
> +    public void visitArrayLength(Java.ArrayLength al) {
> +        this.unparseLhs(al.lhs, ".");
> +        this.pw.print(".length");
> +    }
> +    public void visitArrayType(Java.ArrayType at) {
> +        this.unparseType(at.componentType);
> +        this.pw.print("[]");
> +    }
> +    public void visitBasicType(Java.BasicType bt) {
> +        this.pw.print(bt.toString());
> +    }
> +    public void visitBinaryOperation(Java.BinaryOperation bo) {
> +        this.unparseLhs(bo.lhs, bo.op);
> +        this.pw.print(' ' + bo.op + ' ');
> +        this.unparseRhs(bo.rhs, bo.op);
> +    }
> +    public void visitCast(Java.Cast c) {
> +        this.pw.print('(');
> +        this.unparseType(c.targetType);
> +        this.pw.print(") ");
> +        this.unparseRhs(c.value, "cast");
> +    }
> +    public void visitClassLiteral(Java.ClassLiteral cl) {
> +        this.unparseType(cl.type);
> +        this.pw.print(".class");
> +    }
> +    public void visitConditionalExpression(Java.ConditionalExpression ce)
> {
> +        this.unparseLhs(ce.lhs, "?:");
> +        this.pw.print(" ? ");
> +        this.unparseLhs(ce.mhs, "?:");
> +        this.pw.print(" : ");
> +        this.unparseRhs(ce.rhs, "?:");
> +    }
> +    public void visitCrement(Java.Crement c) {
> +        if (c.pre) {
> +            this.pw.print(c.operator);
> +            this.unparseUnaryOperation(c.operand, c.operator + "x");
> +        } else
> +        {
> +            this.unparseUnaryOperation(c.operand, "x" + c.operator);
> +            this.pw.print(c.operator);
> +        }
> +    }
> +    public void visitFieldAccess(Java.FieldAccess fa) {
> +        this.unparseLhs(fa.lhs, ".");
> +        this.pw.print('.' + fa.field.getName());
> +    }
> +    public void visitFieldAccessExpression(Java.FieldAccessExpression
> fae) {
> +        this.unparseLhs(fae.lhs, ".");
> +        this.pw.print('.' + fae.fieldName);
> +    }
> +    public void
> visitSuperclassFieldAccessExpression(Java.SuperclassFieldAccessExpression
> scfae) {
> +        if (scfae.optionalQualification != null) {
> +            this.unparseType(scfae.optionalQualification);
> +            this.pw.print(".super." + scfae.fieldName);
> +        } else
> +        {
> +            this.pw.print("super." + scfae.fieldName);
> +        }
> +    }
> +    public void visitInstanceof(Java.Instanceof io) {
> +        this.unparseLhs(io.lhs, "instanceof");
> +        this.pw.print(" instanceof ");
> +        this.unparseType(io.rhs);
> +    }
> +    public void visitLiteral(Java.Literal l) {
> this.pw.print(l.toString()); }
> +    public void visitLocalVariableAccess(Java.LocalVariableAccess lva) {
> this.pw.print(lva.toString()); }
> +    public void visitNewArray(Java.NewArray na) {
> +        this.pw.print("new ");
> +        this.unparseType(na.type);
> +        for (int i = 0; i < na.dimExprs.length; ++i) {
> +            this.pw.print('[');
> +            this.unparse(na.dimExprs[i]);
> +            this.pw.print(']');
> +        }
> +        for (int i = 0; i < na.dims; ++i) {
> +            this.pw.print("[]");
> +        }
> +    }
> +    public void visitNewInitializedArray(Java.NewInitializedArray nai) {
> +        this.pw.print("new ");
> +        this.unparseType(nai.arrayType);
> +        this.pw.print(" ");
> +        this.unparseArrayInitializerOrRvalue(nai.arrayInitializer);
> +    }
> +    public void visitPackage(Java.Package p) {
> this.pw.print(p.toString()); }
> +    public void visitParameterAccess(Java.ParameterAccess pa) {
> this.pw.print(pa.toString()); }
> +    public void visitQualifiedThisReference(Java.QualifiedThisReference
> qtr) {
> +        this.unparseType(qtr.qualification);
> +        this.pw.print(".this");
> +    }
> +    public void visitReferenceType(Java.ReferenceType rt) {
> this.pw.print(rt.toString()); }
> +    public void visitRvalueMemberType(Java.RvalueMemberType rmt) {
> this.pw.print(rmt.toString()); }
> +    public void visitSimpleType(Java.SimpleType st) {
> this.pw.print(st.toString()); }
> +    public void
> visitSuperclassMethodInvocation(Java.SuperclassMethodInvocation smi) {
> +        this.pw.print("super." + smi.methodName);
> +        this.unparseFunctionInvocationArguments(smi.arguments);
> +    }
> +    public void visitThisReference(Java.ThisReference tr) {
> +        this.pw.print("this");
> +    }
> +    public void visitUnaryOperation(Java.UnaryOperation uo) {
> +        this.pw.print(uo.operator);
> +        this.unparseUnaryOperation(uo.operand, uo.operator + "x");
> +    }
> +    public void visitParenthesizedExpression(Java.ParenthesizedExpression
> pe) {
> +        this.pw.print('(');
> +        this.unparse(pe.value);
> +        this.pw.print(')');
> +    }
> +
> +    // Helpers
> +
> +    private void unparseBlockStatement(Java.BlockStatement
> blockStatement) {
> +        blockStatement.accept(this);
> +    }
> +
> +    private void unparseTypeDeclaration(Java.TypeDeclaration
> typeDeclaration) {
> +        typeDeclaration.accept(this);
> +    }
> +
> +    private void unparseType(Java.Type type) {
> +        ((Java.Atom) type).accept(this);
> +    }
> +
> +    private void unparse(Java.Atom operand) {
> +        operand.accept(this);
> +    }
> +
> +    /**
> +     * Iff the <code>operand</code> is unnatural for the
> <code>unaryOperator</code>, enclose the
> +     * <code>operand</code> in parentheses. Example: "a+b" is an
> unnatural operand for unary "!x".
> +     *
> +     * @param unaryOperator ++x --x +x -x ~x !x x++ x--
> +     */
> +    private void unparseUnaryOperation(Java.Rvalue operand, String
> unaryOperator) {
> +        int cmp = ModifiedUnparseVisitor.comparePrecedence(unaryOperator,
> operand);
> +        this.unparse(operand, cmp < 0);
> +    }
> +
> +    /**
> +     * Iff the <code>lhs</code> is unnatural for the
> <code>binaryOperator</code>, enclose the
> +     * <code>lhs</code> in parentheses. Example: "a+b" is an unnatural
> lhs for operator "*".
> +     *
> +     * @param binaryOperator = +=... ?: || && | ^ & == != < > <= >=
> instanceof << >> >>> + - * / % cast
> +     */
> +
> +    private void unparseLhs(Java.Atom lhs, String binaryOperator) {
> +        int cmp =
> ModifiedUnparseVisitor.comparePrecedence(binaryOperator, lhs);
> +        this.unparse(lhs, cmp < 0 || (cmp == 0 &&
> ModifiedUnparseVisitor.isLeftAssociate(binaryOperator)));
> +    }
> +
> +
> +    /**
> +     * Iff the <code>rhs</code> is unnatural for the
> <code>binaryOperator</code>, enclose the
> +     * <code>rhs</code> in parentheses. Example: "a+b" is an unnatural
> rhs for operator "*".
> +     */
> +    private void unparseRhs(Java.Rvalue rhs, String binaryOperator) {
> +        int cmp =
> ModifiedUnparseVisitor.comparePrecedence(binaryOperator, rhs);
> +        this.unparse(rhs, cmp < 0 || (cmp == 0 &&
> ModifiedUnparseVisitor.isRightAssociate(binaryOperator)));
> +    }
> +
> +    private void unparse(Java.Atom operand, boolean natural) {
> +        if (!natural) this.pw.print("((( ");
> +        this.unparse(operand);
> +        if (!natural) this.pw.print(" )))");
> +    }
> +
> +    /**
> +     * Return true iff operator is right associative e.g. <code>a = b =
> c</code> evaluates as
> +     * <code>a = (b = c)</code>.
> +     *
> +     * @return Return true iff operator is right associative
> +     */
> +    private static boolean isRightAssociate(String op) {
> +        return
> ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS.contains(op);
> +    }
> +
> +    /**
> +     * Return true iff operator is left associative e.g. <code>a - b -
> c</code> evaluates as
> +     * <code>(a - b) - c</code>.
> +     *
> +     * @return Return true iff operator is left associative
> +     */
> +    private static boolean isLeftAssociate(String op) {
> +        return
> ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS.contains(op);
> +    }
> +
> +    /**
> +     * Returns a value
> +     * <ul>
> +     *   <li>&lt; 0 iff the <code>operator</code> has lower precedence
> than the <code>operand</code>
> +     *   <li>==; 0 iff the <code>operator</code> has equal precedence
> than the <code>operand</code>
> +     *   <li>&gt; 0 iff the <code>operator</code> has higher precedence
> than the <code>operand</code>
> +     * </ul>
> +     */
> +    private static int comparePrecedence(String operator, Java.Atom
> operand) {
> +        if (operand instanceof Java.BinaryOperation) {
> +            return (
> +                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> +                -
> ModifiedUnparseVisitor.getOperatorPrecedence(((Java.BinaryOperation)
> operand).op)
> +            );
> +        } else
> +        if (operand instanceof Java.UnaryOperation) {
> +            return (
> +                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> +                -
> ModifiedUnparseVisitor.getOperatorPrecedence(((Java.UnaryOperation)
> operand).operator + "x")
> +            );
> +        } else
> +        if (operand instanceof Java.ConditionalExpression) {
> +            return ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> - ModifiedUnparseVisitor.getOperatorPrecedence("?:");
> +        } else
> +        if (operand instanceof Java.Instanceof) {
> +            return ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> - ModifiedUnparseVisitor.getOperatorPrecedence("instanceof");
> +        } else
> +        if (operand instanceof Java.Cast) {
> +            return ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> - ModifiedUnparseVisitor.getOperatorPrecedence("cast");
> +        } else
> +        if (operand instanceof Java.MethodInvocation || operand
> instanceof Java.FieldAccess) {
> +            return ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> - ModifiedUnparseVisitor.getOperatorPrecedence(".");
> +        } else
> +        if (operand instanceof Java.NewArray) {
> +            return ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> - ModifiedUnparseVisitor.getOperatorPrecedence("new");
> +        } else
> +        if (operand instanceof Java.Crement) {
> +            Java.Crement c = (Java.Crement) operand;
> +            return (
> +                ModifiedUnparseVisitor.getOperatorPrecedence(operator)
> +                - ModifiedUnparseVisitor.getOperatorPrecedence(c.pre ?
> c.operator + "x" : "x" + c.operator)
> +            );
> +        } else
> +        {
> +            // All other rvalues (e.g. literal) have higher precedence
> than any operator.
> +            return -1;
> +        }
> +    }
> +    private static int getOperatorPrecedence(String operator) {
> +        return ((Integer)
> ModifiedUnparseVisitor.OPERATOR_PRECEDENCE.get(operator)).intValue();
> +    }
> +    private static final Set LEFT_ASSOCIATIVE_OPERATORS = new HashSet();
> +    private static final Set RIGHT_ASSOCIATIVE_OPERATORS = new HashSet();
> +    private static final Set UNARY_OPERATORS = new HashSet();
> +    private static final Map OPERATOR_PRECEDENCE = new HashMap();
> +    static {
> +        Object[] ops = {
> +            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "=",
> "*=", "/=", "%=", "+=", "-=", "<<=", ">>=", ">>>=",
> +                                                        "&=", "^=", "|=",
> +            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "?:",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "||",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "&&",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "|",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "^",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "&",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "==",
> "!=",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "<", ">",
> "<=", ">=", "instanceof",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "<<",
> ">>", ">>>",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "+", "-",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "*", "/",
> "%",
> +            ModifiedUnparseVisitor.RIGHT_ASSOCIATIVE_OPERATORS, "cast",
> +            ModifiedUnparseVisitor.UNARY_OPERATORS,             "++x",
> "--x", "+x", "-x", "~x", "!x",
> +            ModifiedUnparseVisitor.UNARY_OPERATORS,             "x++",
> "x--",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  "new",
> +            ModifiedUnparseVisitor.LEFT_ASSOCIATIVE_OPERATORS,  ".", "[
> ]",
> +        };
> +        int precedence = 0;
> +        LOOP1: for (int i = 0;;) {
> +            Set s = (Set) ops[i++];
> +            Integer pi = new Integer(++precedence);
> +            for (;;) {
> +                if (i == ops.length) break LOOP1;
> +                if (!(ops[i] instanceof String)) break;
> +                String op = (String) ops[i++];
> +                s.add(op);
> +                ModifiedUnparseVisitor.OPERATOR_PRECEDENCE.put(op, pi);
> +            }
> +        }
> +    }
> +
> +    private void unparseNamedClassDeclaration(Java.NamedClassDeclaration
> ncd) {
> +        this.unparseDocComment(ncd);
> +        this.unparseModifiers(ncd.getModifiers());
> +        this.pw.print("class " + ncd.name);
> +        if (ncd.optionalExtendedType != null) {
> +            this.pw.print(" extends ");
> +            this.unparseType(ncd.optionalExtendedType);
> +        }
> +        if (ncd.implementedTypes.length > 0) this.pw.print(" implements "
> + Java.join(ncd.implementedTypes, ", "));
> +        this.pw.println(" {");
> +        this.pw.print(AutoIndentWriter.INDENT);
> +        this.unparseClassDeclarationBody(ncd);
> +        this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +    }
> +    private void
> unparseArrayInitializerOrRvalue(Java.ArrayInitializerOrRvalue aiorv) {
> +        if (aiorv instanceof Java.Rvalue) {
> +            this.unparse((Java.Rvalue) aiorv);
> +        } else
> +        if (aiorv instanceof Java.ArrayInitializer) {
> +            Java.ArrayInitializer ai = (Java.ArrayInitializer) aiorv;
> +            if (ai.values.length == 0) {
> +                this.pw.print("{}");
> +            } else
> +            {
> +                this.pw.print("{ ");
> +                this.unparseArrayInitializerOrRvalue(ai.values[0]);
> +                for (int i = 1; i < ai.values.length; ++i) {
> +                    this.pw.print(", ");
> +                    this.unparseArrayInitializerOrRvalue(ai.values[i]);
> +                }
> +                this.pw.print(" }");
> +            }
> +        } else
> +        {
> +            throw new JaninoRuntimeException(
> +                "Unexpected array initializer or rvalue class "
> +                + aiorv.getClass().getName()
> +            );
> +        }
> +    }
> +
> +    public void
> visitAnonymousClassDeclaration(Java.AnonymousClassDeclaration acd) {
> +        this.unparseType(acd.baseType);
> +        this.pw.println(" {");
> +        this.pw.print(AutoIndentWriter.INDENT);
> +        this.unparseClassDeclarationBody(acd);
> +        this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +    }
> +    public void
> visitNewAnonymousClassInstance(Java.NewAnonymousClassInstance naci) {
> +        if (naci.optionalQualification != null) {
> +            this.unparseLhs(naci.optionalQualification, ".");
> +            this.pw.print('.');
> +        }
> +        this.pw.print("new " +
> naci.anonymousClassDeclaration.baseType.toString() + '(');
> +        for (int i = 0; i < naci.arguments.length; ++i) {
> +            if (i > 0) this.pw.print(", ");
> +            this.unparse(naci.arguments[i]);
> +        }
> +        this.pw.println(") {");
> +        this.pw.print(AutoIndentWriter.INDENT);
> +        this.unparseClassDeclarationBody(naci.anonymousClassDeclaration);
> +        this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +    }
> +    // Multi-line!
> +    private void unparseClassDeclarationBody(Java.ClassDeclaration cd) {
> +        for (Iterator it = cd.constructors.iterator(); it.hasNext();) {
> +            this.pw.println();
> +            ((Java.ConstructorDeclarator) it.next()).accept(this);
> +            this.pw.println();
> +        }
> +        this.unparseAbstractTypeDeclarationBody(cd);
> +        for (Iterator it =
> cd.variableDeclaratorsAndInitializers.iterator(); it.hasNext();) {
> +            this.pw.println();
> +            ((Java.TypeBodyDeclaration) it.next()).accept(this);
> +            this.pw.println();
> +        }
> +    }
> +    private void unparseInterfaceDeclaration(Java.InterfaceDeclaration
> id) {
> +        this.unparseDocComment(id);
> +        this.unparseModifiers(id.getModifiers());
> +        //make sure we print "interface", even if it wasn't in the
> modifiers
> +        if ((id.getModifiers() & Mod.INTERFACE) == 0) {
> +            this.pw.print("interface ");
> +        }
> +        this.pw.print(id.name);
> +        if (id.extendedTypes.length > 0) this.pw.print(" extends " +
> Java.join(id.extendedTypes, ", "));
> +        this.pw.println(" {");
> +        this.pw.print(AutoIndentWriter.INDENT);
> +        this.unparseAbstractTypeDeclarationBody(id);
> +        for (Iterator it = id.constantDeclarations.iterator();
> it.hasNext();) {
> +            ((Java.TypeBodyDeclaration) it.next()).accept(this);
> +            this.pw.println();
> +        }
> +        this.pw.print(AutoIndentWriter.UNINDENT + "}");
> +    }
> +    // Multi-line!
> +    private void
> unparseAbstractTypeDeclarationBody(Java.AbstractTypeDeclaration atd) {
> +        for (Iterator it = atd.getMethodDeclarations().iterator();
> it.hasNext();) {
> +            this.pw.println();
> +            ((Java.MethodDeclarator) it.next()).accept(this);
> +            this.pw.println();
> +        }
> +        for (Iterator it = atd.getMemberTypeDeclarations().iterator();
> it.hasNext();) {
> +            this.pw.println();
> +            ((Java.TypeBodyDeclaration) it.next()).accept(this);
> +            this.pw.println();
> +        }
> +    }
> +    private void unparseFunctionDeclaratorRest(Java.FunctionDeclarator
> fd) {
> +        boolean big = fd.formalParameters.length >= 4;
> +        this.pw.print('(');
> +        if (big) { this.pw.println();
> this.pw.print(AutoIndentWriter.INDENT); }
> +        for (int i = 0; i < fd.formalParameters.length; ++i) {
> +            if (i > 0) {
> +                if (big) {
> +                    this.pw.println(',');
> +                } else
> +                {
> +                    this.pw.print(", ");
> +                }
> +            }
> +            this.unparseFormalParameter(fd.formalParameters[i]);
> +        }
> +        if (big) { this.pw.println();
> this.pw.print(AutoIndentWriter.UNINDENT); }
> +        this.pw.print(')');
> +        if (fd.thrownExceptions.length > 0) this.pw.print(" throws " +
> Java.join(fd.thrownExceptions, ", "));
> +    }
> +    private void unparseDocComment(Java.DocCommentable dc) {
> +        String optionalDocComment = dc.getDocComment();
> +        if (optionalDocComment != null) {
> +            this.pw.print("/**");
> +            BufferedReader br = new BufferedReader(new
> StringReader(optionalDocComment));
> +            for (;;) {
> +                String line;
> +                try {
> +                    line = br.readLine();
> +                } catch (IOException e) {
> +                    throw new JaninoRuntimeException();
> +                }
> +                if (line == null) break;
> +                this.pw.println(line);
> +                this.pw.print(" *");
> +            }
> +            this.pw.println("/");
> +        }
> +    }
> +    private void unparseModifiers(short modifiers) {
> +        if (modifiers != 0) {
> +            this.pw.print(Mod.shortToString(modifiers) + ' ');
> +        }
> +    }
> +    private void unparseFunctionInvocationArguments(Java.Rvalue[]
> arguments) {
> +        boolean big = arguments.length >= 5;
> +        this.pw.print('(');
> +        if (big) { this.pw.println();
> this.pw.print(AutoIndentWriter.INDENT); }
> +        for (int i = 0; i < arguments.length; ++i) {
> +            if (i > 0) {
> +                if (big) {
> +                    this.pw.println(',');
> +                } else
> +                {
> +                    this.pw.print(", ");
> +                }
> +            }
> +            this.unparse(arguments[i]);
> +        }
> +        if (big) { this.pw.println();
> this.pw.print(AutoIndentWriter.UNINDENT); }
> +        this.pw.print(')');
> +    }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> new file mode 100644
> index 0000000..702a0ad
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
> @@ -0,0 +1,48 @@
> +package org.apache.drill.exec.expr.fn.impl;
> +
> +import org.apache.drill.exec.expr.DrillFunc;
> +import org.apache.drill.exec.expr.annotations.FunctionTemplate;
> +import
> org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
> +import
> org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
> +import org.apache.drill.exec.expr.annotations.Output;
> +import org.apache.drill.exec.expr.annotations.Param;
> +import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.IntHolder;
> +import
> org.apache.drill.exec.expr.holders.ValueHolderImplmenetations.LongHolder;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +public class MathFunctions{
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(MathFunctions.class);
> +
> +  private MathFunctions(){}
> +
> +  @FunctionTemplate(name = "add", scope = FunctionScope.SIMPLE, nulls =
> NullHandling.NULL_IF_NULL)
> +  public static class Add1 implements DrillFunc{
> +
> +    @Param IntHolder left;
> +    @Param IntHolder right;
> +    @Output IntHolder out;
> +
> +    public void setup(RecordBatch b){}
> +
> +    public void eval(){
> +      out.value = left.value + right.value;
> +    }
> +
> +  }
> +
> +  @FunctionTemplate(name = "add", scope = FunctionScope.SIMPLE, nulls =
> NullHandling.NULL_IF_NULL)
> +  public static class LongAdd1 implements DrillFunc{
> +
> +    @Param LongHolder left;
> +    @Param LongHolder right;
> +    @Output LongHolder out;
> +
> +    public void setup(RecordBatch b){}
> +
> +    public void eval(){
> +      out.value = left.value + right.value;
> +    }
> +
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
> new file mode 100644
> index 0000000..0ef2e04
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolder.java
> @@ -0,0 +1,4 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +public interface ValueHolder {
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> new file mode 100644
> index 0000000..adf97fe
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/ValueHolderImplmenetations.java
> @@ -0,0 +1,44 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +import org.apache.drill.common.types.TypeProtos.MajorType;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
> +import org.apache.drill.common.types.Types;
> +
> +public class ValueHolderImplmenetations {
> +
> +  private ValueHolderImplmenetations(){}
> +
> +  public final static class BooleanHolder implements ValueHolder{
> +    public static final MajorType TYPE =
> Types.required(MinorType.BOOLEAN);
> +    public int value;
> +  }
> +
> +  public final static class NullableBooleanHolder implements ValueHolder {
> +    public static final MajorType TYPE =
> Types.optional(MinorType.BOOLEAN);
> +    public int value;
> +    public int isSet;
> +  }
> +
> +  public final static class IntHolder implements ValueHolder{
> +    public static final MajorType TYPE = Types.required(MinorType.INT);
> +    public int value;
> +  }
> +
> +  public final static class NullableIntHolder implements ValueHolder {
> +    public static final MajorType TYPE = Types.optional(MinorType.INT);
> +    public int value;
> +    public int isSet;
> +  }
> +
> +  public final static class LongHolder implements ValueHolder {
> +    public static final MajorType TYPE = Types.required(MinorType.BIGINT);
> +    public long value;
> +  }
> +
> +  public final static class NullableLongHolder implements ValueHolder {
> +    public static final MajorType TYPE = Types.optional(MinorType.BIGINT);
> +    public long value;
> +    public int isSet;
> +  }
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
> new file mode 100644
> index 0000000..554c7d3
> --- /dev/null
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
> @@ -0,0 +1,12 @@
> +package org.apache.drill.exec.expr.holders;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +public class VarBinaryHolder {
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(VarBinaryHolder.class);
> +
> +  public ByteBuf buffer;
> +  public int start;
> +  public int length;
> +
> +}
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> index 33707a0..1dd7de8 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> @@ -17,7 +17,15 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.ops;
>
> +import java.io.IOException;
> +
>  import org.apache.drill.common.expression.LogicalExpression;
> +import org.apache.drill.exec.compile.ClassTransformer;
> +import org.apache.drill.exec.compile.QueryClassLoader;
> +import org.apache.drill.exec.compile.TemplateClassDefinition;
> +import org.apache.drill.exec.exception.ClassTransformationException;
> +import org.apache.drill.exec.expr.CodeGenerator;
> +import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
>  import org.apache.drill.exec.memory.BufferAllocator;
>  import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
>  import
> org.apache.drill.exec.physical.impl.FilteringRecordBatchTransformer;
> @@ -54,8 +62,13 @@ public class FragmentContext {
>    private final IncomingBuffers buffers;
>    private volatile Throwable failureCause;
>    private volatile boolean failed = false;
> +  private final FunctionImplementationRegistry funcRegistry;
> +  private final QueryClassLoader loader;
> +  private final ClassTransformer transformer;
>
> -  public FragmentContext(DrillbitContext dbContext, FragmentHandle
> handle, UserClientConnection connection, IncomingBuffers buffers) {
> +  public FragmentContext(DrillbitContext dbContext, FragmentHandle
> handle, UserClientConnection connection, IncomingBuffers buffers,
> FunctionImplementationRegistry funcRegistry) {
> +    this.loader = new QueryClassLoader(true);
> +    this.transformer = new ClassTransformer();
>      this.fragmentTime =
> dbContext.getMetrics().timer(METRIC_TIMER_FRAGMENT_TIME);
>      this.batchesCompleted = new SingleThreadNestedCounter(dbContext,
> METRIC_BATCHES_COMPLETED);
>      this.recordsCompleted = new SingleThreadNestedCounter(dbContext,
> METRIC_RECORDS_COMPLETED);
> @@ -64,6 +77,7 @@ public class FragmentContext {
>      this.connection = connection;
>      this.handle = handle;
>      this.buffers = buffers;
> +    this.funcRegistry = funcRegistry;
>    }
>
>    public void fail(Throwable cause) {
> @@ -89,6 +103,10 @@ public class FragmentContext {
>      return context.getAllocator();
>    }
>
> +  public <T> T getImplementationClass(TemplateClassDefinition<T, Void>
> templateDefinition, CodeGenerator cg) throws ClassTransformationException,
> IOException{
> +    return transformer.getImplementationClass(this.loader,
> templateDefinition, cg.generate(), null);
> +  }
> +
>    public FilteringRecordBatchTransformer
> getFilteringExpression(LogicalExpression expr){
>      return null;
>    }
> @@ -119,5 +137,11 @@ public class FragmentContext {
>      return failed;
>    }
>
> +  public FunctionImplementationRegistry getFunctionRegistry(){
> +    return funcRegistry;
> +  }
>
> +  public QueryClassLoader getClassLoader(){
> +    return loader;
> +  }
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> index 98ef05f..dcaf823 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> @@ -1,28 +1,28 @@
>  package org.apache.drill.exec.opt;
>
> -import com.fasterxml.jackson.core.type.TypeReference;
> -import com.fasterxml.jackson.databind.ObjectMapper;
> -import org.apache.drill.common.JSONOptions;
> +import java.io.IOException;
> +import java.util.ArrayList;
> +import java.util.Collection;
> +import java.util.List;
> +
>  import org.apache.drill.common.PlanProperties;
>  import org.apache.drill.common.config.DrillConfig;
>  import org.apache.drill.common.logical.LogicalPlan;
> -import org.apache.drill.common.logical.data.*;
> +import org.apache.drill.common.logical.data.Project;
> +import org.apache.drill.common.logical.data.Scan;
> +import org.apache.drill.common.logical.data.SinkOperator;
> +import org.apache.drill.common.logical.data.Store;
>  import
> org.apache.drill.common.logical.data.visitors.AbstractLogicalVisitor;
> -import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
> +import org.apache.drill.common.types.TypeProtos.DataMode;
> +import org.apache.drill.common.types.TypeProtos.MinorType;
>  import org.apache.drill.exec.exception.OptimizerException;
>  import org.apache.drill.exec.ops.QueryContext;
>  import org.apache.drill.exec.physical.PhysicalPlan;
>  import org.apache.drill.exec.physical.base.PhysicalOperator;
>  import org.apache.drill.exec.physical.config.MockScanPOP;
> -import org.apache.drill.exec.physical.config.MockStorePOP;
>  import org.apache.drill.exec.physical.config.Screen;
> -import org.apache.drill.exec.proto.SchemaDefProtos;
> -import org.apache.drill.exec.server.DrillbitContext;
>
> -import java.io.IOException;
> -import java.util.ArrayList;
> -import java.util.Collection;
> -import java.util.List;
> +import com.fasterxml.jackson.core.type.TypeReference;
>
>  /**
>   * Created with IntelliJ IDEA.
> @@ -99,8 +99,8 @@ public class BasicOptimizer extends Optimizer{
>                  }
>                  else{
>                      myObjects = new ArrayList<>();
> -                    MockScanPOP.MockColumn[] cols = { new
> MockScanPOP.MockColumn("blah", SchemaDefProtos.MinorType.INT,
> SchemaDefProtos.DataMode.REQUIRED,4,4,4),
> -                            new MockScanPOP.MockColumn("blah_2",
> SchemaDefProtos.MinorType.INT, SchemaDefProtos.DataMode.REQUIRED,4,4,4)};
> +                    MockScanPOP.MockColumn[] cols = { new
> MockScanPOP.MockColumn("blah", MinorType.INT, DataMode.REQUIRED,4,4,4),
> +                            new MockScanPOP.MockColumn("blah_2",
> MinorType.INT, DataMode.REQUIRED,4,4,4)};
>                      myObjects.add(new MockScanPOP.MockScanEntry(50,
> cols));
>                  }
>              } catch (IOException e) {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ce0da88d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
> index db3390a..4e9eb4c 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
> @@ -17,7 +17,7 @@
>
> ******************************************************************************/
>  package org.apache.drill.exec.physical;
>
> -import org.apache.drill.common.expression.types.DataType;
> +import org.apache.drill.common.types.TypeProtos.MajorType;
>
>  import com.fasterxml.jackson.annotation.JsonCreator;
>  import com.fasterxml.jackson.annotation.JsonProperty;
> @@ -26,17 +26,17 @@ public class RecordField {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(RecordField.class);
>
>
> -  private DataType type;
> +  private MajorType type;
>    private ValueMode mode;
>
>    @JsonCreator
> -  public RecordField(@JsonProperty("type") DataType type,
> @JsonProperty("mode") ValueMode mode) {
> +  public RecordField(@JsonProperty("type") MajorType type,
> @JsonProperty("mode") ValueMode mode) {
>      super();
>      this.type = type;
>      this.mode = mode;
>    }
>
> -  public DataType getType() {
> +  public MajorType getType() {
>      return type;
>    }
>
>
>