You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2015/05/12 07:09:23 UTC

drill git commit: DRILL-2343: Add tracing proxy JDBC driver for tracing JDBC method calls.

Repository: drill
Updated Branches:
  refs/heads/master 27b4aae20 -> 7c5a1f57c


DRILL-2343: Add tracing proxy JDBC driver for tracing JDBC method calls.

Overview of org.apache.drill.jdbc.proxy classes/interfaces:

Entry point:
- TracingProxyDriver - Tracing proxy JDBC driver class.  Class description
  Javadoc has usage instructions.

Other core types:
- ProxiesManager - creates and tracks java.lang.reflect.Proxy-based proxy
  objects
- TracingInvocationHandler - java.lang.reflect.Proxy invocation handler; maps
  reflective/proxy invocations to InvocationReporter method call/return/throw
  event calls
- InvocationReporter - defines method call/return/throw event calls
- InvocationReporterImpl - implements rendering of method call/return/throw
  event calls (including rendering of parameter/return/exception values)
- ProxySetupSQLException

Unit tests:
- TracingProxyDriverClassLoadingTest - test of loading proxied driver class
- TracingProxyDriverTest - basic test of proxying (pass-through) and tracing
  output

Other:
- exec/jdbc-all/pom.xml - has change to keep TracingProxyDriver and depended-on
  classes in JDBC-all Jar file.


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

Branch: refs/heads/master
Commit: 7c5a1f57cc51e74c9e3ab965350bd04062b41049
Parents: 27b4aae
Author: dbarclay <db...@maprtech.com>
Authored: Mon Feb 9 10:52:11 2015 -0800
Committer: Parth Chandra <pa...@apache.org>
Committed: Mon May 11 22:04:10 2015 -0700

----------------------------------------------------------------------
 exec/jdbc-all/pom.xml                           |   2 +
 .../drill/jdbc/proxy/InvocationReporter.java    |  80 +++
 .../jdbc/proxy/InvocationReporterImpl.java      | 519 +++++++++++++++++++
 .../apache/drill/jdbc/proxy/ProxiesManager.java | 117 +++++
 .../jdbc/proxy/ProxySetupSQLException.java      |  37 ++
 .../jdbc/proxy/TracingInvocationHandler.java    | 112 ++++
 .../drill/jdbc/proxy/TracingProxyDriver.java    | 288 ++++++++++
 .../apache/drill/jdbc/proxy/package-info.java   |  23 +
 .../TracingProxyDriverClassLoadingTest.java     | 135 +++++
 .../jdbc/proxy/TracingProxyDriverTest.java      | 257 +++++++++
 10 files changed, 1570 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/7c5a1f57/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index 388f8ba..307006f 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -224,10 +224,12 @@
             <option>-keep class org.apache.drill.jdbc.Driver { *; }</option>
             <option>-keep class org.apache.drill.jdbc.DrillJdbc40Factory { *; }</option>
             <option>-keep class org.apache.drill.jdbc.DrillJdbc41Factory { *; }</option>
+            <option>-keep class org.apache.drill.jdbc.proxy.TracingProxyDriver { *; }</option>
             <option>-keep class org.apache.drill.common.config.CommonConstants { *; }</option>
             <option>-keep class org.apache.drill.common.config.ConfigProvider { *; }</option>
             <option>-keep class org.apache.drill.common.config.DrillConfig { *; }</option>
             <option>-keep class org.apache.drill.common.config.NestedConfig { *; }</option>
+
             <option>-keep class ch.qos.logback.** { *; }</option>
             <option>-keep class org.slf4j.** { *; }</option>
             <option>-keep class * implements com.fasterxml.jackson.databind.cfg.ConfigFeature { *; }</option>

http://git-wip-us.apache.org/repos/asf/drill/blob/7c5a1f57/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/InvocationReporter.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/InvocationReporter.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/InvocationReporter.java
new file mode 100644
index 0000000..1aa8ff2
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/InvocationReporter.java
@@ -0,0 +1,80 @@
+/*
+ * 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.jdbc.proxy;
+
+import java.lang.reflect.Method;
+
+
+/**
+ * Reporter of method-invocation events.
+ */
+interface InvocationReporter
+{
+
+  /**
+   * Reports a proxy setup message.
+   */
+  void setupMessage( String message );
+
+  /**
+   * Reports an invocation of a proxied method.
+   *
+   * @param  targetObject
+   *         the target of the method invocation (the object on which the method
+   *         was called)
+   * @param  targetType
+   *         the declared target type of the method (the interface whose method
+   *         was called; not the actual type of {@code targetObject})
+   * @param  method
+   *         the method that was called
+   * @param  arguments
+   *         the arguments that were passed
+   *         (represented as for {@link Method#invoke})
+   */
+  void methodCalled( Object targetObject, Class<?> targetType, Method method,
+                     Object[] arguments );
+
+  /**
+   * Reports the return from an invocation of a proxied method.
+   *
+   * @param  targetObject  same as for {@link #methodCalled} for the call
+   * @param  targetType    same as for {@link #methodCalled} for the call
+   * @param  method        same as for {@link #methodCalled} for the call
+   * @param  arguments     same as for {@link #methodCalled} for the call
+   *
+   * @param result         the value returned from the method
+   *                       (represented as from {@link Method#invoke})
+   *
+   */
+  void methodReturned( Object targetObject, Class<?> targetType, Method method,
+                       Object[] arguments, Object result );
+
+  /**
+   * Reports the throwing of an exception from an invocation of a proxied method.
+   *
+   * @param  targetObject  same as for {@link #methodCalled} for the call
+   * @param  targetType    same as for {@link #methodCalled} for the call
+   * @param  method        same as for {@link #methodCalled} for the call
+   * @param  arguments     same as for {@link #methodCalled} for the call
+   *
+   * @param  exception     the exception thrown by the method.
+   */
+  void methodThrew( Object targetObject, Class<?> targetType, Method method,
+                    Object[] arguments, Throwable exception  );
+
+} // interface InvocationReporter

