You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ve...@apache.org on 2015/03/17 00:30:47 UTC

[3/3] drill git commit: DRILL-1385, along with some cleanup

DRILL-1385, along with some cleanup

Cleaned up option handling. This includes using finals, making member variables
private whenever possible, and some formatting.
- fixed a bug in the string formatting for the double range validator
- OptionValidator, OptionValue, and their implementations now conspire not to
  allow the creation of malformed options because the OptionType has been added
  to validator calls to handle OptionValues that are created on demand.

Started with updated byte code rewrite from Jacques
Fixed several problems with scalar value replacement:
- use consistent ASM api version throughout
- stop using deprecated ASM methods (actually causes bugs)
  - visitMethodInsn()
- added a couple of missing super.visitEnd()s
- fixed a couple of minor FindBugs issues
- accounted for required stack size increases when replacing holders for
  longs and doubles
- added accounting for frame offsets to cope with long and double local
  variables and value holder members
- fixed a few minor bugs found with FindBugs
- stop using carrotlabs' hash map lget() method on shared constant data
- fixed an incorrect use of DUP2 on objectrefs when copying long or double
  holder members into locals
- fixed a problem with redundant POP instructions left behind after replacement
- fixed a problem with incorrect DUPs in multiple assignment statements
- fixed a problem with DUP_X1 replacement when handling constants in multiple
  assignment statements
- fixed a problem with non-replaced holder member post-decrements
- don't replace holders passed to static functions as "out" parameters
  (common with Accessors on repeated value vectors)
- increased the maximum required stack size when transferring holder members to
  locals
- changed the code generation block type mappings for constants for external
  sorts
- fixed problems handling constant and non-constant member variables in
  operator classes
  - in general, if a holder is assigned to or from an operator member variable,
    it can't be replaced (at least not until we replace those as well)
  - Use a derived ASM Analyzer (MethodAnalyzer) and Frame
    (AssignmentTrackingFrame) in order to establish relationships between
    assignments of holders through chains of local variables. This effectively
    back-propagates non-replaceability attributes so that if a holder variable
    that can't be replaced is assigned to from another holder variable, that
    second one cannot be replaced either, and so on through longer chains of
    assignments.
