You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by jn...@apache.org on 2016/12/20 04:27:37 UTC

[8/8] drill git commit: DRILL-5052: Option to debug generated Java code using an IDE

DRILL-5052: Option to debug generated Java code using an IDE

Provides a second compilation path for generated code: \u201cplan old Java\u201d
in which generated code inherit from their templates. Such code can be
compiled directly, allowing easy debugging of generated code.

Also show to generate two classes in the External Sort Batch as \u201cplain
old Java\u201d to enable IDE debugging of that generated code. Required
minor clean-up of the templates.

Fixes some broken toString( ) methods in code generation classes
Fixes a variety of small compilation warnings
Adds Java doc to a few classes

Includes clean-up from code review comments.

close apache/drill#660


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

Branch: refs/heads/master
Commit: bbcf4b765e6946a8b6c7110372c4e1cadbfefa44
Parents: 03928af
Author: Paul Rogers <pr...@maprtech.com>
Authored: Sat Nov 19 18:29:24 2016 -0800
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Mon Dec 19 15:57:36 2016 -0800

----------------------------------------------------------------------
 .../org/apache/drill/exec/ExecConstants.java    |   1 -
 .../exec/compile/AbstractClassCompiler.java     |   7 +-
 .../drill/exec/compile/CachedClassLoader.java   |  69 +++++++
 .../apache/drill/exec/compile/ClassBuilder.java | 189 +++++++++++++++++++
 .../exec/compile/ClassCompilerSelector.java     | 146 ++++++++++++++
 .../drill/exec/compile/ClassTransformer.java    |  27 ++-
 .../apache/drill/exec/compile/CodeCompiler.java |  80 ++++++--
 .../drill/exec/compile/DrillJavaFileObject.java |  21 +++
 .../drill/exec/compile/JDKClassCompiler.java    |  18 +-
 .../drill/exec/compile/JaninoClassCompiler.java |  34 +++-
 .../apache/drill/exec/compile/MergeAdapter.java |  12 +-
 .../drill/exec/compile/QueryClassLoader.java    |  91 +--------
 .../exec/compile/TemplateClassDefinition.java   |  29 ++-
 .../exec/compile/sig/CodeGeneratorMethod.java   |   2 +-
 .../exec/compile/sig/GeneratorMapping.java      |  19 ++
 .../drill/exec/compile/sig/SignatureHolder.java |  27 ++-
 .../apache/drill/exec/expr/ClassGenerator.java  |  68 +++++--
 .../apache/drill/exec/expr/CodeGenerator.java   | 104 ++++++++--
 .../drill/exec/expr/DebugStringBuilder.java     |  62 ++++++
 .../drill/exec/expr/DirectExpression.java       |  14 +-
 .../apache/drill/exec/ops/FragmentContext.java  |   4 +-
 .../apache/drill/exec/ops/OperatorContext.java  |  10 +-
 .../drill/exec/ops/OperatorContextImpl.java     |  10 +
 .../physical/impl/xsort/ExternalSortBatch.java  |  12 +-
 .../physical/impl/xsort/SingleBatchSorter.java  |   2 +-
 .../impl/xsort/SingleBatchSorterTemplate.java   |  11 +-
 .../server/options/SystemOptionManager.java     |   8 +-
 .../src/main/resources/drill-module.conf        |   6 +-
 .../exec/compile/TestClassTransformation.java   |  16 +-
 .../exec/compile/TestLargeFileCompilation.java  |  14 +-
 .../physical/unit/PhysicalOpUnitTestBase.java   |   4 +-
 .../drill/exec/vector/BaseValueVector.java      |   3 +-
 .../vector/complex/AbstractContainerVector.java |   2 +
 .../exec/vector/complex/RepeatedListVector.java |   1 +
 34 files changed, 927 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 0eb9c58..740eb4b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -310,7 +310,6 @@ public interface ExecConstants {
   OptionValidator ENABLE_NEW_TEXT_READER = new BooleanValidator(ENABLE_NEW_TEXT_READER_KEY, true);
 
   String BOOTSTRAP_STORAGE_PLUGINS_FILE = "bootstrap-storage-plugins.json";
-  String MAX_LOADING_CACHE_SIZE_CONFIG = "drill.exec.compile.cache_max_size";
 
   String DRILL_SYS_FILE_SUFFIX = ".sys.drill";
 

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AbstractClassCompiler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AbstractClassCompiler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AbstractClassCompiler.java
index a5c96e3..4caf8e1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AbstractClassCompiler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AbstractClassCompiler.java
@@ -20,12 +20,14 @@ package org.apache.drill.exec.compile;
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.util.Map;
 
 import org.apache.drill.common.util.DrillStringUtils;
 import org.apache.drill.exec.compile.ClassTransformer.ClassNames;
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.codehaus.commons.compiler.CompileException;
 
+@SuppressWarnings("unused")
 public abstract class AbstractClassCompiler {
   protected boolean debug = false;
 
@@ -74,9 +76,10 @@ public abstract class AbstractClassCompiler {
     return out.toString();
   }
 
-  protected abstract byte[][] getByteCode(ClassNames className, String sourcecode)
+  protected abstract byte[][] getByteCode(final ClassNames className, final String sourcecode)
       throws CompileException, IOException, ClassNotFoundException, ClassTransformationException;
-
+  public abstract Map<String,byte[]> compile(final ClassNames className, final String sourceCode)
+      throws CompileException, IOException, ClassNotFoundException;
   protected abstract org.slf4j.Logger getLogger();
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CachedClassLoader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CachedClassLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CachedClassLoader.java
new file mode 100644
index 0000000..5270aa8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CachedClassLoader.java
@@ -0,0 +1,69 @@
+/**
+ * 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.compile;
+
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Class loader for "plain-old Java" generated classes.
+ * Very basic implementation: allows defining a class from
+ * byte codes and finding the loaded classes. Delegates
+ * all other class requests to the thread context class
+ * loader. This structure ensures that a generated class can
+ * find both its own inner classes as well as all the standard
+ * Drill implementation classes.
+ */
+
+public class CachedClassLoader extends URLClassLoader {
+
+  /**
+   * Cache of generated classes. Semantics: a single thread defines
+   * the classes, many threads may access the classes.
+   */
+
+  private ConcurrentMap<String, Class<?>> cache = Maps.newConcurrentMap();
+
+  public CachedClassLoader() {
+    super(new URL[0], Thread.currentThread().getContextClassLoader());
+  }
+
+  public void addClass(String fqcn, byte[] byteCodes) {
+    Class<?> newClass = defineClass(fqcn, byteCodes, 0, byteCodes.length);
+    cache.put(fqcn, newClass);
+  }
+
+  @Override
+  public Class<?> findClass(String className) throws ClassNotFoundException {
+    Class<?> theClass = cache.get(className);
+    if (theClass != null) {
+      return theClass;
+    }
+    return super.findClass(className);
+  }
+
+  public void addClasses(Map<String, byte[]> results) {
+    for (Map.Entry<String, byte[]> result : results.entrySet()) {
+      addClass(result.getKey(), result.getValue());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBuilder.java
new file mode 100644
index 0000000..f5024fe
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassBuilder.java
@@ -0,0 +1,189 @@
+/**
+ * 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.compile;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.compile.ClassTransformer.ClassNames;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.codehaus.commons.compiler.CompileException;
+
+/**
+ * Implements the "plain-old Java" method of code generation and
+ * compilation. Given a {@link CodeGenerator}, obtains the generated
+ * source code, compiles it with the selected compiler, loads the
+ * byte-codes into a class loader and provides the resulting
+ * class. Compared with the {@link ClassTransformer} mechanism,
+ * this one requires the code generator to have generated a complete
+ * Java class that is capable of direct compilation and loading.
+ * This means the generated class must be a subclass of the template
+ * so that the JVM can use normal Java inheritance to associate the
+ * template and generated methods.
+ * <p>
+ * Here is how to use the plain-old Java technique to debug
+ * generated code:
+ * <ul>
+ * <li>Set the config option <var>drill.exec.compile.save_source</var>
+ * to <var>true</var>.</li>
+ * <li>Set the config option <var>drill.exec.compile.code_dir</var>
+ * to the location where you want to save the generated source
+ * code.</li>
+ * <li>Where you generate code (using a {@link CodeGenerator}),
+ * set the "plain-old Java" options:<pre>
+ * CodeGenerator&lt;Foo> cg = ...
+ * cg.plainOldJavaCapable(true); // Class supports plain-old Java
+ * cg.preferPlainOldJava(true); // Actually generate plain-old Java
+ * ...</pre></li>
+ * <li>In your favorite IDE, add to the code lookup path the
+ * code directory saved earlier. In Eclipse, for example, you do
+ * this in the debug configuration you will use to debug Drill.</li>
+ * <li>Set a breakpoint in template used for the generated code.</li>
+ * <li>Run Drill. The IDE will stop at your breakpoint.</li>
+ * <li>Step into the generated code. Examine class field and
+ * local variables. Have fun!</li>
+ * </ul>
+ * <p>
+ * Note: not all generated code is ready to be compiled as plain-old
+ * Java. Some classes omit from the template the proper <code>throws</code>
+ * declarations. Other minor problems may also crop up. All are easy
+ * to fix. Once you've done so, add the following to mark that you've
+ * done the clean-up:<pre>
+ * cg.plainOldJavaCapable(true); // Class supports plain-old Java</pre>
+ * <p>
+ * The setting to prefer plain-old Java is ignored for generated
+ * classes not marked as plain-old Java capable.
+ */
+
+public class ClassBuilder {
+
+  public static final String SAVE_CODE_OPTION = CodeCompiler.COMPILE_BASE + ".save_source";
+  public static final String CODE_DIR_OPTION = CodeCompiler.COMPILE_BASE + ".code_dir";
+
+  private final DrillConfig config;
+  private final OptionManager options;
+  private final boolean saveCode;
+  private final File codeDir;
+
+  public ClassBuilder(DrillConfig config, OptionManager optionManager) {
+    this.config = config;
+    options = optionManager;
+
+    // The option to save code is a boot-time option because
+    // it is used selectively during debugging, but can cause
+    // excessive I/O in a running server if used to save all code.
+
+    saveCode = config.getBoolean(SAVE_CODE_OPTION);
+    codeDir = new File(config.getString(CODE_DIR_OPTION));
+  }
+
+  /**
+   * Given a code generator which has already generated plain-old Java
+   * code, compile the code, create a class loader, and return the
+   * resulting Java class.
+   *
+   * @param cg a plain-old Java capable code generator that has generated
+   * plain-old Java code
+   * @return the class that the code generator defines
+   * @throws ClassTransformationException
+   */
+
+  public Class<?> getImplementationClass(CodeGenerator<?> cg) throws ClassTransformationException {
+    try {
+      return compileClass(cg);
+    } catch (CompileException | ClassNotFoundException|IOException e) {
+      throw new ClassTransformationException(e);
+    }
+  }
+
+  /**
+   * Performs the actual work of compiling the code and loading the class.
+   *
+   * @param cg the code generator that has built the class(es) to be generated.
+   * @return the class, after code generation and (if needed) compilation.
+   * @throws IOException if an error occurs when optionally writing code to disk.
+   * @throws CompileException if the generated code has compile issues.
+   * @throws ClassNotFoundException if the generated code references unknown classes.
+   * @throws ClassTransformationException generic "something is wrong" error from
+   * Drill class compilation code.
+   */
+  private Class<?> compileClass(CodeGenerator<?> cg) throws IOException, CompileException, ClassNotFoundException, ClassTransformationException {
+
+    // Get the plain-old Java code.
+
+    String code = cg.getGeneratedCode();
+
+    // Get the class names (dotted, file path, etc.)
+
+    String className = cg.getMaterializedClassName();
+    ClassTransformer.ClassNames name = new ClassTransformer.ClassNames(className);
+
+    // A key advantage of this method is that the code can be
+    // saved and debugged, if needed.
+
+    saveCode(code, name);
+
+    // Compile the code and load it into a class loader.
+
+    CachedClassLoader classLoader = new CachedClassLoader();
+    ClassCompilerSelector compilerSelector = new ClassCompilerSelector(classLoader, config, options);
+    Map<String,byte[]> results = compilerSelector.compile(name, code);
+    classLoader.addClasses(results);
+
+    // Get the class from the class loader.
+
+    try {
+      return classLoader.findClass(className);
+    } catch (ClassNotFoundException e) {
+      // This should never occur.
+      throw new IllegalStateException("Code load failed", e);
+    }
+  }
+
+  /**
+   * Save code to a predefined location for debugging. To use the code
+   * for debugging, make sure the save location is on your IDE's source
+   * code search path. Code is saved in usual Java format with each
+   * package as a directory. The provided code directory becomes a
+   * source directory, as in Maven's "src/main/java".
+   *
+   * @param code the source code
+   * @param name the class name
+   */
+
+  private void saveCode(String code, ClassNames name) {
+
+    // Skip if we don't want to save the code.
+
+    if (! saveCode) { return; }
+
+    String pathName = name.slash + ".java";
+    File codeFile = new File(codeDir, pathName);
+    codeFile.getParentFile().mkdirs();
+    try (final FileWriter writer = new FileWriter(codeFile)) {
+      writer.write(code);
+    } catch (IOException e) {
+      System.err.println("Could not save: " + codeFile.getAbsolutePath());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompilerSelector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompilerSelector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompilerSelector.java
new file mode 100644
index 0000000..c8afbc6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassCompilerSelector.java
@@ -0,0 +1,146 @@
+/**
+ * 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.compile;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.compile.ClassTransformer.ClassNames;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.server.options.OptionValidator;
+import org.apache.drill.exec.server.options.OptionValue;
+import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
+import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
+import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
+import org.codehaus.commons.compiler.CompileException;
+
+/**
+ * Selects between the two supported Java compilers: Janino and
+ * the build-in Java compiler.
+ *
+ * <h4>Session Options</h4>
+ * <dl>
+ * <dt>exec.java_compiler</dt>
+ * <dd>The compiler to use. Valid options are defined in the
+ * {@link ClassCompilerSelector.CompilerPolicy} enum.</dd>
+ * <dt>exec.java_compiler_debug</dt>
+ * <dd>If debug logging is enabled, then {@link AbstractClassCompiler} writes the
+ * generated Java code to the log file prior to compilation. This option
+ * adds line numbers to the logged code.</dd>
+ * <dt>exec.java_compiler_janino_maxsize</dt>
+ * <dd>The maximum size of code that the Janio compiler can handle. Larger code is
+ * handled by the JDK compiler. Defaults to 256K.</dd>
+ * </dl>
+ * <h4>Configuration Options</h4>
+ * Configuration options are used when the above session options are unset.
+ * <dl>
+ * <dt>drill.exec.compile.compiler</dt>
+ * <dd>Default for <var>exec.java_compiler</var></dd>
+ * <dt>drill.exec.compile.debug</dt>
+ * <dd>Default for <var>exec.java_compiler_debug</var></dd>
+ * <dt>drill.exec.compile.janino_maxsize</dt>
+ * <dd>Default for <var>exec.java_compiler_janino_maxsize</var></dd>
+ * </dl>
+ */
+
+public class ClassCompilerSelector {
+  public enum CompilerPolicy {
+    DEFAULT, JDK, JANINO;
+  }
+
+  public static final String JAVA_COMPILER_JANINO_MAXSIZE_CONFIG = CodeCompiler.COMPILE_BASE + ".janino_maxsize";
+  public static final String JAVA_COMPILER_DEBUG_CONFIG = CodeCompiler.COMPILE_BASE + ".debug";
+  public static final String JAVA_COMPILER_CONFIG = CodeCompiler.COMPILE_BASE + ".compiler";
+
+  public static final String JAVA_COMPILER_OPTION = "exec.java_compiler";
+  public static final String JAVA_COMPILER_JANINO_MAXSIZE_OPTION = "exec.java_compiler_janino_maxsize";
+  public static final OptionValidator JAVA_COMPILER_JANINO_MAXSIZE = new LongValidator(JAVA_COMPILER_JANINO_MAXSIZE_OPTION, 256*1024);
+
+  public static final String JAVA_COMPILER_DEBUG_OPTION = "exec.java_compiler_debug";
+  public static final OptionValidator JAVA_COMPILER_DEBUG = new BooleanValidator(JAVA_COMPILER_DEBUG_OPTION, true);
+
+  public static final StringValidator JAVA_COMPILER_VALIDATOR = new StringValidator(JAVA_COMPILER_OPTION, CompilerPolicy.DEFAULT.toString()) {
+    @Override
+    public void validate(final OptionValue v, final OptionManager manager) {
+      super.validate(v, manager);
+      try {
+        CompilerPolicy.valueOf(v.string_val.toUpperCase());
+      } catch (IllegalArgumentException e) {
+        throw UserException.validationError()
+            .message("Invalid value '%s' specified for option '%s'. Valid values are %s.",
+              v.string_val, getOptionName(), Arrays.toString(CompilerPolicy.values()))
+            .build(QueryClassLoader.logger);
+      }
+    }
+  };
+
+  private final CompilerPolicy policy;
+  private final long janinoThreshold;
+
+  private final AbstractClassCompiler jdkClassCompiler;
+  private final AbstractClassCompiler janinoClassCompiler;
+
+  public ClassCompilerSelector(ClassLoader classLoader, DrillConfig config, OptionManager sessionOptions) {
+    OptionValue value = sessionOptions.getOption(JAVA_COMPILER_OPTION);
+    this.policy = CompilerPolicy.valueOf((value != null) ? value.string_val.toUpperCase() : config.getString(JAVA_COMPILER_CONFIG).toUpperCase());
+
+    value = sessionOptions.getOption(JAVA_COMPILER_JANINO_MAXSIZE_OPTION);
+    this.janinoThreshold = (value != null) ? value.num_val : config.getLong(JAVA_COMPILER_JANINO_MAXSIZE_CONFIG);
+
+    value = sessionOptions.getOption(JAVA_COMPILER_DEBUG_OPTION);
+    boolean debug = (value != null) ? value.bool_val : config.getBoolean(JAVA_COMPILER_DEBUG_CONFIG);
+
+    this.janinoClassCompiler = (policy == CompilerPolicy.JANINO || policy == CompilerPolicy.DEFAULT) ? new JaninoClassCompiler(classLoader, debug) : null;
+    this.jdkClassCompiler = (policy == CompilerPolicy.JDK || policy == CompilerPolicy.DEFAULT) ? JDKClassCompiler.newInstance(classLoader, debug) : null;
+  }
+
+  byte[][] getClassByteCode(ClassNames className, String sourceCode)
+      throws CompileException, ClassNotFoundException, ClassTransformationException, IOException {
+
+    byte[][] bc = getCompiler(sourceCode).getClassByteCode(className, sourceCode);
+
+    // Uncomment the following to save the generated byte codes.
+
+//    final String baseDir = System.getProperty("java.io.tmpdir") + File.separator + className;
+//    File classFile = new File(baseDir + className.clazz);
+//    classFile.getParentFile().mkdirs();
+//    try (BufferedOutputStream out = new BufferedOutputStream(new FileOutputStream(classFile))) {
+//      out.write(bc[0]);
+//    }
+
+    return bc;
+  }
+
+  public Map<String,byte[]> compile(ClassNames className, String sourceCode)
+      throws CompileException, ClassNotFoundException, ClassTransformationException, IOException {
+    return getCompiler(sourceCode).compile(className, sourceCode);
+  }
+
+  private AbstractClassCompiler getCompiler(String sourceCode) {
+    if (jdkClassCompiler != null &&
+        (policy == CompilerPolicy.JDK || (policy == CompilerPolicy.DEFAULT && sourceCode.length() > janinoThreshold))) {
+      return jdkClassCompiler;
+    } else {
+      return janinoClassCompiler;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
index 02323a9..3c3c30e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
@@ -22,12 +22,13 @@ import java.util.LinkedList;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.DrillStringUtils;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.compile.MergeAdapter.MergedClassResult;
 import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.server.options.TypeValidators.EnumeratedStringValidator;
 import org.codehaus.commons.compiler.CompileException;
 import org.objectweb.asm.ClassReader;
@@ -39,12 +40,21 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
+/**
+ * Compiles generated code, merges the resulting class with the
+ * template class, and performs byte-code cleanup on the resulting
+ * byte codes. The most important transform is scalar replacement
+ * which replaces occurences of non-escaping objects with a
+ * collection of member variables.
+ */
+
 public class ClassTransformer {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClassTransformer.class);
 
   private static final int MAX_SCALAR_REPLACE_CODE_SIZE = 2*1024*1024; // 2meg
 
   private final ByteCodeLoader byteCodeLoader = new ByteCodeLoader();
+  private final DrillConfig config;
   private final OptionManager optionManager;
 
   public final static String SCALAR_REPLACEMENT_OPTION =
@@ -73,13 +83,14 @@ public class ClassTransformer {
         return TRY;
       case "on":
         return ON;
+      default:
+        throw new IllegalArgumentException("Invalid ScalarReplacementOption \"" + s + "\"");
       }
-
-      throw new IllegalArgumentException("Invalid ScalarReplacementOption \"" + s + "\"");
     }
   }
 
-  public ClassTransformer(final OptionManager optionManager) {
+  public ClassTransformer(final DrillConfig config, final OptionManager optionManager) {
+    this.config = config;
     this.optionManager = optionManager;
   }
 
@@ -210,6 +221,12 @@ public class ClassTransformer {
     }
   }
 
+  public Class<?> getImplementationClass(CodeGenerator<?> cg) throws ClassTransformationException {
+    final QueryClassLoader loader = new QueryClassLoader(config, optionManager);
+    return getImplementationClass(loader, cg.getDefinition(),
+        cg.getGeneratedCode(), cg.getMaterializedClassName());
+  }
+
   public Class<?> getImplementationClass(
       final QueryClassLoader classLoader,
       final TemplateClassDefinition<?> templateDefinition,
@@ -248,7 +265,7 @@ public class ClassTransformer {
         final ClassNames nextGenerated = nextSet.generated;
         final ClassNode generatedNode = classesToMerge.get(nextGenerated.slash);
 
-        /**
+        /*
          * TODO
          * We're having a problem with some cases of scalar replacement, but we want to get
          * the code in so it doesn't rot anymore.

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeCompiler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeCompiler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeCompiler.java
index af328b1..fb59a4c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeCompiler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeCompiler.java
@@ -22,7 +22,6 @@ import java.util.List;
 import java.util.concurrent.ExecutionException;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.server.options.OptionManager;
@@ -33,36 +32,72 @@ import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.collect.Lists;
 
+/**
+ * Global code compiler mechanism shared by all threads and operators.
+ * Holds a single cache of generated code (keyed by code source) to
+ * prevent compiling identical code multiple times. Supports both
+ * the byte-code merging and plain-old Java methods of code
+ * generation and compilation.
+ */
+
 public class CodeCompiler {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CodeCompiler.class);
+
+  public static final String COMPILE_BASE = "drill.exec.compile";
+  public static final String MAX_LOADING_CACHE_SIZE_CONFIG = COMPILE_BASE + ".cache_max_size";
 
   private final ClassTransformer transformer;
+  private final ClassBuilder classBuilder;
+
+  /**
+   * Google Guava loading cache that defers creating a cache
+   * entry until first needed. Creation is done in a thread-safe
+   * way: if two threads try to create the same class at the same
+   * time, the first does the work, the second waits for the first
+   * to complete, then grabs the new entry.
+   */
+
   private final LoadingCache<CodeGenerator<?>, GeneratedClassEntry> cache;
-  private final DrillConfig config;
-  private final OptionManager optionManager;
 
   public CodeCompiler(final DrillConfig config, final OptionManager optionManager) {
-    transformer = new ClassTransformer(optionManager);
-    final int cacheMaxSize = config.getInt(ExecConstants.MAX_LOADING_CACHE_SIZE_CONFIG);
+    transformer = new ClassTransformer(config, optionManager);
+    classBuilder = new ClassBuilder(config, optionManager);
+    final int cacheMaxSize = config.getInt(MAX_LOADING_CACHE_SIZE_CONFIG);
     cache = CacheBuilder.newBuilder()
         .maximumSize(cacheMaxSize)
         .build(new Loader());
-    this.optionManager = optionManager;
-    this.config = config;
   }
 
+  /**
+   * Create a single instance of the generated class.
+   *
+   * @param cg code generator for the class to be instantiated.
+   * @return an instance of the generated class
+   * @throws ClassTransformationException general "something is wrong" exception
+   * for the Drill compilation chain.
+   */
+
   @SuppressWarnings("unchecked")
-  public <T> T getImplementationClass(final CodeGenerator<?> cg) throws ClassTransformationException, IOException {
-    return (T) getImplementationClass(cg, 1).get(0);
+  public <T> T createInstance(final CodeGenerator<?> cg) throws ClassTransformationException {
+    return (T) createInstances(cg, 1).get(0);
   }
 
+  /**
+   * Create multiple instances of the generated class.
+   *
+   * @param cg code generator for the class to be instantiated.
+   * @param count the number of instances desired.
+   * @return a list of instances of the generated class.
+   * @throws ClassTransformationException general "something is wrong" exception
+   * for the Drill compilation chain.
+   */
+
   @SuppressWarnings("unchecked")
-  public <T> List<T> getImplementationClass(final CodeGenerator<?> cg, int instanceNumber) throws ClassTransformationException, IOException {
+  public <T> List<T> createInstances(final CodeGenerator<?> cg, int count) throws ClassTransformationException {
     cg.generate();
     try {
       final GeneratedClassEntry ce = cache.get(cg);
       List<T> tList = Lists.newArrayList();
-      for ( int i = 0; i < instanceNumber; i++) {
+      for ( int i = 0; i < count; i++) {
         tList.add((T) ce.clazz.newInstance());
       }
       return tList;
@@ -71,12 +106,27 @@ public class CodeCompiler {
     }
   }
 
+  /**
+   * Loader used to create an entry in the class cache when the entry
+   * does not yet exist. Here, we generate the code, compile it,
+   * and place the resulting class into the cache. The class has an
+   * associated class loader which "dangles" from the class itself;
+   * we don't keep track of the class loader itself.
+   */
+
   private class Loader extends CacheLoader<CodeGenerator<?>, GeneratedClassEntry> {
     @Override
     public GeneratedClassEntry load(final CodeGenerator<?> cg) throws Exception {
-      final QueryClassLoader loader = new QueryClassLoader(config, optionManager);
-      final Class<?> c = transformer.getImplementationClass(loader, cg.getDefinition(),
-          cg.getGeneratedCode(), cg.getMaterializedClassName());
+      final Class<?> c;
+      if ( cg.isPlainOldJava( ) ) {
+        // Generate class as plain old Java
+
+        c = classBuilder.getImplementationClass(cg);
+      } else {
+        // Generate class parts and assemble byte-codes.
+
+        c = transformer.getImplementationClass(cg);
+      }
       return new GeneratedClassEntry(c);
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillJavaFileObject.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillJavaFileObject.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillJavaFileObject.java
index acc32b5..7b95374 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillJavaFileObject.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillJavaFileObject.java
@@ -24,6 +24,7 @@ import java.io.Reader;
 import java.io.StringReader;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.HashMap;
 import java.util.Map;
 
 import javax.tools.SimpleJavaFileObject;
@@ -38,14 +39,18 @@ final class DrillJavaFileObject extends SimpleJavaFileObject {
 
   private Map<String, DrillJavaFileObject> outputFiles;
 
+  private final String className;
+
   public DrillJavaFileObject(final String className, final String sourceCode) {
     super(makeURI(className), Kind.SOURCE);
+    this.className = className;
     this.sourceCode = sourceCode;
     this.outputStream = null;
   }
 
   private DrillJavaFileObject(final String name, final Kind kind) {
     super(makeURI(name), kind);
+    this.className = name;
     this.outputStream = new ByteArrayOutputStream();
     this.sourceCode = null;
   }
@@ -67,6 +72,22 @@ final class DrillJavaFileObject extends SimpleJavaFileObject {
     }
   }
 
+  /**
+   * Return the byte codes for the main class and any nested
+   * classes.
+   *
+   * @return map of fully-qualified class names to byte codes
+   * for the class
+   */
+
+  public Map<String,byte[]> getClassByteCodes() {
+    Map<String,byte[]> results = new HashMap<>();
+    for(DrillJavaFileObject outputFile : outputFiles.values()) {
+      results.put(outputFile.className, outputFile.outputStream.toByteArray());
+    }
+    return results;
+  }
+
   public DrillJavaFileObject addOutputJavaFile(String className) {
     if (outputFiles == null) {
       outputFiles = Maps.newLinkedHashMap();

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
index ecd222d..6007078 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.compile;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Map;
 
 import javax.tools.DiagnosticListener;
 import javax.tools.JavaCompiler;
@@ -44,8 +45,7 @@ class JDKClassCompiler extends AbstractClassCompiler {
   public static JDKClassCompiler newInstance(ClassLoader classLoader, boolean debug) {
     JavaCompiler compiler = ToolProvider.getSystemJavaCompiler();
     if (compiler == null) {
-      logger.warn("JDK Java compiler not available - probably you're running Drill with a JRE and not a JDK");
-      return null;
+      throw new RuntimeException("JDK Java compiler not available - probably you're running Drill with a JRE and not a JDK");
     }
     return new JDKClassCompiler(compiler, classLoader, debug);
   }
@@ -61,6 +61,17 @@ class JDKClassCompiler extends AbstractClassCompiler {
   @Override
   protected byte[][] getByteCode(final ClassNames className, final String sourceCode)
       throws CompileException, IOException, ClassNotFoundException {
+    return doCompile(className, sourceCode).getByteCode();
+   }
+
+  @Override
+  public Map<String,byte[]> compile(final ClassNames className, final String sourceCode)
+      throws CompileException, IOException, ClassNotFoundException {
+    return doCompile(className, sourceCode).getClassByteCodes();
+ }
+
+  private DrillJavaFileObject doCompile(final ClassNames className, final String sourceCode)
+        throws CompileException, IOException, ClassNotFoundException {
     try {
       // Create one Java source file in memory, which will be compiled later.
       DrillJavaFileObject compilationUnit = new DrillJavaFileObject(className.dot, sourceCode);
@@ -74,7 +85,7 @@ class JDKClassCompiler extends AbstractClassCompiler {
         throw new ClassNotFoundException(className + ": Class file not created by compilation.");
       }
       // all good
-      return compilationUnit.getByteCode();
+      return compilationUnit;
     } catch (RuntimeException rte) {
       // Unwrap the compilation exception and throw it.
       Throwable cause = rte.getCause();
@@ -93,5 +104,4 @@ class JDKClassCompiler extends AbstractClassCompiler {
 
   @Override
   protected org.slf4j.Logger getLogger() { return logger; }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
index 1bb4465..cab8e22 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JaninoClassCompiler.java
@@ -19,6 +19,8 @@ package org.apache.drill.exec.compile;
 
 import java.io.IOException;
 import java.io.StringReader;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.drill.exec.compile.ClassTransformer.ClassNames;
 import org.apache.drill.exec.exception.ClassTransformationException;
@@ -31,7 +33,7 @@ import org.codehaus.janino.Scanner;
 import org.codehaus.janino.UnitCompiler;
 import org.codehaus.janino.util.ClassFile;
 
-public class JaninoClassCompiler extends AbstractClassCompiler {
+class JaninoClassCompiler extends AbstractClassCompiler {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JaninoClassCompiler.class);
 
   private IClassLoader compilationClassLoader;
@@ -42,13 +44,9 @@ public class JaninoClassCompiler extends AbstractClassCompiler {
   }
 
   @Override
-  protected byte[][] getByteCode(final ClassNames className, final String sourcecode)
+  protected byte[][] getByteCode(final ClassNames className, final String sourceCode)
       throws CompileException, IOException, ClassNotFoundException, ClassTransformationException {
-    StringReader reader = new StringReader(sourcecode);
-    Scanner scanner = new Scanner((String) null, reader);
-    Java.CompilationUnit compilationUnit = new Parser(scanner).parseCompilationUnit();
-    ClassFile[] classFiles = new UnitCompiler(compilationUnit, compilationClassLoader)
-                                  .compileUnit(this.debug, this.debug, this.debug);
+    ClassFile[] classFiles = doCompile(sourceCode);
 
     byte[][] byteCodes = new byte[classFiles.length][];
     for(int i = 0; i < classFiles.length; i++){
@@ -58,5 +56,27 @@ public class JaninoClassCompiler extends AbstractClassCompiler {
   }
 
   @Override
+  public Map<String,byte[]> compile(final ClassNames className, final String sourceCode)
+      throws CompileException, IOException, ClassNotFoundException {
+
+    ClassFile[] classFiles = doCompile(sourceCode);
+    Map<String,byte[]> results = new HashMap<>();
+    for(int i = 0;  i < classFiles.length;  i++) {
+      ClassFile classFile = classFiles[i];
+      results.put(classFile.getThisClassName(), classFile.toByteArray());
+    }
+    return results;
+  }
+
+  private ClassFile[] doCompile(final String sourceCode)
+      throws CompileException, IOException, ClassNotFoundException {
+    StringReader reader = new StringReader(sourceCode);
+    Scanner scanner = new Scanner((String) null, reader);
+    Java.CompilationUnit compilationUnit = new Parser(scanner).parseCompilationUnit();
+    return new UnitCompiler(compilationUnit, compilationClassLoader)
+                                  .compileUnit(this.debug, this.debug, this.debug);
+  }
+
+  @Override
   protected org.slf4j.Logger getLogger() { return logger; }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java
index 82bd413..05e8ac1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.compile;
 
+import java.io.File;
+import java.io.IOException;
 import java.lang.reflect.Modifier;
 import java.util.Collection;
 import java.util.Iterator;
@@ -39,11 +41,13 @@ import org.objectweb.asm.tree.FieldNode;
 import org.objectweb.asm.tree.MethodNode;
 
 import com.google.common.collect.Sets;
+import com.google.common.io.Files;
 
 /**
  * Serves two purposes. Renames all inner classes references to the outer class to the new name. Also adds all the
  * methods and fields of the class to merge to the class that is being visited.
  */
+@SuppressWarnings("unused")
 class MergeAdapter extends ClassVisitor {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MergeAdapter.class);
   private final ClassNode classToMerge;
@@ -253,7 +257,13 @@ class MergeAdapter extends ClassVisitor {
       }
 
       // enable when you want all the generated merged class files to also be written to disk.
-//      Files.write(outputClass, new File(String.format("/src/scratch/drill-generated-classes/%s-output.class", set.generated.dot)));
+//      try {
+//        File destDir = new File( "/tmp/scratch/drill-generated-classes" );
+//        destDir.mkdirs();
+//        Files.write(outputClass, new File(destDir, String.format("%s-output.class", set.generated.dot)));
+//      } catch (IOException e) {
+//        // Ignore;
+//      }
 
       return new MergedClassResult(outputClass, re.getInnerClasses());
     } catch (Error | RuntimeException e) {

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
index da03802..31b464b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
@@ -20,52 +20,23 @@ package org.apache.drill.exec.compile;
 import java.io.IOException;
 import java.net.URL;
 import java.net.URLClassLoader;
-import java.util.Arrays;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.compile.ClassTransformer.ClassNames;
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.exec.server.options.OptionValidator;
-import org.apache.drill.exec.server.options.OptionValue;
-import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
-import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
-import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
 import org.codehaus.commons.compiler.CompileException;
 
 import com.google.common.collect.MapMaker;
 
-public class QueryClassLoader extends URLClassLoader {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryClassLoader.class);
-
-  public static final String JAVA_COMPILER_OPTION = "exec.java_compiler";
-  public static final StringValidator JAVA_COMPILER_VALIDATOR = new StringValidator(JAVA_COMPILER_OPTION, CompilerPolicy.DEFAULT.toString()) {
-    @Override
-    public void validate(final OptionValue v, final OptionManager manager) {
-      super.validate(v, manager);
-      try {
-        CompilerPolicy.valueOf(v.string_val.toUpperCase());
-      } catch (IllegalArgumentException e) {
-        throw UserException.validationError()
-            .message("Invalid value '%s' specified for option '%s'. Valid values are %s.",
-              v.string_val, getOptionName(), Arrays.toString(CompilerPolicy.values()))
-            .build(logger);
-      }
-    }
-  };
-
-  public static final String JAVA_COMPILER_DEBUG_OPTION = "exec.java_compiler_debug";
-  public static final OptionValidator JAVA_COMPILER_DEBUG = new BooleanValidator(JAVA_COMPILER_DEBUG_OPTION, true);
-
-  public static final String JAVA_COMPILER_JANINO_MAXSIZE_OPTION = "exec.java_compiler_janino_maxsize";
-  public static final OptionValidator JAVA_COMPILER_JANINO_MAXSIZE = new LongValidator(JAVA_COMPILER_JANINO_MAXSIZE_OPTION, 256*1024);
+/**
+ * Per-compilation unit class loader that holds both caching and compilation
+ * steps. */
 
-  public static final String JAVA_COMPILER_CONFIG = "drill.exec.compile.compiler";
-  public static final String JAVA_COMPILER_DEBUG_CONFIG = "drill.exec.compile.debug";
-  public static final String JAVA_COMPILER_JANINO_MAXSIZE_CONFIG = "drill.exec.compile.janino_maxsize";
+public class QueryClassLoader extends URLClassLoader {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryClassLoader.class);
 
   private ClassCompilerSelector compilerSelector;
 
@@ -75,7 +46,7 @@ public class QueryClassLoader extends URLClassLoader {
 
   public QueryClassLoader(DrillConfig config, OptionManager sessionOptions) {
     super(new URL[0], Thread.currentThread().getContextClassLoader());
-    compilerSelector = new ClassCompilerSelector(config, sessionOptions);
+    compilerSelector = new ClassCompilerSelector(this, config, sessionOptions);
   }
 
   public long getNextClassIndex() {
@@ -104,54 +75,4 @@ public class QueryClassLoader extends URLClassLoader {
     return compilerSelector.getClassByteCode(className, sourceCode);
   }
 
-  public enum CompilerPolicy {
-    DEFAULT, JDK, JANINO;
-  }
-
-  private class ClassCompilerSelector {
-    private final CompilerPolicy policy;
-    private final long janinoThreshold;
-
-    private final AbstractClassCompiler jdkClassCompiler;
-    private final AbstractClassCompiler janinoClassCompiler;
-
-
-    ClassCompilerSelector(DrillConfig config, OptionManager sessionOptions) {
-      OptionValue value = sessionOptions.getOption(JAVA_COMPILER_OPTION);
-      this.policy = CompilerPolicy.valueOf((value != null) ? value.string_val.toUpperCase() : config.getString(JAVA_COMPILER_CONFIG).toUpperCase());
-
-      value = sessionOptions.getOption(JAVA_COMPILER_JANINO_MAXSIZE_OPTION);
-      this.janinoThreshold = (value != null) ? value.num_val : config.getLong(JAVA_COMPILER_JANINO_MAXSIZE_CONFIG);
-
-      value = sessionOptions.getOption(JAVA_COMPILER_DEBUG_OPTION);
-      boolean debug = (value != null) ? value.bool_val : config.getBoolean(JAVA_COMPILER_DEBUG_CONFIG);
-
-      this.janinoClassCompiler = (policy == CompilerPolicy.JANINO || policy == CompilerPolicy.DEFAULT) ? new JaninoClassCompiler(QueryClassLoader.this, debug) : null;
-      this.jdkClassCompiler = (policy == CompilerPolicy.JDK || policy == CompilerPolicy.DEFAULT) ? JDKClassCompiler.newInstance(QueryClassLoader.this, debug) : null;
-    }
-
-    private byte[][] getClassByteCode(ClassNames className, String sourceCode)
-        throws CompileException, ClassNotFoundException, ClassTransformationException, IOException {
-      AbstractClassCompiler classCompiler;
-      if (jdkClassCompiler != null &&
-          (policy == CompilerPolicy.JDK || (policy == CompilerPolicy.DEFAULT && sourceCode.length() > janinoThreshold))) {
-        classCompiler = jdkClassCompiler;
-      } else {
-        classCompiler = janinoClassCompiler;
-      }
-
-      byte[][] bc = classCompiler.getClassByteCode(className, sourceCode);
-      /*
-       * final String baseDir = System.getProperty("java.io.tmpdir") + File.separator + classCompiler.getClass().getSimpleName();
-       * File classFile = new File(baseDir + className.clazz);
-       * classFile.getParentFile().mkdirs();
-       * BufferedOutputStream out = new BufferedOutputStream(new FileOutputStream(classFile));
-       * out.write(bc[0]);
-       * out.close();
-       */
-      return bc;
-    }
-
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java
index ec5bfcd..1979db1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java
@@ -21,12 +21,25 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.drill.exec.compile.sig.SignatureHolder;
 
+/**
+ * Defines a code generation "template" which consist of:
+ * <ul>
+ * <li>An interface that defines the generated class.</li>
+ * <li>A template class which implements the interface to provide
+ * "generic" methods that need not be generated.</li>
+ * <li>A signature that lists the methods and vector holders used
+ * by the template.</li>
+ * </ul>
+ *
+ * @param <T> The template interface
+ */
+
 public class TemplateClassDefinition<T>{
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TemplateClassDefinition.class);
 
   private final Class<T> iface;
-  private final Class<?> template;
+  private final Class<? extends T> template;
   private final SignatureHolder signature;
   private static final AtomicLong classNumber = new AtomicLong(0);
 
@@ -41,7 +54,6 @@ public class TemplateClassDefinition<T>{
       logger.error("Failure while trying to build signature holder for signature. {}", template.getName(), ex);
     }
     this.signature = holder;
-
   }
 
   public long getNextClassNumber(){
@@ -52,6 +64,9 @@ public class TemplateClassDefinition<T>{
     return iface;
   }
 
+  public Class<? extends T> getTemplateClass() {
+    return template;
+  }
 
   public String getTemplateClassName() {
     return template.getName();
@@ -63,6 +78,14 @@ public class TemplateClassDefinition<T>{
 
   @Override
   public String toString() {
-    return "TemplateClassDefinition [template=" + template + ", signature=" + signature + "]";
+    StringBuilder buf = new StringBuilder();
+    buf.append("TemplateClassDefinition [interface=");
+    buf.append((iface == null) ? "null" : iface.getName());
+    buf.append(", template=");
+    buf.append((template == null) ? "null" : template.getName());
+    buf.append(", signature=\n");
+    buf.append(signature);
+    buf.append("]");
+    return buf.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java
index 9df346c..c83498a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java
@@ -82,7 +82,7 @@ public class CodeGeneratorMethod implements Iterable<CodeGeneratorArgument> {
 
   @Override
   public String toString() {
-    return "CodeGeneratorMethod [" + underlyingMethod.toGenericString() + "]";
+    return "CodeGeneratorMethod [" + ((underlyingMethod == null) ? "null" : underlyingMethod.toGenericString()) + "]";
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/GeneratorMapping.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/GeneratorMapping.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/GeneratorMapping.java
index 9c12116..b9b62a8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/GeneratorMapping.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/GeneratorMapping.java
@@ -21,6 +21,25 @@ import org.apache.drill.exec.expr.ClassGenerator.BlockType;
 
 import com.google.common.base.Preconditions;
 
+/**
+ * The code generator works with four conceptual methods which can
+ * have any actual names. This class identify which conceptual methods
+ * are in use and their actual names. Callers obtain the method
+ * names generically using the {@link BlockType} enum. There is,
+ * however, no way to check which methods are in use; the user of
+ * this method must already know this information from another
+ * source.
+ * <table>
+ * <tr><th>Conceptual Method</th>
+ *     <th>BlockType</th>
+ *     <th>Typical Drill Name</th></tr>
+ * <tr><td>setup</td><td>SETUP</td><td>doSetup</td></tr>
+ * <tr><td>eval</td><td>EVAL</td><td>doEval</td></tr>
+ * <tr><td>reset</td><td>RESET</td><td>?</td></tr>
+ * <tr><td>cleanup</td><td>CLEANUP</td><td>?</td></tr>
+ * </table>
+ */
+
 public class GeneratorMapping {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(GeneratorMapping.class);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
index 7fe8e3b..541a85f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
@@ -20,6 +20,8 @@ package org.apache.drill.exec.compile.sig;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -51,7 +53,6 @@ public class SignatureHolder implements Iterable<CodeGeneratorMethod> {
     return new SignatureHolder(signature, innerClasses.toArray(new SignatureHolder[innerClasses.size()]));
   }
 
-
   private SignatureHolder(Class<?> signature, SignatureHolder[] childHolders) {
     this.childHolders = childHolders;
     this.signature = signature;
@@ -67,6 +68,16 @@ public class SignatureHolder implements Iterable<CodeGeneratorMethod> {
       methodHolders.add(new CodeGeneratorMethod(m));
     }
 
+    // Alphabetize methods to ensure generated code is comparable.
+    // Also eases debugging as the generated code contain different method
+    // order from run to run.
+
+    Collections.sort( methodHolders, new Comparator<CodeGeneratorMethod>( ) {
+      @Override
+      public int compare(CodeGeneratorMethod o1, CodeGeneratorMethod o2) {
+        return o1.getMethodName().compareTo( o2.getMethodName() );
+      } } );
+
     methods = new CodeGeneratorMethod[methodHolders.size()+1];
     for (int i =0; i < methodHolders.size(); i++) {
       methods[i] = methodHolders.get(i);
@@ -99,7 +110,6 @@ public class SignatureHolder implements Iterable<CodeGeneratorMethod> {
     return methods.length;
   }
 
-
   public SignatureHolder[] getChildHolders() {
     return childHolders;
   }
@@ -114,9 +124,16 @@ public class SignatureHolder implements Iterable<CodeGeneratorMethod> {
 
   @Override
   public String toString() {
+    StringBuilder buf = new StringBuilder( );
+    buf.append( "SignatureHolder [methods=" );
     final int maxLen = 10;
-    return "SignatureHolder [methods="
-        + (methods != null ? Arrays.asList(methods).subList(0, Math.min(methods.length, maxLen)) : null) + "]";
+    for ( int i = 0;  i < maxLen  &&  i < methods.length; i++ ) {
+      if ( i > 0 ) {
+        buf.append( ", \n" );
+      }
+      buf.append( methods[i] );
+    }
+    buf.append( "]" );
+    return buf.toString();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
index 1f82682..96f14fb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
@@ -87,7 +87,9 @@ public class ClassGenerator<T>{
   }
 
   @SuppressWarnings("unchecked")
-  ClassGenerator(CodeGenerator<T> codeGenerator, MappingSet mappingSet, SignatureHolder signature, EvaluationVisitor eval, JDefinedClass clazz, JCodeModel model, OptionManager optionManager) throws JClassAlreadyExistsException {
+  ClassGenerator(CodeGenerator<T> codeGenerator, MappingSet mappingSet, SignatureHolder signature,
+                 EvaluationVisitor eval, JDefinedClass clazz, JCodeModel model,
+                 OptionManager optionManager) throws JClassAlreadyExistsException {
     this.codeGenerator = codeGenerator;
     this.clazz = clazz;
     this.mappings = mappingSet;
@@ -171,7 +173,7 @@ public class ClassGenerator<T>{
   }
 
   public JVar declareVectorValueSetupAndMember(String batchName, TypedFieldId fieldId) {
-    return declareVectorValueSetupAndMember( DirectExpression.direct(batchName), fieldId);
+    return declareVectorValueSetupAndMember(DirectExpression.direct(batchName), fieldId);
   }
 
   public JVar declareVectorValueSetupAndMember(DirectExpression batchName, TypedFieldId fieldId) {
@@ -202,26 +204,30 @@ public class ClassGenerator<T>{
 
     JInvocation invoke = batchName
         .invoke("getValueAccessorById") //
-        .arg( vvClass.dotclass())
+        .arg(vvClass.dotclass())
         .arg(fieldArr);
 
-    JVar obj = b.decl( //
-        objClass, //
-        getNextVar("tmp"), //
+    JVar obj = b.decl(
+        objClass,
+        getNextVar("tmp"),
         invoke.invoke(vectorAccess));
 
     b._if(obj.eq(JExpr._null()))._then()._throw(JExpr._new(t).arg(JExpr.lit(String.format("Failure while loading vector %s with id: %s.", vv.name(), fieldId.toString()))));
-    //b.assign(vv, JExpr.cast(retClass, ((JExpression) JExpr.cast(wrapperClass, obj) ).invoke(vectorAccess)));
-    b.assign(vv, JExpr.cast(retClass, obj ));
+    //b.assign(vv, JExpr.cast(retClass, ((JExpression) JExpr.cast(wrapperClass, obj)).invoke(vectorAccess)));
+    b.assign(vv, JExpr.cast(retClass, obj));
     vvDeclaration.put(setup, vv);
 
     return vv;
   }
 
   public enum BlkCreateMode {
-    TRUE,  // Create new block
-    FALSE, // Do not create block; put into existing block.
-    TRUE_IF_BOUND // Create new block only if # of expressions added hit upper-bound (ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE)
+    /** Create new block */
+    TRUE,
+    /** Do not create block; put into existing block. */
+    FALSE,
+    /** Create new block only if # of expressions added hit upper-bound
+     * ({@link ExecConstants#CODE_GEN_EXP_IN_METHOD_SIZE}). */
+    TRUE_IF_BOUND
   }
 
   public HoldingContainer addExpr(LogicalExpression ex) {
@@ -246,6 +252,13 @@ public class ClassGenerator<T>{
     rotateBlock(BlkCreateMode.TRUE);
   }
 
+  /**
+   * Create a new code block, closing the current block.
+   *
+   * @param mode the {@link BlkCreateMode block create mode}
+   * for the new block.
+   */
+
   private void rotateBlock(BlkCreateMode mode) {
     boolean blockRotated = false;
     for (LinkedList<SizedJBlock> b : blocks) {
@@ -361,7 +374,7 @@ public class ClassGenerator<T>{
     return this.workspaceVectors;
   }
 
-  private static class ValueVectorSetup{
+  private static class ValueVectorSetup {
     final DirectExpression batch;
     final TypedFieldId fieldId;
 
@@ -411,7 +424,11 @@ public class ClassGenerator<T>{
 
   }
 
-  public static class HoldingContainer{
+  /**
+   * Represents a (Nullable)?(Type)Holder instance.
+   */
+
+  public static class HoldingContainer {
     private final JVar holder;
     private final JFieldRef value;
     private final JFieldRef isSet;
@@ -483,10 +500,33 @@ public class ClassGenerator<T>{
     public TypeProtos.MinorType getMinorType() {
       return type.getMinorType();
     }
+
+    /**
+     * Convert holder to a string for debugging use.
+     */
+
+    @Override
+    public String toString() {
+      DebugStringBuilder buf = new DebugStringBuilder(this);
+      if (isConstant()) {
+        buf.append("const ");
+      }
+      buf.append(holder.type().fullName())
+        .append(" ")
+        .append(holder.name())
+        .append(", ")
+        .append(type.getMode().name())
+        .append(" ")
+        .append(type.getMinorType().name())
+        .append(", ");
+      holder.generate(buf.formatter());
+      buf.append(", ");
+      value.generate(buf.formatter());
+      return buf.toString();
+    }
   }
 
   public JType getHolderType(MajorType t) {
     return TypeHelper.getHolderType(model, t.getMinorType(), t.getMode());
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
index 12178ee..f50cfde 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
@@ -30,16 +30,30 @@ import com.sun.codemodel.JDefinedClass;
 import org.apache.drill.exec.server.options.OptionManager;
 
 /**
- * A code generator is responsible for generating the Java source code required to complete the implementation of an
- * abstract template. It is used with a class transformer to merge precompiled template code with runtime generated and
+ * A code generator is responsible for generating the Java source code required
+ * to complete the implementation of an abstract template.
+ * A code generator can contain one or more ClassGenerators that implement
+ * outer and inner classes associated with a particular runtime generated instance.
+ * <p>
+ * Drill supports two ways to generate and compile the code from a code
+ * generator: via byte-code manipulations or as "plain-old Java."
+ * <p>
+ * When using byte-code transformations, the code generator is used with a
+ * class transformer to merge precompiled template code with runtime generated and
  * compiled query specific code to create a runtime instance.
- *
- * A code generator can contain one or more ClassGenerators that implement outer and inner classes associated with a
- * particular runtime generated instance.
+ * <p>
+ * The code generator can optionally be marked as "plain-old Java" capable.
+ * This means that the generated code can be compiled directly as a Java
+ * class without the normal byte-code manipulations. Plain-old Java allows
+ * the option to persist, and debug, the generated code when building new
+ * generated classes or otherwise working with generated code. To turn
+ * on debugging, see the explanation in {@link ClassBuilder}.
  *
  * @param <T>
- *          The interface that results from compiling and merging the runtime code that is generated.
+ *          The interface that results from compiling and merging the runtime
+ *          code that is generated.
  */
+
 public class CodeGenerator<T> {
 
   private static final String PACKAGE_NAME = "org.apache.drill.exec.test.generated";
@@ -50,6 +64,21 @@ public class CodeGenerator<T> {
 
   private final JCodeModel model;
   private final ClassGenerator<T> rootGenerator;
+
+  /**
+   * True if the code generated for this class is suitable for compilation
+   * as a plain-old Java class.
+   */
+
+  private boolean plainOldJavaCapable;
+
+  /**
+   * True if the code generated for this class should actually be compiled
+   * via the plain-old Java mechanism. Considered only if the class is
+   * capable of this technique.
+   */
+
+  private boolean usePlainOldJava;
   private String generatedCode;
   private String generifiedCode;
 
@@ -58,7 +87,7 @@ public class CodeGenerator<T> {
   }
 
   CodeGenerator(MappingSet mappingSet, TemplateClassDefinition<T> definition,
-      FunctionImplementationRegistry funcRegistry, OptionManager optionManager) {
+     FunctionImplementationRegistry funcRegistry, OptionManager optionManager) {
     Preconditions.checkNotNull(definition.getSignature(),
         "The signature for defintion %s was incorrectly initialized.", definition);
     this.definition = definition;
@@ -67,6 +96,9 @@ public class CodeGenerator<T> {
     try {
       this.model = new JCodeModel();
       JDefinedClass clazz = model._package(PACKAGE_NAME)._class(className);
+      if ( isPlainOldJava( ) ) {
+        clazz._extends(definition.getTemplateClass( ) );
+      }
       rootGenerator = new ClassGenerator<>(this, mappingSet, definition.getSignature(), new EvaluationVisitor(
           funcRegistry), clazz, model, optionManager);
     } catch (JClassAlreadyExistsException e) {
@@ -74,19 +106,67 @@ public class CodeGenerator<T> {
     }
   }
 
+  /**
+   * Indicates that the code for this class can be generated using the
+   * "Plain Old Java" mechanism based on inheritance. The byte-code
+   * method is more lenient, so some code is missing some features such
+   * as proper exception labeling, etc. Set this option to true once
+   * the generation mechanism for a class has been cleaned up to work
+   * via the plain-old Java mechanism.
+   *
+   * @param flag true if the code generated from this instance is
+   * ready to be compiled as a plain-old Java class
+   */
+
+  public void plainOldJavaCapable(boolean flag) {
+    plainOldJavaCapable = flag;
+  }
+
+  /**
+   * Identifies that this generated class should be generated via the
+   * plain-old Java mechanism. This flag only has meaning if the
+   * generated class is capable of plain-old Java generation.
+   *
+   * @param flag true if the class should be generated and compiled
+   * as a plain-old Java class (rather than via byte-code manipulations)
+   */
+
+  public void preferPlainOldJava(boolean flag) {
+    usePlainOldJava = flag;
+  }
+
+  public boolean isPlainOldJava() {
+    return plainOldJavaCapable && usePlainOldJava;
+  }
+
   public ClassGenerator<T> getRoot() {
     return rootGenerator;
   }
 
-  public void generate() throws IOException {
+  public void generate() {
+
+    // If this generated class uses the "straight Java" technique
+    // (no byte code manipulation), then the class must extend the
+    // template so it plays by normal Java rules for finding the
+    // template methods via inheritance rather than via code injection.
+
+    if (isPlainOldJava()) {
+      rootGenerator.clazz._extends(definition.getTemplateClass( ));
+    }
+
     rootGenerator.flushCode();
 
     SingleClassStringWriter w = new SingleClassStringWriter();
-    model.build(w);
+    try {
+      model.build(w);
+    } catch (IOException e) {
+      // No I/O errors should occur during model building
+      // unless something is terribly wrong.
+      throw new IllegalStateException(e);
+    }
 
     this.generatedCode = w.getCode().toString();
     this.generifiedCode = generatedCode.replaceAll(this.className, "GenericGenerated");
-
   }
 
   public String generateAndGet() throws IOException {
@@ -156,7 +236,7 @@ public class CodeGenerator<T> {
       if (other.definition != null){
         return false;
       }
-    } else if (!definition.equals(other.definition)){
+    } else if (!definition.equals(other.definition)) {
       return false;
     }
     if (generifiedCode == null) {
@@ -164,7 +244,7 @@ public class CodeGenerator<T> {
         return false;
       }
 
-    } else if (!generifiedCode.equals(other.generifiedCode)){
+    } else if (!generifiedCode.equals(other.generifiedCode)) {
       return false;
     }
     return true;

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DebugStringBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DebugStringBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DebugStringBuilder.java
new file mode 100644
index 0000000..057c621
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DebugStringBuilder.java
@@ -0,0 +1,62 @@
+/**
+ * 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.io.PrintWriter;
+import java.io.StringWriter;
+
+import com.sun.codemodel.JFormatter;
+
+/**
+ * Utility class to build a debug string for an object
+ * in a standard format. That format is:
+ * <pre>[<i>className</i>:
+ *  <i>variable=<value>... ]</pre>
+ */
+
+public class DebugStringBuilder {
+
+  private final StringWriter strWriter;
+  private final PrintWriter writer;
+  private final JFormatter fmt;
+
+  public DebugStringBuilder( Object obj ) {
+    strWriter = new StringWriter( );
+    writer = new PrintWriter( strWriter );
+    writer.print( "[" );
+    writer.print( obj.getClass().getSimpleName() );
+    writer.print( ": " );
+    fmt = new JFormatter( writer );
+  }
+
+  public DebugStringBuilder append( String s ) {
+    writer.print( s );
+    return this;
+  }
+
+  @Override
+  public String toString( ) {
+    writer.print( "]" );
+    writer.flush();
+    return strWriter.toString();
+  }
+
+  public JFormatter formatter() { return fmt; }
+  public PrintWriter writer() { return writer; }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DirectExpression.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DirectExpression.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DirectExpression.java
index c4c3e7a..b99cd13 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DirectExpression.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DirectExpression.java
@@ -20,17 +20,24 @@ package org.apache.drill.exec.expr;
 import com.sun.codemodel.JExpressionImpl;
 import com.sun.codemodel.JFormatter;
 
-public class DirectExpression extends JExpressionImpl{
+/**
+ * Encapsulates a Java expression, defined as anything that is
+ * valid in the following code:<br>
+ * <code>(<i>expr</i>)</code>
+ */
+
+public class DirectExpression extends JExpressionImpl {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectExpression.class);
 
-  final String source;
+  private final String source;
 
   private DirectExpression(final String source) {
     super();
     this.source = source;
   }
 
-  public void generate( JFormatter f ) {
+  @Override
+  public void generate(JFormatter f) {
     f.p('(').p(source).p(')');
   }
 
@@ -67,5 +74,4 @@ public class DirectExpression extends JExpressionImpl{
     }
     return true;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index 8229b5b..e288095 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -313,7 +313,7 @@ public class FragmentContext implements AutoCloseable, UdfUtilities {
 
   public <T> T getImplementationClass(final CodeGenerator<T> cg)
       throws ClassTransformationException, IOException {
-    return context.getCompiler().getImplementationClass(cg);
+    return context.getCompiler().createInstance(cg);
   }
 
   public <T> List<T> getImplementationClass(final ClassGenerator<T> cg, final int instanceCount) throws ClassTransformationException, IOException {
@@ -321,7 +321,7 @@ public class FragmentContext implements AutoCloseable, UdfUtilities {
   }
 
   public <T> List<T> getImplementationClass(final CodeGenerator<T> cg, final int instanceCount) throws ClassTransformationException, IOException {
-    return context.getCompiler().getImplementationClass(cg, instanceCount);
+    return context.getCompiler().createInstances(cg, instanceCount);
   }
 
   public AccountingUserConnection getUserDataTunnel() {

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
index 92a7269..d6045fc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
@@ -17,22 +17,22 @@
  */
 package org.apache.drill.exec.ops;
 
-import com.google.common.util.concurrent.ListenableFuture;
-import io.netty.buffer.DrillBuf;
-
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.concurrent.Callable;
-import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.testing.ExecutionControls;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.testing.ExecutionControls;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 
+import com.google.common.util.concurrent.ListenableFuture;
+
+import io.netty.buffer.DrillBuf;
+
 public abstract class OperatorContext {
 
   public abstract DrillBuf replace(DrillBuf old, int newSize);

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
index 390b71c..c19cc1f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
@@ -89,33 +89,41 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
     scanDecodeExecutor = context.getDrillbitContext().getScanDecodeExecutor();
   }
 
+  @Override
   public DrillBuf replace(DrillBuf old, int newSize) {
     return manager.replace(old, newSize);
   }
 
+  @Override
   public DrillBuf getManagedBuffer() {
     return manager.getManagedBuffer();
   }
 
+  @Override
   public DrillBuf getManagedBuffer(int size) {
     return manager.getManagedBuffer(size);
   }
 
   // Allow an operator to use the thread pool
+  @Override
   public ExecutorService getExecutor() {
     return executor;
   }
+  @Override
   public ExecutorService getScanExecutor() {
     return scanExecutor;
   }
+  @Override
   public ExecutorService getScanDecodeExecutor() {
     return scanDecodeExecutor;
   }
 
+  @Override
   public ExecutionControls getExecutionControls() {
     return executionControls;
   }
 
+  @Override
   public BufferAllocator getAllocator() {
     if (allocator == null) {
       throw new UnsupportedOperationException("Operator context does not have an allocator");
@@ -151,10 +159,12 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
     closed = true;
   }
 
+  @Override
   public OperatorStats getStats() {
     return stats;
   }
 
+  @Override
   public <RESULT> ListenableFuture<RESULT> runCallableAs(final UserGroupInformation proxyUgi,
                                                          final Callable<RESULT> callable) {
     synchronized (this) {

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index 54aa72d..95d64bd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -711,18 +711,20 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     g.rotateBlock();
     g.getEvalBlock()._return(JExpr.lit(0));
 
+    cg.plainOldJavaCapable(true); // This class can generate plain-old Java.
+    // Uncomment out this line to debug the generated code.
+//  cg.preferPlainOldJava(true);
     return context.getImplementationClass(cg);
-
-
   }
 
   public SingleBatchSorter createNewSorter(FragmentContext context, VectorAccessible batch)
           throws ClassTransformationException, IOException, SchemaChangeException{
     CodeGenerator<SingleBatchSorter> cg = CodeGenerator.get(SingleBatchSorter.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
-    ClassGenerator<SingleBatchSorter> g = cg.getRoot();
-
-    generateComparisons(g, batch);
+    cg.plainOldJavaCapable(true); // This class can generate plain-old Java.
 
+    // Uncomment out this line to debug the generated code.
+//    cg.preferPlainOldJava(true);
+    generateComparisons(cg.getRoot(), batch);
     return context.getImplementationClass(cg);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/SingleBatchSorter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/SingleBatchSorter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/SingleBatchSorter.java
index b4986ba..e59d1b1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/SingleBatchSorter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/SingleBatchSorter.java
@@ -26,7 +26,7 @@ import org.apache.drill.exec.record.selection.SelectionVector2;
 
 public interface SingleBatchSorter {
   public void setup(FragmentContext context, SelectionVector2 vector2, VectorAccessible incoming) throws SchemaChangeException;
-  public void sort(SelectionVector2 vector2);
+  public void sort(SelectionVector2 vector2) throws SchemaChangeException;
 
   public static TemplateClassDefinition<SingleBatchSorter> TEMPLATE_DEFINITION = new TemplateClassDefinition<SingleBatchSorter>(SingleBatchSorter.class, SingleBatchSorterTemplate.class);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/SingleBatchSorterTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/SingleBatchSorterTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/SingleBatchSorterTemplate.java
index 9a6bc8c..5a8b0c3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/SingleBatchSorterTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/SingleBatchSorterTemplate.java
@@ -37,6 +37,7 @@ public abstract class SingleBatchSorterTemplate implements SingleBatchSorter, In
 
   private SelectionVector2 vector2;
 
+  @Override
   public void setup(FragmentContext context, SelectionVector2 vector2, VectorAccessible incoming) throws SchemaChangeException{
     Preconditions.checkNotNull(vector2);
     this.vector2 = vector2;
@@ -68,10 +69,14 @@ public abstract class SingleBatchSorterTemplate implements SingleBatchSorter, In
   public int compare(int leftIndex, int rightIndex) {
     char sv1 = vector2.getIndex(leftIndex);
     char sv2 = vector2.getIndex(rightIndex);
-    return doEval(sv1, sv2);
+    try {
+      return doEval(sv1, sv2);
+    } catch (SchemaChangeException e) {
+      throw new RuntimeException( e );
+    }
   }
 
-  public abstract void doSetup(@Named("context") FragmentContext context, @Named("incoming") VectorAccessible incoming, @Named("outgoing") RecordBatch outgoing);
-  public abstract int doEval(@Named("leftIndex") char leftIndex, @Named("rightIndex") char rightIndex);
+  public abstract void doSetup(@Named("context") FragmentContext context, @Named("incoming") VectorAccessible incoming, @Named("outgoing") RecordBatch outgoing) throws SchemaChangeException;
+  public abstract int doEval(@Named("leftIndex") char leftIndex, @Named("rightIndex") char rightIndex) throws SchemaChangeException;
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 2c322c7..57c72d5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -32,8 +32,8 @@ import org.apache.drill.common.config.LogicalPlanPersistence;
 import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.compile.ClassCompilerSelector;
 import org.apache.drill.exec.compile.ClassTransformer;
-import org.apache.drill.exec.compile.QueryClassLoader;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
 import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
@@ -145,9 +145,9 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       ExecConstants.ADMIN_USERS_VALIDATOR,
       ExecConstants.ADMIN_USER_GROUPS_VALIDATOR,
       ExecConstants.IMPERSONATION_POLICY_VALIDATOR,
-      QueryClassLoader.JAVA_COMPILER_VALIDATOR,
-      QueryClassLoader.JAVA_COMPILER_JANINO_MAXSIZE,
-      QueryClassLoader.JAVA_COMPILER_DEBUG,
+      ClassCompilerSelector.JAVA_COMPILER_VALIDATOR,
+      ClassCompilerSelector.JAVA_COMPILER_JANINO_MAXSIZE,
+      ClassCompilerSelector.JAVA_COMPILER_DEBUG,
       ExecConstants.ENABLE_VERBOSE_ERRORS,
       ExecConstants.ENABLE_WINDOW_FUNCTIONS_VALIDATOR,
       ClassTransformer.SCALAR_REPLACEMENT_VALIDATOR,

http://git-wip-us.apache.org/repos/asf/drill/blob/bbcf4b76/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index ab73cec..7e6d7c6 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -166,7 +166,11 @@ drill.exec: {
     compiler: "DEFAULT",
     debug: true,
     janino_maxsize: 262144,
-    cache_max_size: 1000
+    cache_max_size: 1000,
+    // Enable to write generated source to disk. See ClassBuilder
+    save_source: false,
+    // Where to save the generated source. See ClassBuilder
+    code_dir: "/tmp/drill/codegen"
   },
   sort: {
     purge.threshold : 1000,