http://git-wip-us.apache.org/repos/asf/drill/blob/7c5a1f57/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/InvocationReporterImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/InvocationReporterImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/InvocationReporterImpl.java
new file mode 100644
index 0000000..031a147
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/InvocationReporterImpl.java
@@ -0,0 +1,519 @@
+/*
+ * 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.jdbc.proxy;
+
+import java.lang.reflect.Method;
+import java.sql.DriverPropertyInfo;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+
+/**
+ * Implementation of InvocationReporter.
+ * <p>
+ *   Currently, just writes to System.err.
+ * </p>
+ */
+class InvocationReporterImpl implements InvocationReporter
+{
+  private static final String LINE_PREFIX = "TRACER: ";
+  private static final String SETUP_LINE_PREFIX   = LINE_PREFIX + "SETUP: ";
+  private static final String WARNING_LINE_PREFIX = LINE_PREFIX + "WARNING: ";
+  private static final String CALL_LINE_PREFIX    = LINE_PREFIX + "CALL:   ";
+  private static final String RETURN_LINE_PREFIX  = LINE_PREFIX + "RETURN: ";
+  private static final String THROW_LINE_PREFIX   = LINE_PREFIX + "THROW:  ";
+
+  private static final Set<Package> JDBC_PACKAGES;
+  static {
+    // Load some class in each JDBC package below so package exists for
+    // getPackage():
+    // Suppressed because we intentionally are only assigning to the variable.
+    @SuppressWarnings("unused")
+    Class<?> someReference;
+    someReference = java.sql.Connection.class;
+    someReference = javax.sql.PooledConnection.class;
+    someReference = javax.sql.rowset.BaseRowSet.class;
+    someReference = javax.sql.rowset.serial.SerialJavaObject.class;
+    someReference = javax.sql.rowset.spi.SyncFactory.class;
+
+    Set<Package> set = new HashSet<>();
+    set.add( Package.getPackage( "java.sql" ) );
+    set.add( Package.getPackage( "javax.sql" ) );
+    set.add( Package.getPackage( "javax.sql.rowset" ) );
+    set.add( Package.getPackage( "javax.sql.rowset.serial" ) );
+    set.add( Package.getPackage( "javax.sql.rowset.spi" ) );
+    for ( Package p : set ) {
+      assert null != p
+          : "null Package; missing reference to class in that package?";
+    }
+    JDBC_PACKAGES = Collections.unmodifiableSet( set );
+  }
+
+  /** Common packages whose names to suppress in rendered type names. */
+  // (Is sorted for abbreviated-packages message to user.)
+  private static final SortedSet<Package> PACKAGES_TO_ABBREVIATE;
+  static {
+    SortedSet<Package> set = new TreeSet<Package>(
+        new Comparator<Package>() {
+          @Override
+          public int compare( Package o1, Package o2 ) {
+            return
+                null == o1 ? -1
+                    : null == o2 ? 1
+                        : o1.getName().compareTo( o2.getName() );
+          }
+        } );
+    set.addAll( JDBC_PACKAGES );
+    set.add( Package.getPackage( "java.util" ) );
+    set.add( Package.getPackage( "java.lang" ) );
+    PACKAGES_TO_ABBREVIATE = Collections.unmodifiableSortedSet( set );
+  }
+
+  private int lastObjNum = 0;
+  private Map<Object, String> objectsToIdsMap = new IdentityHashMap<>();
+
+
+  void reportAbbreviatedPackages() {
+    final List<String> names = new ArrayList<>();
+    for ( Package p : PACKAGES_TO_ABBREVIATE ) {
+      names.add( p.getName() );
+    }
+    setupMessage( "Abbreviating (unique) class names in packages "
+                  + StringUtils.join( names, ", " ) + "." );
+  }
+
+  ////////////////////
+  // Line-output output methods:
+
+  // TODO:  When needed, allow output to something other then System.err
+  // (e.g., appending to file, System.out).  Decide control--probably parameter
+  // in proxy URL and/or JVM system property (checked higher up and configuring
+  // this class).
+
+  /**
+   * Prints a line to the tracing log.
+   * <p>
+   *   Is it intended that all tracing output goes through this method.
+   * </p>
+   */
+  private void printTraceLine( final String line ) {
+    System.err.println( line );
+  }
+
+
+  /**
+   * For warnings such as warning about encountering a type that for which
+   * rendering isn't known to show values well.
+   */
+  private void printWarningLine( final String line ) {
+    printTraceLine( WARNING_LINE_PREFIX + line );
+  }
+
+
+  ////////////////////
+  // Type, value, exception, and arguments formatting methods:
+
+  private String getObjectId( final Object object )
+  {
+    String id;
+    if ( null == object ) {
+      id = "n/a";
+    }
+    else {
+      id = objectsToIdsMap.get( object );
+      if ( null == id ) {
+        ++lastObjNum;
+        id = Integer.toString( lastObjNum );
+        objectsToIdsMap.put( object, id );
+      }
+    }
+    return id;
+  }
+
+  /**
+   * Renders a type name.  Uses simple names for common types (JDBC interfaces
+   * and {code java.lang.*}).
+   */
+  private String formatType( final Class<?> type ) {
+    final String result;
+    if ( type.isArray() ) {
+      result = formatType( type.getComponentType() ) + "[]";
+    } else {
+      // Suppress package name for common (JDBC and java.lang) types, except
+      // when would be ambiguous (e.g., java.sql.Date vs. java.util.Date).
+      if ( PACKAGES_TO_ABBREVIATE.contains( type.getPackage() ) ) {
+        int sameSimpleNameCount = 0;
+        for ( Package p : PACKAGES_TO_ABBREVIATE ) {
+          try {
+            Class.forName( p.getName() + "." + type.getSimpleName() );
+            sameSimpleNameCount++;
+          }
+          catch ( ClassNotFoundException e ) {
+            // Nothing to do.
+          }
+        }
+        if ( 1 == sameSimpleNameCount ) {
+          result = type.getSimpleName();
+        }
+        else {
+          // Multiple classes with same simple name, so would be ambiguous to
+          // abbreviate, so use fully qualified name.
+          result = type.getName();
+        }
+      }
+      else {
+        result = type.getName();
+      }
+    }
+    return result;
+  }
+
+  private String formatString( final String value ) {
+    return
+        "\""
+        + ( ((String) value)
+             .replace( "\\", "\\\\" )  // first encode backslashes (esc. char.)
+             .replace( "\"", "\\\"" )  // then encode quotes (via backslash)
+             .replace( "\n", "\\n" )   // then encode newlines
+             // Anything else?
+            )
+        + "\"";
+  }
+
+  private String formatDriverPropertyInfo( final DriverPropertyInfo info ) {
+    return
+        "[ "
+        + "name = " + formatValue( info.name )
+        + ", value = " + formatValue( info.value )
+        + ", required = " + info.required
+        + ", choices = " + formatValue( info.choices )
+        + ", description = " + formatValue( info.description )
+        + " ]";
+  }
+
+  private String formatValue( final Object value ) {
+    final String result;
+    if ( null == value ) {
+      result = "null";
+    }
+    else {
+      final Class<?> rawActualType = value.getClass();
+      if ( String.class == rawActualType ) {
+        result = formatString( (String) value );
+      }
+      else if ( rawActualType.isArray()
+                && ! rawActualType.getComponentType().isPrimitive() ) {
+        // Array of non-primitive type
+
+        final StringBuilder buffer = new StringBuilder();
+        /* Decide whether to includes this:
+        buffer.append( formatType( elemType ) );
+        buffer.append( "[] " );
+        */
+        buffer.append( "{ " );
+        boolean first = true;
+        for ( Object elemVal : (Object[]) value ) {
+          if ( ! first ) {
+            buffer.append( ", " );
+          }
+          first = false;
+          buffer.append( formatValue( elemVal ) );
+        }
+        buffer.append( " }" );
+        result = buffer.toString();
+      }
+      else if ( DriverPropertyInfo.class == rawActualType ) {
+        result = formatDriverPropertyInfo( (DriverPropertyInfo) value );
+      }
+      else if (
+          // Is type seen and whose toString() renders value well.
+          false
+          || rawActualType == java.lang.Boolean.class
+          || rawActualType == java.lang.Byte.class
+          || rawActualType == java.lang.Double.class
+          || rawActualType == java.lang.Float.class
+          || rawActualType == java.lang.Integer.class
+          || rawActualType == java.lang.Long.class
+          || rawActualType == java.lang.Short.class
+          || rawActualType == java.math.BigDecimal.class
+          || rawActualType == java.lang.Class.class
+          || rawActualType == java.sql.Date.class
+          || rawActualType == java.sql.Timestamp.class
+          ) {
+        result = value.toString();
+      }
+      else if (
+          // Is type seen and whose toString() has rendered value well--in cases
+          // seen so far.
+          false
+          || rawActualType == java.util.Properties.class
+          || rawActualType.isEnum()
+          ) {
+        result = value.toString();
+      }
+      else if (
+          // Is type to warn about (one case).
+          false
+          || rawActualType == org.apache.drill.jdbc.DrillResultSet.class
+          ) {
+        printWarningLine(
+            "Class " + rawActualType.getName() + " should be an interface."
+            + " (While it's a class, it can't be proxied, and some methods can't"
+            + " be traced.)" );
+        result = value.toString();
+      }
+      else if (
+          // Is type to warn about (second case).
+          false
+          || rawActualType == org.apache.hadoop.io.Text.class
+          || rawActualType == org.joda.time.Period.class
+          || rawActualType ==
+             org.apache.drill.exec.vector.accessor.sql.TimePrintMillis.class
+          ) {
+        printWarningLine( "Should " + rawActualType
+                          + " be appearing at JDBC interface?" );
+        result = value.toString();
+      }
+      else {
+        // Is other type--unknown whether it already formats well.
+        // (No handled yet: byte[].)
+        printWarningLine( "Unnoted type encountered in formatting (value might"
+                          + " not render well): " + rawActualType + "." );
+        result = value.toString();
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Renders a value with its corresponding <em>declared</em> type.
+   *
+   * @param  declaredType
+   *         the corresponding declared method parameter or return type
+   * @value  value
+   *         the value to render
+   */
+  private String formatTypeAndValue( Class<?> declaredType, Object value ) {
+    final String declaredTypePart = "(" + formatType( declaredType ) + ") ";
+
+    final String actualTypePart;
+    final String actualValuePart;
+    if ( null == value ) {
+      // Null--show no actual type or object ID.
+      actualTypePart = "";
+      actualValuePart = formatValue( value );
+    }
+    else {
+      // Non-null value--show at least some representation of value.
+      Class<?> rawActualType = value.getClass();
+      Class<?> origActualType =
+          declaredType.isPrimitive() ? declaredType : rawActualType;
+      if ( String.class == rawActualType ) {
+        // String--show no actual type or object ID.
+        actualTypePart = "";
+        actualValuePart = formatValue( value );
+      }
+      else if ( origActualType.isPrimitive() ) {
+        // Primitive type--show no actual type or object ID.
+        actualTypePart = "";
+        // (Remember--primitive type is wrapped here.)
+        actualValuePart = value.toString();
+      }
+      else {
+        // Non-primitive, non-String value--include object ID.
+        final String idPrefix = "<id=" + getObjectId( value ) + "> ";
+        if ( declaredType.isInterface()
+             && JDBC_PACKAGES.contains( declaredType.getPackage() ) ) {
+          // JDBC interface implementation class--show no actual type or value
+          // (because object is proxied and therefore all uses will be traced).
+          actualTypePart = "";
+          actualValuePart = idPrefix + "...";
+        }
+        else if ( origActualType == declaredType ) {
+          // Actual type is same as declared--don't show redundant actual type.
+          actualTypePart = "";
+          actualValuePart = idPrefix + formatValue( value );
+        }
+        else {
+          // Other--show actual type and (try to) show value.
+          actualTypePart = "(" + formatType( rawActualType) + ") ";
+          actualValuePart = idPrefix + formatValue( value );
+        }
+      }
+    }
+    final String result = declaredTypePart + actualTypePart + actualValuePart;
+    return result;
+  }
+
+  /**
+   * Renders given type and value of target (receiver) of method call.
+   */
+  private String formatTargetTypeAndValue( Class<?> declaredType, Object value ) {
+    return formatTypeAndValue( declaredType, value );
+  }
+
+  /**
+   * Renders given type and value of method call argument.
+   */
+  // Expect JDBC interface types; (mostly) doesn't expect other JDBC types;
+  // expect mostly primitives and String.
+  private String formatArgTypeAndValue( Class<?> declaredType, Object value ) {
+    return formatTypeAndValue( declaredType, value );
+  }
+
+  /**
+   * Renders given type and value of method return value.
+   */
+  // Expect declared type Object (need actual type); expect primitive types
+  // (maybe wrapper classes too?)
+  private String formatReturnTypeAndValue( Class<?> declaredType, Object value ) {
+    return formatTypeAndValue( declaredType, value );
+  }
+
+  /**
+   * Renders given exception.
+   * Includes test of chained exceptions.
+   */
+  private String formatThrowable( final Throwable thrown ) {
+    final StringBuffer s = new StringBuffer();
+    boolean first = true;
+    Throwable current = thrown;
+    while ( null != current ) {
+      if ( ! first ) {
+        s.append( " ==> ");
+      }
+      first = false;
+
+      s.append( "(" );
+      s.append( formatType( current.getClass() ) );
+      s.append( ") ");
+      s.append( formatString( current.toString() ) );
+      current = current.getCause();
+    }
+    final String result = s.toString();
+    return result;
+  }
+
+  /**
+   * Renders corresponding given sequence of declared types and given sequence
+   * of values from method call.
+   */
+  private String formatArgs( Class<?>[] declaredTypes, Object[] argValues )
+  {
+    final String result;
+    if ( null == argValues ) {
+      result = "()";
+    }
+    else {
+      final StringBuilder s = new StringBuilder();
+      s.append( "( " );
+      for ( int ax = 0; ax < argValues.length; ax++ ) {
+        if ( ax > 0 ) {
+          s.append( ", " );
+        }
+        s.append( formatArgTypeAndValue( declaredTypes[ ax ], argValues[ ax ] ) );
+      }
+      s.append( " )" );
+      result = s.toString();
+    }
+    return result;
+  }
+
+  /**
+   * Renders the call part for a method call, method return, or exception-thrown
+   * event.
+   * @param target
+   *        the target (receiver) of the method call
+   * @param targetType
+   *        the interface containing called method
+   * @param method
+   *        the called method
+   * @param argValues
+   *        the argument values (represented as for {@link Method#invoke})
+   *
+   */
+  private String formatCallPart( final Object target,
+                                 final Class<?> targetType,
+                                 final Method method,
+                                 final Object[] argValues ) {
+    return
+        "(" + formatTargetTypeAndValue( targetType, target ) + ") . "
+        + method.getName() + formatArgs( method.getParameterTypes(), argValues );
+  }
+
+
+  ////////////////////
+  // Invocation-level methods:
+
+  @Override
+  public void setupMessage( final String message ) {
+    printTraceLine( SETUP_LINE_PREFIX + message );
+  }
+
+  @Override
+  public void methodCalled( final Object target,
+                            final Class<?> targetType,
+                            final Method method,
+                            final Object[] args ) {
+    printTraceLine( CALL_LINE_PREFIX
+                    + formatCallPart( target, targetType, method, args ) );
+  }
+
+  @Override
+  public void methodReturned( final Object target,
+                              final Class<?> targetType,
+                              final Method method,
+                              final Object[] args, final
+                              Object result ) {
+    final String callPart =
+        RETURN_LINE_PREFIX + formatCallPart( target, targetType, method, args );
+    if ( void.class == method.getReturnType() ) {
+      assert null == result
+          : "unexpected non-null result value " + result
+            + " for method returning " + method.getReturnType();
+      printTraceLine( callPart + ", RESULT: (none--void) " );
+    } else {
+      printTraceLine(
+          callPart + ", RESULT: "
+          + formatReturnTypeAndValue( method.getReturnType(), result ) );
+    }
+  }
+
+  @Override
+  public void methodThrew( final Object target,
+                           final Class<?> targetType,
+                           final Method method,
+                           final Object[] args,
+                           final Throwable exception ) {
+    printTraceLine( THROW_LINE_PREFIX
+                    + formatCallPart( target, targetType, method, args )
+                    + ", threw: " + formatThrowable( exception ) );
+  }
+
+} // class SimpleInvocationReporter

http://git-wip-us.apache.org/repos/asf/drill/blob/7c5a1f57/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/ProxiesManager.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/ProxiesManager.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/ProxiesManager.java
new file mode 100644
index 0000000..7d2da53
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/ProxiesManager.java
@@ -0,0 +1,117 @@
+/*
+ * 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.jdbc.proxy;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Proxy;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+
+/**
+ * Manager of proxy classes and instances.
+ * Managing includes creating proxies and tracking to re-use proxies.
+ */
+class ProxiesManager {
+  private final InvocationReporter reporter;
+
+  private Map<Class<?>, Class<Proxy>> interfacesToProxyClassesMap =
+      new IdentityHashMap<>();
+
+  /** Map of proxied original objects from proxied JDBC driver to
+   *  proxy objects to be returned by tracing proxy driver. */
+  private Map<Object, Object> proxiedsToProxiesMap = new IdentityHashMap<>();
+
+
+  public ProxiesManager( final InvocationReporter reporter ) {
+    this.reporter = reporter;
+  }
+
+  private Class<Proxy> getProxyClassForInterface( final Class<?> interfaceType ) {
+    assert interfaceType.isInterface();
+
+    Class<Proxy> proxyReturnClass = interfacesToProxyClassesMap.get( interfaceType );
+    if ( null == proxyReturnClass ) {
+
+      // Suppressed because we know getProxyClass returns class extending Proxy.
+      @SuppressWarnings("unchecked")
+      Class<Proxy> newProxyReturnClass =
+          (Class<Proxy>) Proxy.getProxyClass( interfaceType.getClassLoader(),
+                                              new Class[] { interfaceType });
+      interfacesToProxyClassesMap.put( interfaceType, proxyReturnClass );
+      proxyReturnClass = newProxyReturnClass;
+    }
+    return proxyReturnClass;
+  }
+
+
+  /**
+   * Creates or retrieves proxy instance to be returned for given original
+   * instance.
+   * @param  originalInstance
+   *         the original object
+   * @param  declaredType
+   *         the declared type of source of the original object; interface type
+   */
+  public <INTF> INTF getProxyInstanceForOriginal( final INTF originalInstance,
+                                                  final Class<?> declaredType ) {
+    final INTF proxyInstance;
+
+    assert declaredType.isAssignableFrom( originalInstance.getClass() )
+        : "toBeProxied is of class (" + originalInstance.getClass().getName()
+        + ") that doesn't implement specified interface " + declaredType.getName();
+    // Suppressed because checked immediately above.
+    @SuppressWarnings("unchecked")
+    final INTF existingProxy = (INTF) proxiedsToProxiesMap.get( originalInstance );
+
+    if ( null != existingProxy ) {
+      // Repeated occurrence of original--return same proxy instance as before.
+      proxyInstance = existingProxy;
+    }
+    else {
+      // Original we haven't seen yet--create proxy instance and return that.
+
+      Class<Proxy> proxyReturnClass = getProxyClassForInterface( declaredType );
+
+      // Create tracing handler instance for this proxy/original pair.
+      final InvocationHandler callHandler =
+          new TracingInvocationHandler<INTF>( this, reporter,
+                                              originalInstance, declaredType );
+      try {
+        // Suppressed because we know that proxy class implements INTF.
+        @SuppressWarnings("unchecked")
+        INTF newProxyInstance =
+            (INTF)
+            proxyReturnClass
+            .getConstructor( new Class[] { InvocationHandler.class } )
+            .newInstance( new Object[] { callHandler } );
+        proxiedsToProxiesMap.put( originalInstance, newProxyInstance );
+        proxyInstance = newProxyInstance;
+      }
+      catch ( InstantiationException | IllegalAccessException
+              | IllegalArgumentException | InvocationTargetException
+              | NoSuchMethodException | SecurityException e ) {
+        throw new RuntimeException(
+            "Error creating proxy for " + declaredType + ": " + e , e );
+      }
+    }
+    return proxyInstance;
+  }
+
+} // class ProxiesManager

http://git-wip-us.apache.org/repos/asf/drill/blob/7c5a1f57/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/ProxySetupSQLException.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/ProxySetupSQLException.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/ProxySetupSQLException.java
new file mode 100644
index 0000000..e1bc582
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/ProxySetupSQLException.java
@@ -0,0 +1,37 @@
+/*
+ * 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.jdbc.proxy;
+
+import java.sql.SQLNonTransientConnectionException;
+
+/**
+ * SQLException for tracing-proxy connection and setup problems.
+ */
+class ProxySetupSQLException extends SQLNonTransientConnectionException {
+
+  private static final long serialVersionUID = 2015_02_08L;
+
+  ProxySetupSQLException( String message, Throwable cause ) {
+    super( message, cause );
+  }
+
+  ProxySetupSQLException( String message ) {
+    super( message );
+  }
+
+} // class ProxySetupSQLException

http://git-wip-us.apache.org/repos/asf/drill/blob/7c5a1f57/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/TracingInvocationHandler.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/TracingInvocationHandler.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/TracingInvocationHandler.java
new file mode 100644
index 0000000..8c94309
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/TracingInvocationHandler.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.jdbc.proxy;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+/**
+ * Tracing proxy-invocation handler.
+ * Reports invocations of methods of given proxied object to given invocation
+ * reporter.
+ */
+class TracingInvocationHandler<INTF> implements InvocationHandler
+{
+  private final ProxiesManager proxiesManager;
+  private final InvocationReporter callReporter;
+  private final INTF proxiedObject;
+  private final Class<?> proxiedInterface;
+
+
+  /**
+   * Constructs invocation handler for given object, treats ~as given (single)
+   * interface.
+   *
+   * @param  proxiesManager
+   *         the proxies manager to use for creating new proxy objects
+   * @param  reporter
+   *         the invocation report to use to report invocation events
+   * @param  proxiedObject
+   *         ...
+   * @param  proxiedInterface
+   *         ...
+   */
+  TracingInvocationHandler( final ProxiesManager proxiesManager,
+                            final InvocationReporter reporter,
+                            final INTF proxiedObject,
+                            final Class<?> proxiedInterface ) {
+    this.proxiesManager = proxiesManager;
+    this.callReporter = reporter;
+    this.proxiedObject = proxiedObject;
+    this.proxiedInterface = proxiedInterface;
+  }
+
+
+  @Override
+  public Object invoke( Object proxy, Method method, Object[] args )
+      throws Throwable {
+
+    // Report that method was called:
+    callReporter.methodCalled( proxiedObject, proxiedInterface, method, args );
+
+    final Object rawReturnedResult;
+    final Object netReturnedResult;
+    try {
+      // Invoke proxied original object's method:
+      rawReturnedResult = method.invoke( proxiedObject, args );
+
+      if ( null == rawReturnedResult ) {
+        netReturnedResult = null;
+      }
+      else {
+        Class<?> methodReturnType = method.getReturnType();
+
+        if ( ! methodReturnType.isInterface() ) {
+          // Declared type is not an interface type, so don't proxy the returned
+          // instance.  (We could proxy and intercept some methods, but we can't
+          // intercept all, so intercepting only some would be misleading.)
+          netReturnedResult = rawReturnedResult;
+        }
+        else {
+          // Get the new or existing proxying instance for the returned instance.
+          netReturnedResult =
+              proxiesManager.getProxyInstanceForOriginal( rawReturnedResult,
+                                                          methodReturnType );
+        }
+      }
+    }
+    catch ( IllegalAccessException | IllegalArgumentException e ) {
+      throw new RuntimeException(
+          "Unexpected/unhandled error calling proxied method: " + e, e );
+    }
+    catch ( InvocationTargetException e ) {
+      Throwable thrownResult = e.getCause();
+      // Report that method threw exception:
+      callReporter.methodThrew( proxiedObject, proxiedInterface, method, args,
+                                thrownResult );
+      throw thrownResult;
+    }
+
+    // Report that method returned:
+    callReporter.methodReturned( proxiedObject, proxiedInterface, method, args,
+                                 rawReturnedResult );
+    return netReturnedResult;
+  } // invoke(...)
+
+} // class ProxyInvocationHandler<INTF>

http://git-wip-us.apache.org/repos/asf/drill/blob/7c5a1f57/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/TracingProxyDriver.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/TracingProxyDriver.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/TracingProxyDriver.java
new file mode 100644
index 0000000..d575d4f
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/TracingProxyDriver.java
@@ -0,0 +1,288 @@
+/*
+ * 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.jdbc.proxy;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.Properties;
+
+import org.slf4j.Logger;
+
+import static org.slf4j.LoggerFactory.getLogger;
+
+/**
+ * Proxy driver for tracing calls to a JDBC driver.
+ * Reports calls and parameter values to, and return values and exceptions from,
+ * methods defined by JDBC interfaces.
+ *
+ * <p><strong>Invocation:</strong></p>
+ * <p>
+ *   To set up a tracing version of a JDBC connection:
+ * </p>
+ * <ul>
+ *   <li>Construct the proxying URL corresponding to the original URL.  (See
+ *       below).</li>
+ *   <li>Replace the occurrence, in the JDBC-client code or tool, of the
+ *       original URL with the corresponding proxying URL.</li>
+ *   <li>Make sure that class {@link TracingProxyDriver} will be loaded (e.g.,
+ *       configure the client to use as the driver class).</li>
+ * </ul>
+ * <p>
+ *   The proxying URL corresponding to an original URL is
+ *   "<code>jdbc:proxy:<i>original.Driver</i>:<i>original_URL</i></code>",
+ *   where:
+ * </p>
+ * <ul>
+ *   <li>
+ *     <code><i>original.Driver</i></code> is the fully qualified name of the
+ *     driver class to proxy and trace (used to load the class to get it
+ *     registered with JDBC's {@link DriverManager}, when or in case it's not
+ *     already loaded); it can blank if the driver class will already be loaded,
+ *     and
+ *   </li>
+ *   <li>
+ *     <code><i>original_URL</i></code> is the original URL for the JDBC data
+ *     source to now be traced.
+ *   </li>
+ * </ul>
+ * <p>
+ *   For example, for "{@code jdbc:drill:zk=local}", the tracing URL is
+ *   "{@code jdbc:proxy:org.apache.drill.jdbc.Driver:jdbc:drill:zk=local}".
+ * </p>
+ * <p><strong>Output:</strong></p>
+ * <p>
+ *   Currently, the tracing output line are simply written to {@link System.out}
+ *   ({@code stdout} or "standard output").
+ * </p>
+ */
+public class TracingProxyDriver implements java.sql.Driver {
+
+  /** JDBC URL prefix that tracing proxy driver recognizes. */
+  private static final String JDBC_URL_PREFIX = "jdbc:proxy:";
+
+  // TODO:  Maybe split into static setup reporter vs. non-static tracing
+  // reporter, especially if output destination becomes configurable via
+  // something in the proxy/tracing URI.
+  // (Static because called in class initialization.)
+  private static final InvocationReporter reporter;
+  static {
+    InvocationReporterImpl simpleReporter = new InvocationReporterImpl();
+    // Note:  This is intended to be first line written to output:
+    simpleReporter.setupMessage( "Proxy driver " + TracingProxyDriver.class
+                                 + " initializing.");
+    simpleReporter.reportAbbreviatedPackages();
+    reporter = simpleReporter;
+  }
+
+  private final ProxiesManager proxiesManager = new ProxiesManager( reporter );
+
+  /** Most recent (and usually only) proxyDriver created by this (usually
+   *  singleton) instance. */
+  private Driver proxyDriver;
+
+
+  // Statically create and register an instance with JDBC's DriverManager.
+  static {
+    reporter.setupMessage( "Proxy driver registering with DriverManager." );
+    final Driver proxyDriver;
+    try {
+      proxyDriver = new TracingProxyDriver();
+    }
+    catch ( SQLException e ) {
+      throw new RuntimeException(
+          "Error in initializing " + TracingProxyDriver.class + ": " + e, e );
+    }
+    try {
+      DriverManager.registerDriver( proxyDriver );
+    }
+    catch ( SQLException e ) {
+      throw new RuntimeException(
+          "Error in registering " + TracingProxyDriver.class + ": " + e, e );
+    }
+  }
+
+
+  public TracingProxyDriver() throws SQLException {
+  }
+
+  private static class UrlHandler {
+    private static final String SYNTAX_TEXT =
+        "proxy URL syntax: \"" + JDBC_URL_PREFIX + "\" + \":\" "
+        + "+ optional original driver class name + \":\" "
+        + "+ proxied (original) JDBC URL";
+
+    private final String classSpec;
+    private final String proxiedURL;
+    private final Driver proxiedDriverForProxiedUrl;
+    private final Driver proxyDriver;
+
+
+    UrlHandler( ProxiesManager proxiesManager, String url )
+        throws ProxySetupSQLException {
+
+      final String variablePart = url.substring( JDBC_URL_PREFIX.length() );
+
+      final int classEndColonPos = variablePart.indexOf( ':' );
+      if ( -1 == classEndColonPos ) {
+        throw new ProxySetupSQLException(
+            "Connection URL syntax error: no third colon in proxy URL \"" + url
+            + "\"; (" + SYNTAX_TEXT + ")" );
+      }
+      // Either class name (load named class) or empty string (don't load any).
+      classSpec = variablePart.substring( 0, classEndColonPos );
+      proxiedURL = variablePart.substring( 1 + classEndColonPos );
+
+      if ( ! "".equals( classSpec ) ) {
+        try {
+          Class.forName( classSpec);
+        }
+        catch ( ClassNotFoundException e ) {
+          throw new ProxySetupSQLException(
+              "Couldn't load class \"" + classSpec + "\""
+              + " (from proxy driver URL \"" + url + "\" (between second and "
+              + "third colons)): " + e,
+              e );
+        }
+      }
+
+      try {
+        reporter.setupMessage( "Proxy calling DriverManager.getDriver(...) for"
+                               + " proxied URL \"" + proxiedURL + "\"." );
+        proxiedDriverForProxiedUrl = DriverManager.getDriver( proxiedURL );
+        reporter.setupMessage(
+            "DriverManager.getDriver( \"" + proxiedURL + "\" ) returned a(n) "
+            + proxiedDriverForProxiedUrl.getClass().getName() + ": "
+            + proxiedDriverForProxiedUrl + "." );
+      }
+      catch ( SQLException e ) {
+        final String message =
+            "Error getting driver from DriverManager for proxied URL \""
+            + proxiedURL + "\" (from proxy driver URL \"" + url + "\""
+            + " (after third colon)): " + e;
+        reporter.setupMessage( message );
+        throw new ProxySetupSQLException( message, e );
+      }
+      proxyDriver =
+          proxiesManager.getProxyInstanceForOriginal( proxiedDriverForProxiedUrl,
+                                                      Driver.class );
+    }
+
+    public String getProxiedUrl() {
+      return proxiedURL;
+    }
+
+    public Driver getProxiedDriver() {
+      return proxiedDriverForProxiedUrl;
+    }
+
+    public Driver getProxyDriver() {
+      return proxyDriver;
+    }
+
+  }  // class UrlHandler
+
+
+  private void setProxyDriver( final Driver newProxyDriver,
+                               final Driver proxiedDriver ) {
+    // Note if different proxy than before.
+    if ( null != this.proxyDriver && newProxyDriver != this.proxyDriver ) {
+      reporter.setupMessage(
+          "Note:  Multiple drivers proxied; Driver-level methods such as "
+          + "getMajorVersion() will be routed to latest"
+          + " (" + proxiedDriver + ")." );
+    }
+    this.proxyDriver = newProxyDriver;
+  }
+
+  @Override
+  public boolean acceptsURL( String url ) throws SQLException {
+    reporter.setupMessage( "Proxy's acceptsURL(...) called with "
+                           + ( null == url ? "null" : "\"" + url + "\"." ) );
+    final boolean accepted;
+    if ( null == url || ! url.startsWith( JDBC_URL_PREFIX ) ) {
+      accepted = false;
+    }
+    else {
+      UrlHandler urlHandler = new UrlHandler( proxiesManager, url );
+      setProxyDriver( urlHandler.getProxyDriver(), urlHandler.getProxiedDriver() );
+
+      accepted = true;  // (If no exception in UrlHandler(...)
+    }
+    reporter.setupMessage(
+        "Proxy's acceptsURL( " + ( null == url ? "null" : "\"" + url + "\"" )
+        + " ) returning " + accepted + "." );
+    return accepted;
+  }
+
+  @Override
+  public Connection connect( String url, Properties info )
+      throws ProxySetupSQLException {
+    final Connection result;
+    reporter.setupMessage( "Proxy's connect(...) called with URL "
+                           + ( null == url ? "null" : "\"" + url + "\"" ) + "." );
+
+    if ( null == url || ! url.startsWith( JDBC_URL_PREFIX ) ) {
+      result = null;  // (Not a URL understood by this driver.)
+    }
+    else {
+      UrlHandler urlHandler = new UrlHandler( proxiesManager, url );
+      setProxyDriver( urlHandler.getProxyDriver(), urlHandler.getProxiedDriver() );
+
+      // (Call connect() through proxy so it gets traced too.)
+      try {
+        result = proxyDriver.connect( urlHandler.getProxiedUrl(), info );
+      }
+      catch ( SQLException e ) {
+        throw new ProxySetupSQLException( "Exception from proxied driver: " + e,
+                                           e );
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public DriverPropertyInfo[] getPropertyInfo( String url, Properties info )
+      throws SQLException {
+    return proxyDriver.getPropertyInfo( url, info );
+  }
+
+  @Override
+  public int getMajorVersion() {
+    return proxyDriver.getMajorVersion();
+  }
+
+  @Override
+  public int getMinorVersion() {
+    return proxyDriver.getMinorVersion();
+  }
+  @Override
+  public boolean jdbcCompliant() {
+    return proxyDriver.jdbcCompliant();
+  }
+
+  @Override
+  public java.util.logging.Logger getParentLogger()
+      throws SQLFeatureNotSupportedException {
+    return proxyDriver.getParentLogger();
+  }
+
+} // class TracingProxyDriver

http://git-wip-us.apache.org/repos/asf/drill/blob/7c5a1f57/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/package-info.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/package-info.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/package-info.java
new file mode 100644
index 0000000..911b298
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/proxy/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+/**
+ * Tracing proxy JDBC driver.  Traces calls to another JDBC driver.
+ *
+ * @see TracingProxyDriver
+ */
+package org.apache.drill.jdbc.proxy;

http://git-wip-us.apache.org/repos/asf/drill/blob/7c5a1f57/exec/jdbc/src/test/java/org/apache/drill/jdbc/proxy/TracingProxyDriverClassLoadingTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/proxy/TracingProxyDriverClassLoadingTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/proxy/TracingProxyDriverClassLoadingTest.java
new file mode 100644
index 0000000..13becac
--- /dev/null
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/proxy/TracingProxyDriverClassLoadingTest.java
@@ -0,0 +1,135 @@
+/*
+ * 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.jdbc.proxy;
+
+import org.apache.drill.test.DrillTest;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+import static org.junit.Assert.assertThat;
+import static org.hamcrest.CoreMatchers.*;
+
+
+// NOTE:  Currently, must not inherit from anything that loads the Drill driver
+// class (and must not be run in JVM where the Drill driver class has already
+// been loaded).
+
+/**
+ * Test of TracingProxyDriver's loading of driver class.
+ */
+public class TracingProxyDriverClassLoadingTest extends DrillTest {
+
+  @Ignore( "except when run in own JVM (so Drill Driver not already loaded)" )
+  @Test
+  public void testClassLoading() throws SQLException, ClassNotFoundException {
+
+    // Note:  Throwing exceptions for test setup problems so they're JUnit
+    // errors (red in Eclipse's JUnit view), not just JUnit test failures
+    // (blue in Eclipse).
+
+    // 1.  Confirm that Drill driver is not loaded/registered.
+    try {
+      DriverManager.getDriver( "jdbc:drill:zk=local" );
+      throw new IllegalStateException(
+          "Drill driver seems loaded already; can't test loading." );
+    }
+    catch ( SQLException e ) {
+      // (Message as of JDK 1.7.)
+      assertThat( "Not expected messsage.  (Did JDK change?)",
+                  e.getMessage(), equalTo( "No suitable driver" ) );
+    }
+    try {
+      DriverManager.getConnection( "jdbc:drill:zk=local", null );
+      throw new IllegalStateException(
+          "Drill driver seems loaded already; can't test loading." );
+    }
+    catch ( SQLException e ) {
+      // (Message form as of JDK 1.7.)
+      assertThat( "Not expected messsage.  (Did JDK change?)",
+                  e.getMessage(),
+                  equalTo( "No suitable driver found for jdbc:drill:zk=local" ) );
+    }
+
+    // 2.  Confirm that TracingProxyDriver is not loaded/registered.
+    try {
+      DriverManager.getDriver( "jdbc:proxy::jdbc:drill:zk=local" );
+      throw new IllegalStateException(
+         "Proxy driver seems loaded already; can't test loading." );
+    }
+    catch ( SQLException e ) {
+      assertThat( "Not expected messsage.  (Did JDK change?)",
+                  e.getMessage(), equalTo( "No suitable driver" ) );
+    }
+    try {
+      DriverManager.getConnection( "jdbc:proxy::jdbc:drill:zk=local", null );
+      throw new IllegalStateException(
+         "Proxy driver seems loaded already; can't test loading." );
+    }
+    catch ( SQLException e ) {
+      assertThat(
+          "Not expected messsage.  (Did JDK change?)",
+          e.getMessage(),
+          equalTo( "No suitable driver found for jdbc:proxy::jdbc:drill:zk=local" ) );
+    }
+
+    // 3.  Load TracingProxyDriver.
+    Class.forName( "org.apache.drill.jdbc.proxy.TracingProxyDriver" );
+
+    // 4.  Confirm that Drill driver still is not registered.
+    try {
+      DriverManager.getConnection( "jdbc:proxy::jdbc:drill:zk=local", null );
+      throw new IllegalStateException(
+          "Drill driver seems loaded already; can't test loading." );
+    }
+    catch ( ProxySetupSQLException e ) {
+      assertThat(
+          "Not expected messsage.  (Was it just modified?)",
+          e.getMessage(),
+          equalTo(
+              "Error getting driver from DriverManager for proxied URL"
+              + " \"jdbc:drill:zk=local\" (from proxy driver URL"
+              + " \"jdbc:proxy::jdbc:drill:zk=local\" (after third colon))"
+              + ": java.sql.SQLException: No suitable driver" ) );
+    }
+
+    // 5.  Test that TracingProxyDriver can load and use a specified Driver class.
+    final Driver driver =
+        DriverManager.getDriver(
+            "jdbc:proxy:org.apache.drill.jdbc.Driver:jdbc:drill:zk=local" );
+
+    assertThat( driver.acceptsURL( "jdbc:proxy::jdbc:drill:zk=local" ),
+                equalTo( true ) );
+    assertThat( driver.acceptsURL( "jdbc:drill:zk=local" ), equalTo( false ) );
+
+    // 7.  Test minimally that driver can get connection that works.
+    final Connection proxyConnection  =
+        DriverManager.getConnection( "jdbc:proxy::jdbc:drill:zk=local", null );
+    assertThat( proxyConnection, notNullValue() );
+
+    final DatabaseMetaData dbMetaData = proxyConnection.getMetaData();
+    assertThat( dbMetaData, instanceOf( DatabaseMetaData.class ) );
+  }
+
+} // class TracingProxyDriverClassLoadingTest

http://git-wip-us.apache.org/repos/asf/drill/blob/7c5a1f57/exec/jdbc/src/test/java/org/apache/drill/jdbc/proxy/TracingProxyDriverTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/proxy/TracingProxyDriverTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/proxy/TracingProxyDriverTest.java
new file mode 100644
index 0000000..389cbac
--- /dev/null
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/proxy/TracingProxyDriverTest.java
@@ -0,0 +1,257 @@
+/*
+ * 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.jdbc.proxy;
+
+import org.apache.drill.test.DrillTest;
+
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.hamcrest.CoreMatchers.*;
+
+/**
+ * Test of TracingProxyDriver other than loading of driver classes.
+ */
+public class TracingProxyDriverTest extends DrillTest {
+
+  private static Driver proxyDriver;
+  private static Connection proxyConnection;
+
+  @BeforeClass
+  public static void setUpTestCase() throws SQLException,
+                                            ClassNotFoundException {
+    Class.forName( "org.apache.drill.jdbc.proxy.TracingProxyDriver" );
+    proxyDriver =
+        DriverManager.getDriver(
+            "jdbc:proxy:org.apache.drill.jdbc.Driver:jdbc:drill:zk=local" );
+    proxyConnection =
+        DriverManager.getConnection( "jdbc:proxy::jdbc:drill:zk=local" );
+  }
+
+  @AfterClass
+  public static void tearDownTestCase() {
+  }
+
+  @Test
+  public void testBasicProxying() throws SQLException {
+    try ( final Statement stmt = proxyConnection.createStatement() ) {
+      final ResultSet rs =
+          stmt.executeQuery( "SELECT * FROM INFORMATION_SCHEMA.CATALOGS" );
+      assertTrue( rs.next() );
+      assertThat( rs.getString( 1 ), equalTo( "DRILL" ) );
+    }
+  }
+
+  private static class StdErrCapturer {
+    private final PrintStream savedStdErr;
+    private final ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+    private final PrintStream capturingStream = new PrintStream( buffer );
+    private boolean redirected;
+
+    StdErrCapturer() {
+      savedStdErr = System.err;
+    }
+
+    void redirect() {
+      assert ! redirected;
+      redirected = true;
+      System.setErr( capturingStream );
+    }
+
+    void unredirect() {
+      assert redirected;
+      redirected = false;
+      System.setErr( savedStdErr );
+    }
+
+    String getOutput() {
+      assert ! redirected;
+      return new String( buffer.toByteArray(), StandardCharsets.UTF_8 );
+    }
+  }
+
+
+  @Test
+  public void testBasicReturnTrace() throws SQLException {
+    final StdErrCapturer nameThis = new StdErrCapturer();
+
+    try {
+      nameThis.redirect();
+      proxyConnection.isClosed();
+    }
+    finally {
+      nameThis.unredirect();
+    }
+
+    // Check captured System.err:
+
+    final String output = nameThis.getOutput();
+    final String[] lines = output.split( "\n" );
+    assertThat( "Not 2 lines: \"\"\"" + output + "\"\"\"",
+                lines.length, equalTo( 2 ) );
+    final String callLine = lines[ 0 ];
+    final String returnLine = lines[ 1 ];
+
+    // Expect something like current:
+    // TRACER: CALL:   ((Connection) <id=3> ...) . isClosed()
+    // TRACER: RETURN: ((Connection) <id=3> ...) . isClosed(), RESULT: (boolean) false
+
+    assertThat( callLine,   containsString( " CALL:" ) );
+    assertThat( returnLine, containsString( " RETURN:" ) );
+    assertThat( callLine,   containsString( "(Connection)" ) );
+    assertThat( returnLine, containsString( "(Connection)" ) );
+    assertThat( callLine,   containsString( "isClosed()" ) );
+    assertThat( returnLine, containsString( "isClosed()" ) );
+    assertThat( callLine,   not( containsString( " (boolean) " ) ) );
+    assertThat( returnLine,      containsString( " (boolean) " ) );
+    assertThat( callLine,   not( containsString( "false" ) ) );
+    assertThat( returnLine,      containsString( "false" ) );
+  }
+
+  @Test
+  public void testBasicThrowTrace() throws SQLException {
+    final StdErrCapturer stdErrCapturer = new StdErrCapturer();
+
+    final Statement statement = proxyConnection.createStatement();
+    statement.close();
+
+    try {
+      stdErrCapturer.redirect();
+      statement.execute( "" );
+    }
+    catch ( final SQLException e ) {
+      // "already closed" is expected
+    }
+    finally {
+      stdErrCapturer.unredirect();
+    }
+
+    // Check captured System.err:
+
+    final String output = stdErrCapturer.getOutput();
+    final String[] lines = output.split( "\n" );
+    assertThat( "Not 2 lines: \"\"\"" + output + "\"\"\"",
+                lines.length, equalTo( 2 ) );
+    final String callLine = lines[ 0 ];
+    final String returnLine = lines[ 1 ];
+
+    // Expect something like current:
+    // TRACER: CALL:   ((Statement) <id=6> ...) . execute( (String) "" )
+    // TRACER: THROW:  ((Statement) <id=6> ...) . execute( (String) "" ), th\
+    // rew: (org.apache.drill.jdbc.AlreadyClosedSqlException) org.apache.dri\
+    // ll.jdbc.AlreadyClosedSqlException: Statement is already closed.
+
+    assertThat( callLine,   containsString( " CALL:" ) );
+    assertThat( returnLine, containsString( " THROW:" ) );
+    assertThat( callLine,   containsString( "(Statement)" ) );
+    assertThat( returnLine, containsString( "(Statement)" ) );
+    assertThat( callLine,   containsString( "execute(" ) );
+    assertThat( returnLine, containsString( "execute(" ) );
+    assertThat( callLine,   not( containsString( "threw:" ) ) );
+    assertThat( returnLine,      containsString( "threw:" ) );
+    assertThat( callLine,   not( anyOf( containsString( "exception" ),
+                                        containsString( "Exception" ) ) ) );
+    assertThat( returnLine,      anyOf( containsString( "exception" ),
+                                        containsString( "Exception" ) )  );
+    assertThat( callLine,   not( anyOf( containsString( "closed" ),
+                                        containsString( "Closed" ) ) ) );
+    assertThat( returnLine,      anyOf( containsString( "closed" ),
+                                        containsString( "Closed" ) )  );
+  }
+
+  // TODO:  Clean up these assorted remnants; probably move into separate test
+  // methods.
+  @Test
+  public void testUnsortedMethods() throws SQLException {
+
+    // Exercise these, even though we don't check results.
+    proxyDriver.getMajorVersion();
+    proxyDriver.getMinorVersion();
+    proxyDriver.jdbcCompliant();
+    proxyDriver.getParentLogger();
+    proxyDriver.getPropertyInfo( "jdbc:proxy::jdbc:drill:zk=local", new Properties() );
+
+    final DatabaseMetaData dbMetaData = proxyConnection.getMetaData();
+    assertThat( dbMetaData, instanceOf( DatabaseMetaData.class ) );
+    assertThat( dbMetaData, notNullValue() );
+
+    assertThat( dbMetaData.getConnection(), sameInstance( proxyConnection ) );
+
+
+    dbMetaData.allTablesAreSelectable();
+    try {
+      dbMetaData.ownUpdatesAreVisible( ResultSet.TYPE_FORWARD_ONLY );
+      fail();
+    }
+    catch ( SQLException | RuntimeException e ) {
+      // expected
+    }
+
+
+    final ResultSet catalogsResultSet = dbMetaData.getCatalogs();
+    assertThat( catalogsResultSet, notNullValue() );
+    assertThat( catalogsResultSet, instanceOf( ResultSet.class ) );
+
+    catalogsResultSet.next();
+    catalogsResultSet.getString( 1 );
+    catalogsResultSet.getObject( 1 );
+
+    final ResultSetMetaData rsMetaData = catalogsResultSet.getMetaData();
+    assertThat( rsMetaData, notNullValue() );
+    assertThat( rsMetaData, instanceOf( ResultSetMetaData.class ) );
+
+    int colCount = rsMetaData.getColumnCount();
+    for ( int cx = 1; cx <= colCount; cx++ ) {
+      catalogsResultSet.getObject( cx );
+      catalogsResultSet.getString( cx );
+      try {
+        catalogsResultSet.getInt( cx );
+        fail( "Expected some kind of string-to-int exception.");
+      }
+      catch ( SQLException e ) {
+        // expected;
+      }
+
+    }
+
+    assertThat( proxyConnection.getMetaData(), sameInstance( dbMetaData ) );
+    assertThat( catalogsResultSet.getMetaData(), sameInstance( rsMetaData ) );
+  }
+
+} // class ProxyDriverTest