- code for dumping generated source code
- MergeAdapter dumps before and after results of scalar replacement
  (if it's on)
- fixed some problems in ReplacingBasicValue by replacing HashSet with
  IdentityHashMap
- made loggers private
- added a retry strategy for scalar replacement
  if a scalar replacement code rewriting fails, then this will try to
  regenerate the bytecode again without the scalar replacement.
- bytecode verification is always on now (required for the retry strategy)
- use system option to determine whether scalar replacement should be used
  - default option: if scalar replacement fails, retry without it
  - force replacement on or off
- unit tests for the retry strategy are based on a single known failure case,
  covered by DRILL-2326.
  - add tests TestConvertFunctions to test the three scalar replacement options
    for the failing test case (testVarCharReturnTripConvertLogical)
- made it possible to set a SYSTEM option as a java property in Drillbit
- added a command line argument to force scalar replacement to be on during
  testing in the rootmost pom.xml

In the course of this, added increased checking of intermediate stages of code
rewriting, as well as logging of classes that cause failures.
- work around a bug in ASM's CheckClassAdapter that doesn't allow for checking
  of inner classes
Added comments, tidied up formatting, and added "final" in a number of places.

Signed-off-by: vkorukanti <ve...@gmail.com>


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

Branch: refs/heads/master
Commit: 7ea212a26dca9b5eca2719300df27751e73dbbcf
Parents: 34932e1
Author: Chris Westin <cw...@yahoo.com>
Authored: Wed Feb 25 14:53:41 2015 -0800
Committer: vkorukanti <ve...@gmail.com>
Committed: Mon Mar 16 14:34:15 2015 -0700

----------------------------------------------------------------------
 .../apache/drill/common/config/DrillConfig.java |   4 +-
 .../apache/drill/common/util/PathScanner.java   |  27 +-
 .../org/apache/drill/exec/ExecConstants.java    |   2 +
 .../exec/compile/AbstractClassCompiler.java     |  14 +
 .../org/apache/drill/exec/compile/AsmUtil.java  | 173 ++++++
 .../exec/compile/CheckClassVisitorFsm.java      | 149 ++++++
 .../exec/compile/CheckMethodVisitorFsm.java     | 300 +++++++++++
 .../drill/exec/compile/ClassTransformer.java    | 126 ++++-
 .../apache/drill/exec/compile/CodeCompiler.java |  59 ++-
 .../drill/exec/compile/CompilationConfig.java   |  28 +
 .../exec/compile/DrillCheckClassAdapter.java    | 118 +++++
 .../exec/compile/DrillInitMethodVisitor.java    |  13 +-
 .../apache/drill/exec/compile/FsmCursor.java    |  63 +++
 .../drill/exec/compile/FsmDescriptor.java       | 138 +++++
 .../exec/compile/InnerClassAccessStripper.java  |  89 ++++
 .../drill/exec/compile/JaninoClassCompiler.java |   1 -
 .../apache/drill/exec/compile/LogWriter.java    | 133 +++++
 .../apache/drill/exec/compile/MergeAdapter.java | 132 +++--
 .../drill/exec/compile/QueryClassLoader.java    |   2 +-
 .../exec/compile/RetargetableClassVisitor.java  |  62 +++
 .../exec/compile/TemplateClassDefinition.java   |   8 -
 .../exec/compile/bytecode/AloadPopRemover.java  | 333 ++++++++++++
 .../exec/compile/bytecode/DirectSorter.java     |   3 +-
 .../compile/bytecode/InstructionModifier.java   | 526 ++++++++++++++-----
 .../exec/compile/bytecode/MethodAnalyzer.java   | 105 ++++
 .../compile/bytecode/ReplacingBasicValue.java   | 413 ++++++++++++++-
 .../compile/bytecode/ReplacingInterpreter.java  | 129 ++++-
 .../compile/bytecode/ScalarReplacementNode.java |  61 ++-
 .../bytecode/ScalarReplacementTypes.java        |  25 +-
 .../bytecode/TrackingInstructionList.java       |  41 +-
 .../exec/compile/bytecode/ValueHolderIden.java  | 189 +++++--
 .../bytecode/ValueHolderReplacementVisitor.java |  41 +-
 .../exec/compile/sig/GeneratorMapping.java      |  27 +-
 .../exec/expr/fn/DrillFunctionRegistry.java     |   4 +
 .../expr/fn/FunctionImplementationRegistry.java |   6 +
 .../physical/impl/xsort/ExternalSortBatch.java  |   2 +-
 .../apache/drill/exec/record/SchemaBuilder.java |   2 +-
 .../org/apache/drill/exec/server/Drillbit.java  |  77 ++-
 .../drill/exec/server/DrillbitContext.java      |   5 +-
 .../server/options/DrillConfigIterator.java     |  32 +-
 .../server/options/FallbackOptionManager.java   |  19 +-
 .../exec/server/options/OptionManager.java      |   9 +-
 .../exec/server/options/OptionValidator.java    |  12 +-
 .../drill/exec/server/options/OptionValue.java  |  25 +-
 .../server/options/SystemOptionManager.java     |  87 ++-
 .../exec/server/options/TypeValidators.java     |  79 ++-
 .../java/org/apache/drill/BaseTestQuery.java    |   3 +-
 .../java/org/apache/drill/TestBugFixes.java     |   1 +
 .../org/apache/drill/TestTpchDistributed.java   |   6 +-
 .../exec/compile/TestClassTransformation.java   |   2 +-
 .../compile/bytecode/ReplaceMethodInvoke.java   |  73 +--
 .../fn/impl/TestSimpleRepeatedFunctions.java    |   9 +-
 .../physical/impl/TestConvertFunctions.java     | 111 +++-
 pom.xml                                         |   1 +
 54 files changed, 3471 insertions(+), 628 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
index dee8a8a..e8b2478 100644
--- a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
+++ b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
@@ -58,7 +58,7 @@ public final class DrillConfig extends NestedConfig{
   @VisibleForTesting
   public DrillConfig(Config config, boolean enableServer) {
     super(config);
-
+    logger.debug("Setting up config object.");
     mapper = new ObjectMapper();
 
     if (enableServer) {
@@ -80,7 +80,7 @@ public final class DrillConfig extends NestedConfig{
 
     RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean();
     this.startupArguments = ImmutableList.copyOf(bean.getInputArguments());
-
+    logger.debug("Config object initialized.");
   };
 
   public List<String> getStartupArguments() {

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/common/src/main/java/org/apache/drill/common/util/PathScanner.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/util/PathScanner.java b/common/src/main/java/org/apache/drill/common/util/PathScanner.java
index 6223777..730c770 100644
--- a/common/src/main/java/org/apache/drill/common/util/PathScanner.java
+++ b/common/src/main/java/org/apache/drill/common/util/PathScanner.java
@@ -28,6 +28,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.common.config.CommonConstants;
 import org.reflections.Reflections;
@@ -37,6 +38,7 @@ import org.reflections.scanners.TypeAnnotationsScanner;
 import org.reflections.util.ClasspathHelper;
 import org.reflections.util.ConfigurationBuilder;
 
+import com.google.common.base.Stopwatch;
 import com.google.common.collect.Sets;
 
 public class PathScanner {
@@ -73,20 +75,27 @@ public class PathScanner {
 
   public static <T> Class<?>[] scanForImplementationsArr(Class<T> baseClass, final List<String> scanPackages) {
     Collection<Class<? extends T>> imps = scanForImplementations(baseClass, scanPackages);
-    return imps.toArray(new Class<?>[imps.size()]);
+    Class<?>[] arr = imps.toArray(new Class<?>[imps.size()]);
+    return arr;
   }
 
   public static <T> Set<Class<? extends T>> scanForImplementations(Class<T> baseClass, final List<String> scanPackages) {
-    synchronized(SYNC) {
-      Set<Class<? extends T>> classes = getReflections().getSubTypesOf(baseClass);
-      for (Iterator<Class<? extends T>> i = classes.iterator(); i.hasNext();) {
-        Class<? extends T> c = i.next();
-        assert baseClass.isAssignableFrom(c);
-        if (Modifier.isAbstract(c.getModifiers())) {
-          i.remove();
+    Stopwatch w = new Stopwatch().start();
+    try {
+      synchronized(SYNC) {
+
+        Set<Class<? extends T>> classes = getReflections().getSubTypesOf(baseClass);
+        for (Iterator<Class<? extends T>> i = classes.iterator(); i.hasNext();) {
+          Class<? extends T> c = i.next();
+          assert baseClass.isAssignableFrom(c);
+          if (Modifier.isAbstract(c.getModifiers())) {
+            i.remove();
+          }
         }
+        return classes;
       }
-      return classes;
+    } finally{
+      logger.debug("Classpath scanning took {}ms", w.elapsed(TimeUnit.MILLISECONDS));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/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 33b05af..93d06f0 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
@@ -100,6 +100,8 @@ public interface ExecConstants {
   public static String PARQUET_NEW_RECORD_READER = "store.parquet.use_new_reader";
   public static OptionValidator PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR = new BooleanValidator(PARQUET_NEW_RECORD_READER, false);
 
+  public static OptionValidator COMPILE_SCALAR_REPLACEMENT = new BooleanValidator("exec.compile.scalar_replacement", false);
+
   public static String JSON_ALL_TEXT_MODE = "store.json.all_text_mode";
   public static OptionValidator JSON_READER_ALL_TEXT_MODE_VALIDATOR = new BooleanValidator(JSON_ALL_TEXT_MODE, false);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/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 f4a3cc9..a5c96e3 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
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.compile;
 
+import java.io.File;
+import java.io.FileWriter;
 import java.io.IOException;
 
 import org.apache.drill.common.util.DrillStringUtils;
@@ -35,6 +37,18 @@ public abstract class AbstractClassCompiler {
       throws CompileException, IOException, ClassNotFoundException, ClassTransformationException {
     if (getLogger().isDebugEnabled()) {
       getLogger().debug("Compiling (source size={}):\n{}", DrillStringUtils.readable(sourceCode.length()), prefixLineNumbers(sourceCode));
+
+/* uncomment this to get a dump of the generated source in /tmp
+      // This can be used to write out the generated operator classes for debugging purposes
+      // TODO: should these be put into a directory named with the query id and/or fragment id
+      final int lastSlash = className.slash.lastIndexOf('/');
+      final File dir = new File("/tmp", className.slash.substring(0, lastSlash));
+      dir.mkdirs();
+      final File file = new File(dir, className.slash.substring(lastSlash + 1) + ".java");
+      final FileWriter writer = new FileWriter(file);
+      writer.write(sourceCode);
+      writer.close();
+*/
     }
     return getByteCode(className, sourceCode);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AsmUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AsmUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AsmUtil.java
new file mode 100644
index 0000000..032aebd
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AsmUtil.java
@@ -0,0 +1,173 @@
+/**
+ * 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.PrintWriter;
+import java.io.StringWriter;
+
+import org.objectweb.asm.ClassReader;
+import org.objectweb.asm.ClassWriter;
+import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.tree.ClassNode;
+import org.objectweb.asm.util.TraceClassVisitor;
+import org.slf4j.Logger;
+
+/**
+ * Utilities commonly used with ASM.
+ *
+ * <p>There are several class verification utilities which use
+ * CheckClassAdapter (DrillCheckClassAdapter) to ensure classes are well-formed;
+ * these are packaged as boolean functions so that they can be used in assertions.
+ */
+public class AsmUtil {
+  // This class only contains static utilities.
+  private AsmUtil() {
+  }
+
+  /**
+   * Check to see if a class is well-formed.
+   *
+   * @param classNode the class to check
+   * @param logTag a tag to print to the log if a problem is found
+   * @param logger the logger to write to if a problem is found
+   * @return true if the class is ok, false otherwise
+   */
+  public static boolean isClassOk(final Logger logger, final ClassNode classNode, final String logTag) {
+    final StringWriter sw = new StringWriter();
+    final ClassWriter verifyWriter = new ClassWriter(ClassWriter.COMPUTE_FRAMES);
+    classNode.accept(verifyWriter);
+    final ClassReader ver = new ClassReader(verifyWriter.toByteArray());
+    try {
+      DrillCheckClassAdapter.verify(ver, false, new PrintWriter(sw));
+    } catch(Exception e) {
+      logger.info("Caught exception verifying class:");
+      logClass(logger, logTag, classNode);
+      throw e;
+    }
+    final String output = sw.toString();
+    if (!output.isEmpty()) {
+      logger.info("Invalid class:\n" +  output);
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Check to see if a class is well-formed.
+   *
+   * @param classBytes the bytecode of the class to check
+   * @param logTag a tag to print to the log if a problem is found
+   * @param logger the logger to write to if a problem is found
+   * @return true if the class is ok, false otherwise
+   */
+  public static boolean isClassBytesOk(final Logger logger, final byte[] classBytes, final String logTag) {
+    final ClassNode classNode = classFromBytes(classBytes);
+    return isClassOk(logger, classNode, logTag);
+  }
+
+  /**
+   * Create a ClassNode from bytecode.
+   *
+   * @param classBytes the bytecode
+   * @return the ClassNode
+   */
+  public static ClassNode classFromBytes(final byte[] classBytes) {
+    final ClassNode classNode = new ClassNode(CompilationConfig.ASM_API_VERSION);
+    final ClassReader classReader = new ClassReader(classBytes);
+    classReader.accept(classNode, 0);
+    return classNode;
+  }
+
+  /**
+   * Write a class to the log.
+   *
+   * <p>Writes at level DEBUG.
+   *
+   * @param logTag a tag to print to the log
+   * @param classNode the class
+   * @param logger the logger to write to
+   */
+  public static void logClass(final Logger logger, final String logTag, final ClassNode classNode) {
+    logger.debug(logTag);
+    final StringWriter stringWriter = new StringWriter();
+    final PrintWriter printWriter = new PrintWriter(stringWriter);
+    final TraceClassVisitor traceClassVisitor = new TraceClassVisitor(printWriter);
+    classNode.accept(traceClassVisitor);
+    logger.debug(stringWriter.toString());
+  }
+
+  /**
+   * Write a class to the log.
+   *
+   * <p>Writes at level DEBUG.
+   *
+   * @param logTag a tag to print to the log
+   * @param classBytes the class' bytecode
+   * @param logger the logger to write to
+   */
+  public static void logClassFromBytes(final Logger logger, final String logTag, final byte[] classBytes) {
+    final ClassNode classNode = classFromBytes(classBytes);
+    logClass(logger, logTag, classNode);
+  }
+
+  /**
+   * Determine if the given opcode is a load of a constant (xCONST_y).
+   *
+   * @param opcode the opcode
+   * @return true if the opcode is one of the constant loading ones, false otherwise
+   */
+  public static boolean isXconst(final int opcode) {
+    switch(opcode) {
+    case Opcodes.ICONST_0:
+    case Opcodes.ICONST_1:
+    case Opcodes.ICONST_2:
+    case Opcodes.ICONST_3:
+    case Opcodes.ICONST_4:
+    case Opcodes.ICONST_5:
+    case Opcodes.ICONST_M1:
+    case Opcodes.DCONST_0:
+    case Opcodes.DCONST_1:
+    case Opcodes.FCONST_0:
+    case Opcodes.FCONST_1:
+    case Opcodes.LCONST_0:
+    case Opcodes.LCONST_1:
+      return true;
+    }
+
+    return false;
+  }
+
+  /**
+   * Determine if the given opcode is an ADD of some kind (xADD).
+   *
+   * @param opcode the opcode
+   * @return true if the opcode is one of the ADDs, false otherwise
+   */
+  public static boolean isXadd(final int opcode) {
+    switch(opcode) {
+    case Opcodes.IADD:
+    case Opcodes.DADD:
+    case Opcodes.FADD:
+    case Opcodes.LADD:
+      return true;
+    }
+
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CheckClassVisitorFsm.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CheckClassVisitorFsm.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CheckClassVisitorFsm.java
new file mode 100644
index 0000000..d43824a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CheckClassVisitorFsm.java
@@ -0,0 +1,149 @@
+/**
+ * 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.util.HashMap;
+
+import org.objectweb.asm.AnnotationVisitor;
+import org.objectweb.asm.Attribute;
+import org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.FieldVisitor;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.TypePath;
+
+/**
+ * A ClassVisitor that verifies the required call sequence described in
+ * http://asm.ow2.org/asm50/javadoc/user/org/objectweb/asm/ClassVisitor.html .
+ *
+ * <p>
+ * There is no CheckAnnotationVisitorFsm at this time. There is no
+ * CheckFieldVisitorFsm at this time.
+ */
+public class CheckClassVisitorFsm extends ClassVisitor {
+  /*
+   * From
+   * http://asm.ow2.org/asm50/javadoc/user/org/objectweb/asm/ClassVisitor.html
+   * "A visitor to visit a Java class. The methods of this class must be called
+   * in the following order: visit [ visitSource ] [ visitOuterClass ] (
+   * visitAnnotation | visitTypeAnnotation | visitAttribute )* ( visitInnerClass
+   * | visitField | visitMethod )* visitEnd."
+   */
+  private final static FsmDescriptor fsmDescriptor = createFsmDescriptor();
+
+  private static FsmDescriptor createFsmDescriptor() {
+    final HashMap<String, Character> tokenMap = new HashMap<>();
+    tokenMap.put("visit", 'v');
+    tokenMap.put("visitSource", 'S');
+    tokenMap.put("visitOuterClass", 'O');
+    tokenMap.put("visitAnnotation", 'A');
+    tokenMap.put("visitTypeAnnotation", 'T');
+    tokenMap.put("visitAttribute", 'R');
+    tokenMap.put("visitInnerClass", 'I');
+    tokenMap.put("visitField", 'F');
+    tokenMap.put("visitMethod", 'M');
+    tokenMap.put("visitEnd", 'E');
+
+    return new FsmDescriptor(tokenMap, "vS?+O?+(A|T|R)*+(I|F|M)*+E", "visitEnd");
+  }
+
+  private final FsmCursor fsmCursor;
+
+  /**
+   * See {@link org.objectweb.asm.ClassVisitor#ClassVisitor(int, ClassVisitor)}.
+   */
+  public CheckClassVisitorFsm(final int api, final ClassVisitor cv) {
+    super(api, cv);
+    fsmCursor = fsmDescriptor.createCursor();
+  }
+
+  @Override
+  public void visit(final int version, final int access, final String name,
+      final String signature, final String superName, final String[] interfaces) {
+    fsmCursor.transition("visit");
+    super.visit(version, access, name, signature, superName, interfaces);
+  }
+
+  @Override
+  public AnnotationVisitor visitAnnotation(final String desc,
+      final boolean visible) {
+    fsmCursor.transition("visitAnnotation");
+    final AnnotationVisitor annotationVisitor = super.visitAnnotation(desc,
+        visible);
+    return annotationVisitor; // TODO: add CheckAnnotationVisitorFsm
+  }
+
+  @Override
+  public void visitAttribute(final Attribute attr) {
+    fsmCursor.transition("visitAttribute");
+    super.visitAttribute(attr);
+  }
+
+  @Override
+  public void visitEnd() {
+    fsmCursor.transition("visitEnd");
+    fsmCursor.reset();
+    super.visitEnd();
+  }
+
+  @Override
+  public FieldVisitor visitField(final int access, final String name,
+      final String desc, final String signature, final Object value) {
+    fsmCursor.transition("visitField");
+    final FieldVisitor fieldVisitor = super.visitField(access, name, desc,
+        signature, value);
+    return fieldVisitor; // TODO: add CheckFieldVisitorFsm
+  }
+
+  @Override
+  public void visitInnerClass(final String name, final String outerName,
+      final String innerName, final int access) {
+    fsmCursor.transition("visitInnerClass");
+    super.visitInnerClass(name, outerName, innerName, access);
+  }
+
+  @Override
+  public MethodVisitor visitMethod(final int access, final String name,
+      final String desc, final String signature, final String[] exceptions) {
+    fsmCursor.transition("visitMethod");
+    final MethodVisitor methodVisitor = super.visitMethod(access, name, desc,
+        signature, exceptions);
+    return new CheckMethodVisitorFsm(api, methodVisitor);
+  }
+
+  @Override
+  public void visitOuterClass(final String owner, final String name,
+      final String desc) {
+    fsmCursor.transition("visitOuterClass");
+    super.visitOuterClass(owner, name, desc);
+  }
+
+  @Override
+  public void visitSource(final String source, final String debug) {
+    fsmCursor.transition("visitSource");
+    super.visitSource(source, debug);
+  }
+
+  @Override
+  public AnnotationVisitor visitTypeAnnotation(final int typeRef,
+      final TypePath typePath, final String desc, final boolean visible) {
+    fsmCursor.transition("visitTypeAnnotation");
+    final AnnotationVisitor annotationVisitor = super.visitTypeAnnotation(
+        typeRef, typePath, desc, visible);
+    return annotationVisitor; // TODO: add CheckAnnotationVisitorFsm
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CheckMethodVisitorFsm.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CheckMethodVisitorFsm.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CheckMethodVisitorFsm.java
new file mode 100644
index 0000000..ceb1ae3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CheckMethodVisitorFsm.java
@@ -0,0 +1,300 @@
+/**
+ * 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.util.HashMap;
+
+import org.objectweb.asm.AnnotationVisitor;
+import org.objectweb.asm.Attribute;
+import org.objectweb.asm.Handle;
+import org.objectweb.asm.Label;
+import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.TypePath;
+
+/**
+ * A MethodVisitor that verifies the required call sequence according to
+ * http://asm.ow2.org/asm50/javadoc/user/org/objectweb/asm/MethodVisitor.html .
+ *
+ * <p>
+ * There is no CheckAnnotationVisitorFsm at this time.
+ */
+public class CheckMethodVisitorFsm extends MethodVisitor {
+  /*
+   * From
+   * http://asm.ow2.org/asm50/javadoc/user/org/objectweb/asm/MethodVisitor.html:
+   * "The methods of this class must be called in the following order: (
+   * visitParameter )* [ visitAnnotationDefault ] ( visitAnnotation |
+   * visitTypeAnnotation | visitAttribute )* [ visitCode ( visitFrame |
+   * visitXInsn | visitLabel | visitInsnAnnotation | visitTryCatchBlock |
+   * visitTryCatchBlockAnnotation | visitLocalVariable |
+   * visitLocalVariableAnnotation | visitLineNumber )* visitMaxs ] visitEnd.
+   */
+  private final static FsmDescriptor fsmDescriptor = createFsmDescriptor();
+
+  private static FsmDescriptor createFsmDescriptor() {
+    final HashMap<String, Character> tokenMap = new HashMap<>();
+    tokenMap.put("visitParameter", 'P');
+    tokenMap.put("visitAnnotationDefault", 'D');
+    tokenMap.put("visitAnnotation", 'A');
+    tokenMap.put("visitTypeAnnotation", 'T');
+    tokenMap.put("visitAttribute", 'R');
+    tokenMap.put("visitCode", 'C');
+    tokenMap.put("visitFrame", 'F');
+    tokenMap.put("visitXInsn", 'X'); // represents all Insn calls
+    tokenMap.put("visitLabel", 'L');
+    tokenMap.put("visitInsnAnnotation", 'I');
+    tokenMap.put("visitTryCatchBlock", 'B');
+    tokenMap.put("visitTryCatchBlockAnnotation", 'E'); // "Exception"
+    tokenMap.put("visitLocalVariable", 'V');
+    tokenMap.put("visitLocalVariableAnnotation", 'N'); // "Note"
+    tokenMap.put("visitLineNumber", '#');
+    tokenMap.put("visitMaxs", 'M');
+    tokenMap.put("visitEnd", 'e');
+
+    return new FsmDescriptor(tokenMap,
+        "P*+D?+(A|T|R)*+(C(F|X|L|I|B|E|V|N|\\#)*+M)?+e", "visitEnd");
+  }
+
+  private final FsmCursor fsmCursor;
+
+  /**
+   * See {@link org.objectweb.asm.MethodVisitor#MethodVisitor(int, MethodVisitor)}.
+   */
+  public CheckMethodVisitorFsm(final int api, final MethodVisitor mv) {
+    super(api, mv);
+    fsmCursor = fsmDescriptor.createCursor();
+  }
+
+  @Override
+  public AnnotationVisitor visitAnnotation(final String desc,
+      final boolean visible) {
+    fsmCursor.transition("visitAnnotation");
+    final AnnotationVisitor annotationVisitor = super.visitAnnotation(desc,
+        visible);
+    return annotationVisitor; // TODO: add CheckAnnotationVisitorFsm
+  }
+
+  @Override
+  public AnnotationVisitor visitAnnotationDefault() {
+    fsmCursor.transition("visitAnnotationDefault");
+    final AnnotationVisitor annotationVisitor = super.visitAnnotationDefault();
+    return annotationVisitor; // TODO: add CheckAnnotationVisitorFsm
+  }
+
+  @Override
+  public void visitAttribute(final Attribute attr) {
+    fsmCursor.transition("visitAttribute");
+    super.visitAttribute(attr);
+  }
+
+  @Override
+  public void visitCode() {
+    fsmCursor.transition("visitCode");
+    super.visitCode();
+  }
+
+  @Override
+  public void visitEnd() {
+    fsmCursor.transition("visitEnd");
+    super.visitEnd();
+  }
+
+  @Override
+  public void visitFieldInsn(final int opcode, final String owner,
+      final String name, final String desc) {
+    fsmCursor.transition("visitXInsn");
+    super.visitFieldInsn(opcode, owner, name, desc);
+  }
+
+  @Override
+  public void visitFrame(final int type, final int nLocal,
+      final Object[] local, final int nStack, final Object[] stack) {
+    fsmCursor.transition("visitFrame");
+    super.visitFrame(type, nLocal, local, nStack, stack);
+  }
+
+  @Override
+  public void visitIincInsn(final int var, final int increment) {
+    fsmCursor.transition("visitXInsn");
+    super.visitIincInsn(var, increment);
+  }
+
+  @Override
+  public void visitInsn(final int opcode) {
+    fsmCursor.transition("visitXInsn");
+    super.visitInsn(opcode);
+  }
+
+  @Override
+  public AnnotationVisitor visitInsnAnnotation(final int typeRef,
+      final TypePath typePath, final String desc, final boolean visible) {
+    fsmCursor.transition("visitInsnAnnotation");
+    final AnnotationVisitor annotationVisitor = super.visitInsnAnnotation(
+        typeRef, typePath, desc, visible);
+    return annotationVisitor; // TODO: add CheckAnnotationVisitorFsm
+  }
+
+  @Override
+  public void visitIntInsn(final int opcode, final int operand) {
+    fsmCursor.transition("visitXInsn");
+    super.visitIntInsn(opcode, operand);
+  }
+
+  @Override
+  public void visitInvokeDynamicInsn(final String name, final String desc,
+      final Handle bsm, final Object... bsmArgs) {
+    fsmCursor.transition("visitXInsn");
+    super.visitInvokeDynamicInsn(name, desc, bsm, bsmArgs);
+  }
+
+  @Override
+  public void visitJumpInsn(final int opcode, final Label label) {
+    fsmCursor.transition("visitXInsn");
+    super.visitJumpInsn(opcode, label);
+  }
+
+  @Override
+  public void visitLabel(final Label label) {
+    fsmCursor.transition("visitLabel");
+    super.visitLabel(label);
+  }
+
+  @Override
+  public void visitLdcInsn(final Object cst) {
+    fsmCursor.transition("visitXInsn");
+    super.visitLdcInsn(cst);
+  }
+
+  @Override
+  public void visitLineNumber(final int line, final Label start) {
+    fsmCursor.transition("visitLineNumber");
+    super.visitLineNumber(line, start);
+  }
+
+  @Override
+  public void visitLocalVariable(final String name, final String desc,
+      final String signature, final Label start, final Label end,
+      final int index) {
+    fsmCursor.transition("visitLocalVariable");
+    super.visitLocalVariable(name, desc, signature, start, end, index);
+  }
+
+  @Override
+  public AnnotationVisitor visitLocalVariableAnnotation(final int typeRef,
+      final TypePath typePath, final Label[] start, final Label[] end,
+      final int[] index, final String desc, final boolean visible) {
+    fsmCursor.transition("visitLocalVariableAnnotation");
+    final AnnotationVisitor annotationVisitor = super
+        .visitLocalVariableAnnotation(typeRef, typePath, start, end, index,
+            desc, visible);
+    return annotationVisitor; // TODO: add CheckAnnotationVisitorFsm
+  }
+
+  @Override
+  public void visitLookupSwitchInsn(final Label dflt, final int[] keys,
+      final Label[] labels) {
+    fsmCursor.transition("visitXInsn");
+    super.visitLookupSwitchInsn(dflt, keys, labels);
+  }
+
+  @Override
+  public void visitMaxs(final int maxStack, final int maxLocals) {
+    fsmCursor.transition("visitMaxs");
+    super.visitMaxs(maxStack, maxLocals);
+  }
+
+  @Deprecated
+  @Override
+  public void visitMethodInsn(final int opcode, final String owner,
+      final String name, final String desc) {
+    fsmCursor.transition("visitXInsn");
+    super.visitMethodInsn(opcode, owner, name, desc);
+  }
+
+  @Override
+  public void visitMethodInsn(final int opcode, final String owner,
+      final String name, final String desc, final boolean itf) {
+    fsmCursor.transition("visitXInsn");
+    super.visitMethodInsn(opcode, owner, name, desc, itf);
+  }
+
+  @Override
+  public void visitMultiANewArrayInsn(final String desc, final int dims) {
+    fsmCursor.transition("visitXInsn");
+    super.visitMultiANewArrayInsn(desc, dims);
+  }
+
+  @Override
+  public void visitParameter(final String name, final int access) {
+    fsmCursor.transition("visitParameter");
+    super.visitParameter(name, access);
+  }
+
+  @Override
+  public AnnotationVisitor visitParameterAnnotation(final int parameter,
+      final String desc, final boolean visible) {
+    fsmCursor.transition("visitParameterAnnotation");
+    final AnnotationVisitor annotationVisitor = super.visitParameterAnnotation(
+        parameter, desc, visible);
+    return annotationVisitor; // TODO: add CheckAnnotationVisitorFsm
+  }
+
+  @Override
+  public void visitTableSwitchInsn(final int min, final int max,
+      final Label dflt, final Label... labels) {
+    fsmCursor.transition("visitXInsn");
+    super.visitTableSwitchInsn(min, max, dflt, labels);
+  }
+
+  @Override
+  public AnnotationVisitor visitTryCatchAnnotation(final int typeRef,
+      final TypePath typePath, final String desc, final boolean visible) {
+    fsmCursor.transition("visitTryCatchAnnotation");
+    final AnnotationVisitor annotationVisitor = super.visitTryCatchAnnotation(
+        typeRef, typePath, desc, visible);
+    return annotationVisitor; // TODO: add CheckAnnotationVisitorFsm
+  }
+
+  @Override
+  public void visitTryCatchBlock(final Label start, final Label end,
+      final Label handler, final String type) {
+    fsmCursor.transition("visitTryCatchBlock");
+    super.visitTryCatchBlock(start, end, handler, type);
+  }
+
+  @Override
+  public AnnotationVisitor visitTypeAnnotation(final int typeRef,
+      final TypePath typePath, final String desc, final boolean visible) {
+    fsmCursor.transition("visitTypeAnnotation");
+    final AnnotationVisitor annotationVisitor = super.visitTypeAnnotation(
+        typeRef, typePath, desc, visible);
+    return annotationVisitor; // TODO: add CheckAnnotationVisitorFsm
+  }
+
+  @Override
+  public void visitTypeInsn(final int opcode, final String type) {
+    fsmCursor.transition("visitXInsn");
+    super.visitTypeInsn(opcode, type);
+  }
+
+  @Override
+  public void visitVarInsn(final int opcode, final int var) {
+    fsmCursor.transition("visitXInsn");
+    super.visitVarInsn(opcode, var);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/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 52d9e34..5c65724 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
@@ -26,36 +26,72 @@ 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.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;
 import org.objectweb.asm.tree.ClassNode;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 public class ClassTransformer {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClassTransformer.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClassTransformer.class);
 
   private final ByteCodeLoader byteCodeLoader = new ByteCodeLoader();
+  private final OptionManager optionManager;
 
-  public ClassTransformer() {
+  public final static String SCALAR_REPLACEMENT_OPTION =
+      "org.apache.drill.exec.compile.ClassTransformer.scalar_replacement";
+  public final static EnumeratedStringValidator SCALAR_REPLACEMENT_VALIDATOR = new EnumeratedStringValidator(
+      SCALAR_REPLACEMENT_OPTION, "try", "off", "try", "on");
+
+  @VisibleForTesting // although we need it even if it weren't used in testing
+  public enum ScalarReplacementOption {
+    OFF, // scalar replacement will not ever be used
+    TRY, // scalar replacement will be attempted, and if there is an error, we fall back to not using it
+    ON; // scalar replacement will always be used, and any errors cause user visible errors
+
+    /**
+     * Convert a string to an enum value.
+     *
+     * @param s the string
+     * @return an enum value
+     * @throws IllegalArgumentException if the string doesn't match any of the enum values
+     */
+    public static ScalarReplacementOption fromString(final String s) {
+      switch(s) {
+      case "off":
+        return OFF;
+      case "try":
+        return TRY;
+      case "on":
+        return ON;
+      }
+
+      throw new IllegalArgumentException("Invalid ScalarReplacementOption \"" + s + "\"");
+    }
   }
 
-  public static class ClassSet{
+  public ClassTransformer(final OptionManager optionManager) {
+    this.optionManager = optionManager;
+  }
+
+  public static class ClassSet {
     public final ClassSet parent;
     public final ClassNames precompiled;
     public final ClassNames generated;
 
     public ClassSet(ClassSet parent, String precompiled, String generated) {
-      super();
+      Preconditions.checkArgument(!generated.startsWith(precompiled),
+          String.format("The new name of a class cannot start with the old name of a class, otherwise class renaming will cause problems. Precompiled class name %s. Generated class name %s",
+              precompiled, generated));
       this.parent = parent;
-
       this.precompiled = new ClassNames(precompiled);
       this.generated = new ClassNames(generated);
-      Preconditions.checkArgument(!generated.startsWith(precompiled),
-          String.format("The new name of a class cannot start with the old name of a class, otherwise class renaming will cause problems.  Precompiled class name %s.  Generated class name %s", precompiled, generated));
     }
 
     public ClassSet getChild(String precompiled, String generated) {
@@ -111,11 +147,9 @@ public class ClassTransformer {
       }
       return true;
     }
-
   }
 
   public static class ClassNames {
-
     public final String dot;
     public final String slash;
     public final String clazz;
@@ -173,21 +207,18 @@ public class ClassTransformer {
     }
   }
 
-  private static ClassNode getClassNodeFromByteCode(byte[] bytes) {
-    ClassReader iReader = new ClassReader(bytes);
-    ClassNode impl = new ClassNode();
-    iReader.accept(impl, ClassReader.EXPAND_FRAMES);
-    return impl;
-  }
-
-  public Class<?> getImplementationClass( //
-      QueryClassLoader classLoader, //
-      TemplateClassDefinition<?> templateDefinition, //
-      String entireClass, //
-      String materializedClassName) throws ClassTransformationException {
+  public Class<?> getImplementationClass(
+      final QueryClassLoader classLoader,
+      final TemplateClassDefinition<?> templateDefinition,
+      final String entireClass,
+      final String materializedClassName) throws ClassTransformationException {
+    // unfortunately, this hasn't been set up at construction time, so we have to do it here
+    final OptionValue optionValue = optionManager.getOption(SCALAR_REPLACEMENT_OPTION);
+    final ScalarReplacementOption scalarReplacementOption =
+        ScalarReplacementOption.fromString((String) optionValue.getValue()); // TODO(DRILL-2474)
 
     try {
-      long t1 = System.nanoTime();
+      final long t1 = System.nanoTime();
       final ClassSet set = new ClassSet(null, templateDefinition.getTemplateClassName(), materializedClassName);
       final byte[][] implementationClasses = classLoader.getClassByteCode(set.generated, entireClass);
 
@@ -195,12 +226,15 @@ public class ClassTransformer {
       Map<String, ClassNode> classesToMerge = Maps.newHashMap();
       for (byte[] clazz : implementationClasses) {
         totalBytecodeSize += clazz.length;
-        ClassNode node = getClassNodeFromByteCode(clazz);
+        final ClassNode node = AsmUtil.classFromBytes(clazz);
+        if (!AsmUtil.isClassOk(logger, node, "implementationClasses")) {
+          throw new IllegalStateException("Problem found with implementationClasses");
+        }
         classesToMerge.put(node.name, node);
       }
 
-      LinkedList<ClassSet> names = Lists.newLinkedList();
-      Set<ClassSet> namesCompleted = Sets.newHashSet();
+      final LinkedList<ClassSet> names = Lists.newLinkedList();
+      final Set<ClassSet> namesCompleted = Sets.newHashSet();
       names.add(set);
 
       while ( !names.isEmpty() ) {
@@ -210,9 +244,43 @@ public class ClassTransformer {
         }
         final ClassNames nextPrecompiled = nextSet.precompiled;
         final byte[] precompiledBytes = byteCodeLoader.getClassByteCodeFromPath(nextPrecompiled.clazz);
-        ClassNames nextGenerated = nextSet.generated;
-        ClassNode generatedNode = classesToMerge.get(nextGenerated.slash);
-        MergedClassResult result = MergeAdapter.getMergedClass(nextSet, precompiledBytes, generatedNode);
+        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.
+         *
+         *  Here, we use the specified replacement option. The loop will allow us to retry if
+         *  we're using TRY.
+         */
+        MergedClassResult result = null;
+        boolean scalarReplace = scalarReplacementOption != ScalarReplacementOption.OFF;
+        while(true) {
+          try {
+            result = MergeAdapter.getMergedClass(nextSet, precompiledBytes, generatedNode, scalarReplace);
+            break;
+          } catch(RuntimeException e) {
+            // if we had a problem without using scalar replacement, then rethrow
+            if (!scalarReplace) {
+              throw e;
+            }
+
+            // if we did try to use scalar replacement, decide if we need to retry or not
+            if (scalarReplacementOption == ScalarReplacementOption.ON) {
+              // option is forced on, so this is a hard error
+              throw e;
+            }
+
+            /*
+             * We tried to use scalar replacement, with the option to fall back to not using it.
+             * Log this failure before trying again without scalar replacement.
+             */
+            logger.info("scalar replacement failure (retrying)\n", e);
+            scalarReplace = false;
+          }
+        }
 
         for (String s : result.innerClasses) {
           s = s.replace(FileUtils.separatorChar, '.');

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/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 7cc350e..57a6660 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
@@ -28,63 +28,74 @@ import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.store.sys.local.LocalPStoreProvider;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 
 public class CodeCompiler {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CodeCompiler.class);
+//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CodeCompiler.class);
 
   private final ClassTransformer transformer;
-  private final LoadingCache<CodeGenerator<?>, GeneratedClassEntry>  cache;
+  private final LoadingCache<CodeGenerator<?>, GeneratedClassEntry> cache;
   private final DrillConfig config;
-  private final OptionManager systemOptionManager;
+  private final OptionManager optionManager;
 
-  public CodeCompiler(DrillConfig config, OptionManager systemOptionManager){
-    this.transformer = new ClassTransformer();
-    int cacheMaxSize = config.getInt(ExecConstants.MAX_LOADING_CACHE_SIZE_CONFIG);
-    this.cache = CacheBuilder //
-        .newBuilder() //
-        .maximumSize(cacheMaxSize) //
+  public CodeCompiler(final DrillConfig config, final OptionManager optionManager) {
+    transformer = new ClassTransformer(optionManager);
+    final int cacheMaxSize = config.getInt(ExecConstants.MAX_LOADING_CACHE_SIZE_CONFIG);
+    cache = CacheBuilder.newBuilder()
+        .maximumSize(cacheMaxSize)
         .build(new Loader());
-    this.systemOptionManager = systemOptionManager;
+    this.optionManager = optionManager;
     this.config = config;
   }
 
   @SuppressWarnings("unchecked")
-  public <T> T getImplementationClass(CodeGenerator<?> cg) throws ClassTransformationException, IOException {
+  public <T> T getImplementationClass(final CodeGenerator<?> cg) throws ClassTransformationException, IOException {
     cg.generate();
     try {
-      GeneratedClassEntry ce = cache.get(cg);
+      final GeneratedClassEntry ce = cache.get(cg);
       return (T) ce.clazz.newInstance();
     } catch (ExecutionException | InstantiationException | IllegalAccessException e) {
       throw new ClassTransformationException(e);
     }
   }
 
-  private class Loader extends CacheLoader<CodeGenerator<?>, GeneratedClassEntry>{
+  private class Loader extends CacheLoader<CodeGenerator<?>, GeneratedClassEntry> {
     @Override
-    public GeneratedClassEntry load(CodeGenerator<?> cg) throws Exception {
-      QueryClassLoader loader = new QueryClassLoader(config, systemOptionManager);
-      Class<?> c = transformer.getImplementationClass(loader, cg.getDefinition(), cg.getGeneratedCode(), cg.getMaterializedClassName());
-      return new GeneratedClassEntry(loader, c);
+    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());
+      return new GeneratedClassEntry(c);
     }
   }
 
   private class GeneratedClassEntry {
-
-    private final QueryClassLoader classLoader;
     private final Class<?> clazz;
 
-    public GeneratedClassEntry(QueryClassLoader classLoader, Class<?> c) {
-      super();
-      this.classLoader = classLoader;
-      this.clazz = c;
+    public GeneratedClassEntry(final Class<?> clazz) {
+      this.clazz = clazz;
     }
-
   }
 
   public static CodeCompiler getTestCompiler(DrillConfig c) throws IOException{
     return new CodeCompiler(c, new SystemOptionManager(c, new LocalPStoreProvider(c)).init());
   }
+
+  /**
+   * Flush the compiled classes from the cache.
+   *
+   * <p>The cache has DrillbitContext lifetime, so the only way items go out of it
+   * now is by being aged out because of the maximum cache size.
+   *
+   * <p>The intent of flushCache() is to make it possible to flush the cache for
+   * testing purposes, although this could be used by users in case issues arise. If
+   * that happens, remove the visible for testing annotation.
+   */
+  @VisibleForTesting
+  public void flushCache() {
+    cache.invalidateAll();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CompilationConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CompilationConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CompilationConfig.java
new file mode 100644
index 0000000..c724549
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CompilationConfig.java
@@ -0,0 +1,28 @@
+/**
+ * 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 org.objectweb.asm.Opcodes;
+
+public class CompilationConfig {
+  /*
+   * Never use asm.Opcodes values directly in calls that require them. Use ASM_OPCODES
+   * instead, so that we can change it here once for all references.
+   */
+  public final static int ASM_API_VERSION = Opcodes.ASM5;
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillCheckClassAdapter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillCheckClassAdapter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillCheckClassAdapter.java
new file mode 100644
index 0000000..1043bfc
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillCheckClassAdapter.java
@@ -0,0 +1,118 @@
+/**
+ * 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.PrintWriter;
+
+import org.objectweb.asm.ClassReader;
+import org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.ClassWriter;
+import org.objectweb.asm.util.CheckClassAdapter;
+
+/**
+ * There's a bug in ASM's CheckClassAdapter. It chokes on classes that
+ * have the access bits for ACC_PROTECTED, ACC_PRIVATE, or ACC_FINAL set;
+ * it appears to be assuming that it is only called on top-level classes.
+ * However, when we compile classes, nested classes come out as byte arrays
+ * that are otherwise indistinguishable from top-level classes', except for
+ * the possible appearance of those access bits.
+ *
+ * <p>In order to get around this, we use the DrillCheckClassAdapter
+ * instead. This strips off the offending bits before passing the class
+ * to CheckClassAdapter, and then restores them before passing the class
+ * on to whatever delegate the DrillCheckClassAdapter is created with.
+ */
+public class DrillCheckClassAdapter extends RetargetableClassVisitor {
+  /*
+   * CheckClassAdapter has final methods, and it's constructor checks that
+   * getClass() == CheckClassAdapter.class, so you can't derive from it even to
+   * reset the (protected) delegate. The delegates can't be initialized and set up
+   * before the call to super(), so we're left with this implementation strategy.
+   */
+  private final InnerClassAccessStripper accessStripper; // removes the access bits
+
+  /*
+   * This inner class is used to restore the access bits that the
+   * accessStripper has removed.
+   */
+  private class AccessRestorer extends ClassVisitor {
+    public AccessRestorer(final int api, final ClassVisitor cv) {
+      super(api, cv);
+    }
+
+    @Override
+    public void visit(final int version, final int access, final String name,
+        final String signature, final String superName,
+        final String[] interfaces) {
+      super.visit(version, accessStripper.getOriginalAccess(), name, signature,
+          superName, interfaces);
+    }
+  }
+
+  /**
+   * See {@link org.objectweb.asm.util.CheckClassAdapter#CheckClassAdapter(ClassVisitor)}.
+   */
+  public DrillCheckClassAdapter(final ClassVisitor cv) {
+    this(CompilationConfig.ASM_API_VERSION, cv, false);
+  }
+
+  /**
+   * See {@link org.objectweb.asm.util.CheckClassAdapter#CheckClassAdapter(ClassVisitor)}.
+   * @param api the api version
+   */
+  public DrillCheckClassAdapter(final int api, final ClassVisitor cv) {
+    this(api, cv, false);
+  }
+
+  /**
+   * See {@link org.objectweb.asm.util.CheckClassAdapter#CheckClassAdapter(ClassVisitor, boolean)}.
+   * @param api the api version
+   */
+  protected DrillCheckClassAdapter(final int api, final ClassVisitor cv,
+      final boolean checkDataFlow) {
+    super(api);
+
+    /*
+     * We set up a chain of class visitors:
+     * this -> InnerAccessStripper -> CheckClassAdapter -> AccessRestorer -> cv
+     * Note the AccessRestorer references accessStripper to get the original
+     * access bits; the inner class could not be constructed before the call to
+     * super(), hence the need to set the delegate after that.
+     */
+    accessStripper = new InnerClassAccessStripper(api, new CheckClassAdapter(
+        new AccessRestorer(api, cv), checkDataFlow));
+    setDelegate(accessStripper);
+  }
+
+  /**
+   * See {@link org.objectweb.asm.util.CheckClassAdapter#verify(ClassReader, boolean, PrintWriter)}.
+   */
+  public static void verify(final ClassReader cr, final boolean dump,
+      final PrintWriter pw) {
+    /*
+     * For plain verification, we don't need to restore the original access
+     * bytes the way we do when the check adapter is used as part of a chain, so
+     * we can just strip it and use the ASM version directly.
+     */
+    final ClassWriter classWriter = new ClassWriter(0);
+    cr.accept(new InnerClassAccessStripper(CompilationConfig.ASM_API_VERSION,
+        classWriter), ClassReader.SKIP_DEBUG);
+    final ClassReader strippedCr = new ClassReader(classWriter.toByteArray());
+    CheckClassAdapter.verify(strippedCr, dump, pw);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillInitMethodVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillInitMethodVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillInitMethodVisitor.java
index 859a14c..a077853 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillInitMethodVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillInitMethodVisitor.java
@@ -27,18 +27,19 @@ public class DrillInitMethodVisitor extends MethodVisitor {
   final String className;
 
   public DrillInitMethodVisitor(String className, MethodVisitor mv){
-    super(Opcodes.ASM4, mv);
+    super(CompilationConfig.ASM_API_VERSION, mv);
     this.className = className;
   }
 
   @Override
   public void visitInsn(int opcode) {
-    if(opcode == Opcodes.RETURN){
-      super.visitVarInsn(Opcodes.ALOAD, 0); // load this.
-      super.visitMethodInsn(Opcodes.INVOKEVIRTUAL, className, SignatureHolder.DRILL_INIT_METHOD, "()V"); // execute drill init.
+    if(opcode == Opcodes.RETURN) {
+      // Load this.
+      super.visitVarInsn(Opcodes.ALOAD, 0);
+
+      // Execute drill init.
+      super.visitMethodInsn(Opcodes.INVOKEVIRTUAL, className, SignatureHolder.DRILL_INIT_METHOD, "()V", false);
     }
     super.visitInsn(opcode);
   }
-
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/exec/java-exec/src/main/java/org/apache/drill/exec/compile/FsmCursor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/FsmCursor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/FsmCursor.java
new file mode 100644
index 0000000..7b833af
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/FsmCursor.java
@@ -0,0 +1,63 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+
+/**
+ * Maintains state while traversing a finite state machine described by
+ * an FsmDescriptor. To begin, use {@link FsmDescriptor#createCursor() createCursor}.
+ */
+public class FsmCursor {
+  private final FsmDescriptor fsmDescriptor;
+  private final StringBuilder stringBuilder;
+
+  /**
+   * Constructor.
+   *
+   * @param fsmDescriptor the descriptor for the FSM this cursor will track
+   */
+  FsmCursor(final FsmDescriptor fsmDescriptor) {
+    this.fsmDescriptor = fsmDescriptor;
+    stringBuilder = new StringBuilder();
+  }
+
+  /**
+   * Record a transition in the state machine.
+   *
+   * <p>Validates the transition using {@link FsmDescriptor#validateTransitions(CharSequence)}.
+   *
+   * @param token the name of the transition
+   */
+  public void transition(final String token) {
+    Preconditions.checkNotNull(token);
+    final char c = fsmDescriptor.getChar(token);
+    stringBuilder.append(c);
+
+    if (fsmDescriptor.isLastTransition(c)) {
+      fsmDescriptor.validateTransitions(stringBuilder);
+    }
+  }
+
+  /**
+   * Resets the state machine, setting it back to the start state.
+   */
+  public void reset() {
+    stringBuilder.setLength(0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/exec/java-exec/src/main/java/org/apache/drill/exec/compile/FsmDescriptor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/FsmDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/FsmDescriptor.java
new file mode 100644
index 0000000..a6b2da1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/FsmDescriptor.java
@@ -0,0 +1,138 @@
+/**
+ * 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.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Describes a finite state machine in terms of a mapping of tokens to
+ * characters, a regular expression describing the valid transitions
+ * using the characters, and an end state. Used to validate state
+ * transitions. One simple example is the implementation of the
+ * Check*VisitorFsm classes used to validate call sequences in ASM
+ * visitors (classes like CheckClassAdapter only validate per-call
+ * arguments, not the entire sequence of calls, according to
+ * http://asm.ow2.org/asm50/javadoc/user/org/objectweb/asm/util/CheckClassAdapter.html
+ *
+ * <p>The current implementation is very simple, and requires some user setup.
+ * Basically, we use Java's {@link java.util.regex.Pattern} and
+ * {@link java.util.regex.Matcher} to verify a regular expression on demand.
+ * The user must map state transitions' string names onto characters, and
+ * specify a regex that describes the state machine. Using this technique, we
+ * can only validate when we transition to an end state; we just check to see
+ * if accumulated characters comprise an allowed regular expression.
+ *
+ * <p>In this simple implementation, the tokens and characters may represent
+ * states or transitions, depending on what is most convenient. In either case,
+ * we just check that a sequence of them matches the regular expression governing
+ * this state machine.
+ */
+public class FsmDescriptor {
+  private final Map<String, Character> tokenMap; // state/transition -> character
+  private final Pattern fsmPattern; // compiled regex representing the FSM
+  private final char lastTransition; // the character that represents the last state/transition
+
+  /**
+   * Create a finite state machine descriptor. The descriptor is immutable, and may
+   * be shared across many cursors that are executing the FSM.
+   *
+   * @param tokenMap mapping of transitions/states to characters
+   * @param fsmRegex the regular expression, defined using the characters from the tokenMap
+   * @param lastTransition the name of the final transition/state
+   */
+  public FsmDescriptor(final Map<String, Character> tokenMap,
+      final String fsmRegex, final String lastTransition) {
+    Preconditions.checkNotNull(tokenMap);
+    Preconditions.checkNotNull(fsmRegex);
+    Preconditions.checkNotNull(lastTransition);
+    Preconditions.checkArgument(tokenMap.containsKey(lastTransition));
+
+    // make sure the characters in the tokenMap are unique
+    final HashSet<Character> charSet = new HashSet<>();
+    for(Map.Entry<String, Character> me : tokenMap.entrySet()) {
+      final Character character = me.getValue();
+      if (charSet.contains(character)) {
+        throw new IllegalArgumentException("Duplicate tokenMap char: '" + character + "'");
+      }
+      charSet.add(character);
+    }
+
+    this.tokenMap = Collections.unmodifiableMap(tokenMap);
+    this.fsmPattern = Pattern.compile(fsmRegex);
+    this.lastTransition = this.tokenMap.get(lastTransition).charValue();
+  }
+
+  /**
+   * Create a cursor for performing and validating transitions according to this
+   * state machine.
+   *
+   * @return the new cursor
+   */
+  public FsmCursor createCursor() {
+    return new FsmCursor(this);
+  }
+
+  /**
+   * Validate the given transitions against this state machine.
+   *
+   * @param transitions a character sequence indicating a set of transitions or
+   *   states as defined by the tokenMap used at construction time.
+   * @throws IllegalStateException if the set of transitions is not allowed
+   */
+  void validateTransitions(final CharSequence transitions) {
+    final long length = transitions.length();
+    if (length == 0) {
+      return; // assume we haven't started yet
+    }
+
+    final Matcher matcher = fsmPattern.matcher(transitions);
+    if (!matcher.find() || (matcher.start() != 0) || (matcher.end() != length)) {
+      throw new IllegalStateException("Illegal state transition(s): " + transitions);
+    }
+  }
+
+  /**
+   * Look up the character used to represent the transition or state denoted by
+   * the given token.
+   *
+   * @param token the transition or state to look up
+   * @return the character used to represent that transition or state
+   */
+  char getChar(final String token) {
+    Preconditions.checkNotNull(token);
+    final Character character = tokenMap.get(token);
+    Preconditions.checkNotNull(character);
+    return character.charValue();
+  }
+
+  /**
+   * Determine whether the indicated character represents the final transition or state.
+   *
+   * @param transitionChar the character to look up
+   * @return true if the character represents the final transition or state
+   */
+  boolean isLastTransition(final char transitionChar) {
+    return transitionChar == lastTransition;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/exec/java-exec/src/main/java/org/apache/drill/exec/compile/InnerClassAccessStripper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/InnerClassAccessStripper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/InnerClassAccessStripper.java
new file mode 100644
index 0000000..ed83384
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/InnerClassAccessStripper.java
@@ -0,0 +1,89 @@
+/**
+ * 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 org.objectweb.asm.ClassVisitor;
+import org.objectweb.asm.Opcodes;
+
+/**
+ * An ASM ClassVisitor that strips class the access bits that are only possible
+ * on inner classes (ACC_PROTECTED, ACC_PRIVATE, and ACC_FINAL). These bits are
+ * only stripped from the class' visit() call if the class' name contains a
+ * '$'.
+ *
+ * <p>This visitor is meant to be used on classes that will undergo validation
+ * with CheckClassAdapter. CheckClassAdapter assumes it will only be called on
+ * non-inner classes, and throws an IllegalArgumentException if the class is
+ * protected, private, or final. However, once classes are compiled, they appear
+ * in their class files alone, and these options may be present, with no way
+ * for an outside observer to tell if they were originally inner classes.
+ */
+public class InnerClassAccessStripper extends ClassVisitor {
+  private int originalClassAccess;
+  private boolean accessCaptured;
+
+  /**
+   * See {@link org.objectweb.asm.ClassVisitor#ClassVisitor(int)}.
+   */
+  public InnerClassAccessStripper(final int api) {
+    super(api);
+  }
+
+  /**
+   * See {@link org.objectweb.asm.ClassVisitor#ClassVisitor(int, ClassVisitor)}.
+   */
+  public InnerClassAccessStripper(final int api, final ClassVisitor cv) {
+    super(api, cv);
+  }
+
+  /**
+   * Return the original class' access bits.
+   *
+   * <p>This may only be called after {@link ClassVisitor#visit(int, int, String, String, String, String[])}
+   * has been called; that's where the bits are stripped and captured.
+   *
+   * @return the original class bits
+   * @throws IllegalStateException if visit() hasn't been called yet
+   */
+  public int getOriginalAccess() {
+    if (!accessCaptured) {
+      throw new IllegalStateException(
+          "can't get original access before it is captured");
+    }
+
+    return originalClassAccess;
+  }
+
+  @Override
+  public void visit(final int version, final int access, final String name,
+      final String signature, final String superName, final String[] interfaces) {
+    /*
+     * Record the original access bits so we can restore them before the next
+     * link in the visitor chain.
+     */
+    originalClassAccess = access;
+    accessCaptured = true;
+
+    // If we're checking an inner class, suppress access bits that ASM chokes on.
+    int checkClassAccess = access;
+    if (name.indexOf('$') >= 0) {
+      checkClassAccess &= ~(Opcodes.ACC_PROTECTED | Opcodes.ACC_PRIVATE | Opcodes.ACC_STATIC);
+    }
+    super.visit(version, checkClassAccess, name, signature, superName, interfaces);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/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 84bf4b0..1bb4465 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
@@ -59,5 +59,4 @@ public class JaninoClassCompiler extends AbstractClassCompiler {
 
   @Override
   protected org.slf4j.Logger getLogger() { return logger; }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7ea212a2/exec/java-exec/src/main/java/org/apache/drill/exec/compile/LogWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/LogWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/LogWriter.java
new file mode 100644
index 0000000..7ea7471
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/LogWriter.java
@@ -0,0 +1,133 @@
+/**
+ * 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.io.Writer;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.log4j.Level;
+import org.slf4j.Logger;
+
+/**
+ * A simple Writer that will forward whole lines (lines ending with a newline) to
+ * a Logger. The newlines themselves are not forwarded.
+ */
+public class LogWriter extends Writer {
+  private final Logger logger; // the underlying logger to write to
+  private final int level; // the debug level to write at
+  private final StringBuilder stringBuilder; // a buffer for incomplete lines
+  private boolean isClosed; // close() has been called
+
+  /**
+   * Constructor.
+   *
+   * @param logger the logger this Writer should write to
+   * @param level the debug level to write to the logger with
+   */
+  public LogWriter(final Logger logger, final Level level) {
+    Preconditions.checkNotNull(logger);
+    Preconditions.checkArgument((level == Level.DEBUG) || (level == Level.ERROR) ||
+        (level == Level.INFO) || (level == Level.TRACE) || (level == Level.WARN),
+        "level must be a logging level");
+
+    this.logger = logger;
+    this.level = level.toInt();
+    stringBuilder = new StringBuilder(80);
+    isClosed = false;
+  }
+
+  @Override
+  public void write(char[] cbuf, int off, int len) throws IOException {
+    checkNotClosed();
+
+    stringBuilder.append(cbuf, off, len);
+
+    // log all of the whole lines we have
+    do {
+      final int newlinePos = stringBuilder.indexOf("\n");
+      if (newlinePos < 0) {
+        break;
+      }
+
+      final String oneLine = stringBuilder.substring(0, newlinePos); // leaves out the newline
+      writeToLog(oneLine);
+      stringBuilder.delete(0, newlinePos + 1); // removes the newline as well
+
+    } while (stringBuilder.length() > 0);
+  }
+
+  @Override
+  public void flush() throws IOException {
+    checkNotClosed();
+    flushToLog();
+  }
+
+  @Override
+  public void close() throws IOException {
+    checkNotClosed();
+    isClosed = true;
+    flushToLog();
+  }
+
+  /*
+   * Assumes there are no newlines.
+   */
+  private void flushToLog() {
+    writeToLog(stringBuilder.toString());
+    stringBuilder.setLength(0);
+  }
+
+  private void writeToLog(final String s) {
+    if ((s == null) || s.isEmpty()) {
+      return;
+    }
+
+    switch(level) {
+    case Level.DEBUG_INT:
+      logger.debug(s);
+      break;
+
+    case Level.ERROR_INT:
+      logger.error(s);
+      break;
+
+    case Level.INFO_INT:
+      logger.info(s);
+      break;
+
+    case Level.TRACE_INT:
+      logger.trace(s);
+      break;
+
+    case Level.WARN_INT:
+      logger.warn(s);
+      break;
+
+    default:
+      throw new IllegalStateException();
+    }
+  }
+
+  private void checkNotClosed() throws IOException {
+    if (isClosed) {
+      throw new IOException("LogWriter is already closed()");
+    }
+  }
+}