You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by jn...@apache.org on 2017/03/02 20:59:30 UTC

[03/27] drill git commit: DRILL-4994: Add back JDBC prepared statement for older servers

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
index 96b5669..847b726 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
@@ -17,14 +17,6 @@
  */
 package org.apache.drill.exec.rpc.user;
 
-import com.google.common.base.Strings;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
-import com.google.common.util.concurrent.AbstractCheckedFuture;
-import com.google.common.util.concurrent.CheckedFuture;
-import com.google.common.util.concurrent.SettableFuture;
-import io.netty.buffer.ByteBuf;
-
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
@@ -32,10 +24,12 @@ import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 
-import io.netty.channel.socket.SocketChannel;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+
 import org.apache.drill.common.KerberosUtil;
-import org.apache.drill.common.config.DrillProperties;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.config.DrillProperties;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
@@ -60,29 +54,35 @@ import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
 import org.apache.drill.exec.rpc.AbstractClientConnection;
 import org.apache.drill.exec.rpc.Acks;
 import org.apache.drill.exec.rpc.BasicClient;
-import org.apache.drill.exec.rpc.NonTransientRpcException;
-import org.apache.drill.exec.rpc.security.AuthenticationOutcomeListener;
 import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.rpc.NonTransientRpcException;
 import org.apache.drill.exec.rpc.OutOfMemoryHandler;
 import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
 import org.apache.drill.exec.rpc.Response;
 import org.apache.drill.exec.rpc.ResponseSender;
 import org.apache.drill.exec.rpc.RpcConnectionHandler;
 import org.apache.drill.exec.rpc.RpcException;
-
-import com.google.protobuf.MessageLite;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.rpc.security.AuthStringUtil;
+import org.apache.drill.exec.rpc.security.AuthenticationOutcomeListener;
 import org.apache.drill.exec.rpc.security.AuthenticatorFactory;
-import org.apache.drill.exec.rpc.security.plain.PlainFactory;
 import org.apache.drill.exec.rpc.security.ClientAuthenticatorProvider;
+import org.apache.drill.exec.rpc.security.plain.PlainFactory;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
 
-import javax.security.sasl.SaslClient;
-import javax.security.sasl.SaslException;
+import com.google.common.base.Strings;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.AbstractCheckedFuture;
+import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import com.google.protobuf.MessageLite;
 
+import io.netty.buffer.ByteBuf;
 import io.netty.channel.EventLoopGroup;
-import org.slf4j.Logger;
+import io.netty.channel.socket.SocketChannel;
 
 public class UserClient extends BasicClient<RpcType, UserClient.UserToBitConnection,
     UserToBitHandshake, BitToUserHandshake> {
@@ -91,9 +91,10 @@ public class UserClient extends BasicClient<RpcType, UserClient.UserToBitConnect
   private final BufferAllocator allocator;
   private final QueryResultHandler queryResultHandler = new QueryResultHandler();
   private final String clientName;
+  private final boolean supportComplexTypes;
 
   private RpcEndpointInfos serverInfos = null;
-  private boolean supportComplexTypes = true;
+  private Set<RpcType> supportedMethods = null;
 
   // these are used for authentication
   private volatile List<String> serverAuthMechanisms = null;
@@ -117,6 +118,10 @@ public class UserClient extends BasicClient<RpcType, UserClient.UserToBitConnect
     return serverInfos;
   }
 
+  public Set<RpcType> getSupportedMethods() {
+    return supportedMethods;
+  }
+
   public void submitQuery(UserResultsListener resultsListener, RunQuery query) {
     send(queryResultHandler.getWrappedListener(resultsListener), RpcType.RUN_QUERY, query, QueryId.class);
   }
@@ -346,6 +351,8 @@ public class UserClient extends BasicClient<RpcType, UserClient.UserToBitConnect
     if (inbound.hasServerInfos()) {
       serverInfos = inbound.getServerInfos();
     }
+    supportedMethods = Sets.immutableEnumSet(inbound.getSupportedMethodsList());
+
     switch (inbound.getStatus()) {
     case SUCCESS:
       break;

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
index 645ded5..ecf15dd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.rpc.user;
 
+import java.util.Set;
 import java.util.concurrent.Executor;
 
 import org.apache.drill.common.config.DrillConfig;
@@ -29,8 +30,8 @@ import org.apache.drill.exec.proto.UserBitShared.SaslMessage;
 import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
 import org.apache.drill.exec.proto.UserProtos.CreatePreparedStatementReq;
 import org.apache.drill.exec.proto.UserProtos.CreatePreparedStatementResp;
-import org.apache.drill.exec.proto.UserProtos.GetCatalogsResp;
 import org.apache.drill.exec.proto.UserProtos.GetCatalogsReq;
+import org.apache.drill.exec.proto.UserProtos.GetCatalogsResp;
 import org.apache.drill.exec.proto.UserProtos.GetColumnsReq;
 import org.apache.drill.exec.proto.UserProtos.GetColumnsResp;
 import org.apache.drill.exec.proto.UserProtos.GetQueryPlanFragments;
@@ -44,6 +45,9 @@ import org.apache.drill.exec.proto.UserProtos.RunQuery;
 import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
 import org.apache.drill.exec.rpc.RpcConfig;
 
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
 public class UserRpcConfig {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserRpcConfig.class);
 
@@ -75,4 +79,16 @@ public class UserRpcConfig {
   // prevent instantiation
   private UserRpcConfig() {
   }
+
+  /**
+   * Contains the list of methods supported by the server (from user to bit)
+   */
+  public static final Set<RpcType> SUPPORTED_SERVER_METHODS = Sets.immutableEnumSet(
+      ImmutableSet
+        .<RpcType> builder()
+        .add(RpcType.RUN_QUERY, RpcType.CANCEL_QUERY, RpcType.GET_QUERY_PLAN_FRAGMENTS, RpcType.RESUME_PAUSED_QUERY,
+          RpcType.GET_CATALOGS, RpcType.GET_SCHEMAS, RpcType.GET_TABLES, RpcType.GET_COLUMNS,
+          RpcType.CREATE_PREPARED_STATEMENT)
+        .build()
+        );
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcUtils.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcUtils.java
index e7e9ffd..c513d11 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcUtils.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.rpc.user;
 import java.lang.management.ManagementFactory;
 import java.lang.management.RuntimeMXBean;
 
+import org.apache.drill.common.Version;
 import org.apache.drill.common.util.DrillVersionInfo;
 import org.apache.drill.exec.proto.UserProtos.RpcEndpointInfos;
 
@@ -52,8 +53,23 @@ public final class UserRpcUtils {
         .setMajorVersion(DrillVersionInfo.getMajorVersion())
         .setMinorVersion(DrillVersionInfo.getMinorVersion())
         .setPatchVersion(DrillVersionInfo.getPatchVersion())
+        .setBuildNumber(DrillVersionInfo.getBuildNumber())
+        .setVersionQualifier(DrillVersionInfo.getQualifier())
         .build();
 
     return infos;
   }
+
+  /**
+   * Get the version from a {@code RpcEndpointInfos} instance
+   */
+  public static Version getVersion(RpcEndpointInfos infos) {
+    return new Version(
+        infos.getVersion(),
+        infos.getMajorVersion(),
+        infos.getMinorVersion(),
+        infos.getPatchVersion(),
+        infos.getBuildNumber(),
+        infos.getVersionQualifier());
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index 6854a3e..e917b3e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -275,7 +275,8 @@ public class UserServer extends BasicServer<RpcType, BitToUserConnection> {
 
         BitToUserHandshake.Builder respBuilder = BitToUserHandshake.newBuilder()
             .setRpcVersion(UserRpcConfig.RPC_VERSION)
-            .setServerInfos(UserRpcUtils.getRpcEndpointInfos(SERVER_NAME));
+            .setServerInfos(UserRpcUtils.getRpcEndpointInfos(SERVER_NAME))
+            .addAllSupportedMethods(UserRpcConfig.SUPPORTED_SERVER_METHODS);
 
         try {
           if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION) {

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java
index ca20c01..55cb1ff 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java
@@ -66,4 +66,8 @@ public class DrillConnectionConfig extends ConnectionConfigImpl {
     return TimeZone.getDefault();
   }
 
+  public boolean disableServerPreparedStatement() {
+    return Boolean.valueOf(props.getProperty("preparedstatement.server.disabled"));
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
index ed279a3..9b9a4c8 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
@@ -43,6 +43,7 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
+import org.apache.drill.exec.proto.UserProtos.PreparedStatement;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatchLoader;
@@ -527,10 +528,18 @@ class DrillCursor implements Cursor {
         : "currentBatchHolder.getRecordCount() not 0 (is "
           + currentBatchHolder.getRecordCount() + " in loadInitialSchema()";
 
+    final PreparedStatement preparedStatement;
     if (statement instanceof DrillPreparedStatementImpl) {
       DrillPreparedStatementImpl drillPreparedStatement = (DrillPreparedStatementImpl) statement;
-      connection.getClient().executePreparedStatement(drillPreparedStatement.getPreparedStatementHandle().getServerHandle(), resultsListener);
+      preparedStatement = drillPreparedStatement.getPreparedStatementHandle();
     } else {
+      preparedStatement = null;
+    }
+
+    if (preparedStatement != null) {
+      connection.getClient().executePreparedStatement(preparedStatement.getServerHandle(), resultsListener);
+    }
+    else {
       connection.getClient().runQuery(QueryType.SQL, signature.sql, resultsListener);
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
index 43c6c21..1c350f3 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
@@ -26,7 +26,7 @@ import java.sql.SQLFeatureNotSupportedException;
 
 import org.apache.calcite.avatica.AvaticaConnection;
 import org.apache.calcite.avatica.AvaticaDatabaseMetaData;
-import org.apache.drill.exec.proto.UserProtos.RpcEndpointInfos;
+import org.apache.drill.common.Version;
 import org.apache.drill.jdbc.AlreadyClosedSqlException;
 import org.apache.drill.jdbc.DrillDatabaseMetaData;
 
@@ -55,9 +55,14 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
     }
   }
 
-  private RpcEndpointInfos getServerInfos() throws SQLException {
+  private String getServerName() throws SQLException {
     DrillConnectionImpl connection = (DrillConnectionImpl) getConnection();
-    return connection.getClient().getServerInfos();
+    return connection.getClient().getServerName();
+  }
+
+  private Version getServerVersion() throws SQLException {
+    DrillConnectionImpl connection = (DrillConnectionImpl) getConnection();
+    return connection.getClient().getServerVersion();
   }
 
   // Note:  Dynamic proxies could be used to reduce the quantity (450?) of
@@ -130,21 +135,21 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   @Override
   public String getDatabaseProductName() throws SQLException {
     throwIfClosed();
-    RpcEndpointInfos infos = getServerInfos();
-    if (infos == null) {
+    String name = getServerName();
+    if (name == null) {
       return super.getDatabaseProductName();
     }
-    return infos.getName();
+    return name;
   }
 
   @Override
   public String getDatabaseProductVersion() throws SQLException {
     throwIfClosed();
-    RpcEndpointInfos infos = getServerInfos();
-    if (infos == null) {
+    Version version = getServerVersion();
+    if (version == null) {
       return super.getDatabaseProductVersion();
     }
-    return infos.getVersion();
+    return version.getVersion();
   }
 
   @Override
@@ -1184,21 +1189,21 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   @Override
   public int getDatabaseMajorVersion() throws SQLException {
     throwIfClosed();
-    RpcEndpointInfos infos = getServerInfos();
-    if (infos == null) {
+    Version version = getServerVersion();
+    if (version == null) {
       return super.getDatabaseMajorVersion();
     }
-    return infos.getMajorVersion();
+    return version.getMajorVersion();
   }
 
   @Override
   public int getDatabaseMinorVersion() throws SQLException {
     throwIfClosed();
-    RpcEndpointInfos infos = getServerInfos();
-    if (infos == null) {
+    Version version = getServerVersion();
+    if (version == null) {
       return super.getDatabaseMinorVersion();
     }
-    return infos.getMinorVersion();
+    return version.getMinorVersion();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
index 670a5f2..28a4372 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
@@ -34,6 +34,8 @@ import org.apache.calcite.avatica.AvaticaStatement;
 import org.apache.calcite.avatica.Helper;
 import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.Meta.StatementHandle;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.client.ServerMethod;
 import org.apache.drill.exec.proto.UserProtos.CreatePreparedStatementResp;
 import org.apache.drill.exec.proto.UserProtos.RequestStatus;
 import org.apache.drill.exec.rpc.DrillRpcFuture;
@@ -97,10 +99,27 @@ public class DrillJdbc41Factory extends DrillFactory {
                                                        int resultSetConcurrency,
                                                        int resultSetHoldability)
       throws SQLException {
-    String sql = signature.sql;
     DrillConnectionImpl drillConnection = (DrillConnectionImpl) connection;
+    DrillClient client = drillConnection.getClient();
+    if (drillConnection.getConfig().disableServerPreparedStatement() || !client.getSupportedMethods().contains(ServerMethod.PREPARED_STATEMENT)) {
+      // fallback to client side prepared statement
+      return new DrillJdbc41PreparedStatement(drillConnection, h, signature, null, resultSetType, resultSetConcurrency, resultSetHoldability);
+    }
+    return newServerPreparedStatement(drillConnection, h, signature, resultSetType,
+        resultSetConcurrency, resultSetHoldability);
+  }
+
+  private DrillJdbc41PreparedStatement newServerPreparedStatement(DrillConnectionImpl connection,
+                                                                  StatementHandle h,
+                                                                  Meta.Signature signature,
+                                                                  int resultSetType,
+                                                                  int resultSetConcurrency,
+                                                                  int resultSetHoldability
+      ) throws SQLException {
+    String sql = signature.sql;
+
     try {
-      DrillRpcFuture<CreatePreparedStatementResp> respFuture = drillConnection.getClient().createPreparedStatement(signature.sql);
+      DrillRpcFuture<CreatePreparedStatementResp> respFuture = connection.getClient().createPreparedStatement(signature.sql);
 
       CreatePreparedStatementResp resp;
       try {
@@ -133,7 +152,7 @@ public class DrillJdbc41Factory extends DrillFactory {
             "Failed to create prepared statement. Unknown status: %s, Error: %s", status, errMsgFromServer));
       }
 
-      return new DrillJdbc41PreparedStatement((DrillConnectionImpl) connection,
+      return new DrillJdbc41PreparedStatement(connection,
           h,
           signature,
           resp.getPreparedStatement(),
@@ -147,7 +166,6 @@ public class DrillJdbc41Factory extends DrillFactory {
     } catch (Exception e) {
       throw Helper.INSTANCE.createException("Error while preparing statement [" + sql + "]", e);
     }
-
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
index 2894f61..f1ba4c1 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
@@ -58,9 +58,13 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
           resultSetType, resultSetConcurrency, resultSetHoldability);
     connection.openStatementsRegistry.addStatement(this);
     this.preparedStatementHandle = preparedStatementHandle;
-    ((DrillColumnMetaDataList) signature.columns).updateColumnMetaData(preparedStatementHandle.getColumnsList());
+    if (preparedStatementHandle != null) {
+      ((DrillColumnMetaDataList) signature.columns).updateColumnMetaData(preparedStatementHandle.getColumnsList());
+    }
   }
 
+
+
   /**
    * Throws AlreadyClosedSqlException <i>iff</i> this PreparedStatement is closed.
    *

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/exec/jdbc/src/test/java/org/apache/drill/jdbc/LegacyPreparedStatementTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/LegacyPreparedStatementTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/LegacyPreparedStatementTest.java
new file mode 100644
index 0000000..46d675f
--- /dev/null
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/LegacyPreparedStatementTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import static org.hamcrest.CoreMatchers.allOf;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.hamcrest.Matcher;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test that prepared statements works even if not supported on server, to some extent.
+ */
+public class LegacyPreparedStatementTest extends JdbcTestBase {
+  /** Fuzzy matcher for parameters-not-supported message assertions.  (Based on
+   *  current "Prepared-statement dynamic parameters are not supported.") */
+  private static final Matcher<String> PARAMETERS_NOT_SUPPORTED_MSG_MATCHER =
+      allOf( containsString( "arameter" ),   // allows "Parameter"
+             containsString( "not" ),        // (could have false matches)
+             containsString( "support" ) );  // allows "supported"
+
+  private static Connection connection;
+
+  @BeforeClass
+  public static void setUpConnection() throws SQLException {
+    Driver.load();
+    Properties properties = new Properties();
+    properties.setProperty("preparedstatement.server.disabled", "true");
+
+    connection = DriverManager.getConnection( "jdbc:drill:zk=local", properties);
+  }
+
+  @AfterClass
+  public static void tearDownConnection() throws SQLException {
+    if (connection != null) {
+      try (Statement stmt = connection.createStatement()) {
+        stmt.execute(String.format("alter session set `%s` = false", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY));
+      }
+    }
+    connection.close();
+  }
+
+  //////////
+  // Basic querying-works test:
+
+  /** Tests that basic executeQuery() (with query statement) works. */
+  @Test
+  public void testExecuteQueryBasicCaseWorks() throws SQLException {
+    try (PreparedStatement stmt = connection.prepareStatement( "VALUES 11" )) {
+      try(ResultSet rs = stmt.executeQuery()) {
+        assertThat("Unexpected column count",
+            rs.getMetaData().getColumnCount(), equalTo(1)
+        );
+        assertTrue("No expected first row", rs.next());
+        assertThat(rs.getInt(1), equalTo(11));
+        assertFalse("Unexpected second row", rs.next());
+      }
+    }
+  }
+
+  //////////
+  // Parameters-not-implemented tests:
+
+  /** Tests that "not supported" has priority over possible "no parameters"
+   *  check. */
+  @Test( expected = SQLFeatureNotSupportedException.class )
+  public void testParamSettingWhenNoParametersIndexSaysUnsupported() throws SQLException {
+    try(PreparedStatement prepStmt = connection.prepareStatement( "VALUES 1" )) {
+      try {
+        prepStmt.setBytes(4, null);
+      } catch (final SQLFeatureNotSupportedException e) {
+        assertThat(
+            "Check whether params.-unsupported wording changed or checks changed.",
+            e.toString(), PARAMETERS_NOT_SUPPORTED_MSG_MATCHER
+        );
+        throw e;
+      }
+    }
+  }
+
+  /** Tests that "not supported" has priority over possible "type not supported"
+   *  check. */
+  @Test( expected = SQLFeatureNotSupportedException.class )
+  public void testParamSettingWhenUnsupportedTypeSaysUnsupported() throws SQLException {
+    try(PreparedStatement prepStmt = connection.prepareStatement( "VALUES 1" )) {
+      try {
+        prepStmt.setClob(2, (Clob) null);
+      } catch (final SQLFeatureNotSupportedException e) {
+        assertThat(
+            "Check whether params.-unsupported wording changed or checks changed.",
+            e.toString(), PARAMETERS_NOT_SUPPORTED_MSG_MATCHER
+        );
+        throw e;
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
index 1adf7a7..dd8c684 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
@@ -278,6 +278,10 @@ public final class SchemaUserProtos
                     output.writeUInt32(5, message.getPatchVersion(), false);
                 if(message.hasApplication())
                     output.writeString(6, message.getApplication(), false);
+                if(message.hasBuildNumber())
+                    output.writeUInt32(7, message.getBuildNumber(), false);
+                if(message.hasVersionQualifier())
+                    output.writeString(8, message.getVersionQualifier(), false);
             }
             public boolean isInitialized(org.apache.drill.exec.proto.UserProtos.RpcEndpointInfos message)
             {
@@ -335,6 +339,12 @@ public final class SchemaUserProtos
                         case 6:
                             builder.setApplication(input.readString());
                             break;
+                        case 7:
+                            builder.setBuildNumber(input.readUInt32());
+                            break;
+                        case 8:
+                            builder.setVersionQualifier(input.readString());
+                            break;
                         default:
                             input.handleUnknownField(number, this);
                     }
@@ -381,6 +391,8 @@ public final class SchemaUserProtos
                 case 4: return "minorVersion";
                 case 5: return "patchVersion";
                 case 6: return "application";
+                case 7: return "buildNumber";
+                case 8: return "versionQualifier";
                 default: return null;
             }
         }
@@ -398,6 +410,8 @@ public final class SchemaUserProtos
             fieldMap.put("minorVersion", 4);
             fieldMap.put("patchVersion", 5);
             fieldMap.put("application", 6);
+            fieldMap.put("buildNumber", 7);
+            fieldMap.put("versionQualifier", 8);
         }
     }
 
@@ -981,6 +995,8 @@ public final class SchemaUserProtos
 
                 for(String authenticationMechanisms : message.getAuthenticationMechanismsList())
                     output.writeString(7, authenticationMechanisms, true);
+                for(org.apache.drill.exec.proto.UserProtos.RpcType supportedMethods : message.getSupportedMethodsList())
+                    output.writeEnum(8, supportedMethods.getNumber(), true);
             }
             public boolean isInitialized(org.apache.drill.exec.proto.UserProtos.BitToUserHandshake message)
             {
@@ -1039,6 +1055,9 @@ public final class SchemaUserProtos
                         case 7:
                             builder.addAuthenticationMechanisms(input.readString());
                             break;
+                        case 8:
+                            builder.addSupportedMethods(org.apache.drill.exec.proto.UserProtos.RpcType.valueOf(input.readEnum()));
+                            break;
                         default:
                             input.handleUnknownField(number, this);
                     }
@@ -1085,6 +1104,7 @@ public final class SchemaUserProtos
                 case 5: return "errorMessage";
                 case 6: return "serverInfos";
                 case 7: return "authenticationMechanisms";
+                case 8: return "supportedMethods";
                 default: return null;
             }
         }
@@ -1102,6 +1122,7 @@ public final class SchemaUserProtos
             fieldMap.put("errorMessage", 5);
             fieldMap.put("serverInfos", 6);
             fieldMap.put("authenticationMechanisms", 7);
+            fieldMap.put("supportedMethods", 8);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
index e82d22b..daa3903 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
@@ -2597,6 +2597,51 @@ public final class UserProtos {
      */
     com.google.protobuf.ByteString
         getApplicationBytes();
+
+    // optional uint32 buildNumber = 7;
+    /**
+     * <code>optional uint32 buildNumber = 7;</code>
+     *
+     * <pre>
+     * example: 32
+     * </pre>
+     */
+    boolean hasBuildNumber();
+    /**
+     * <code>optional uint32 buildNumber = 7;</code>
+     *
+     * <pre>
+     * example: 32
+     * </pre>
+     */
+    int getBuildNumber();
+
+    // optional string versionQualifier = 8;
+    /**
+     * <code>optional string versionQualifier = 8;</code>
+     *
+     * <pre>
+     * example: SNAPSHOT
+     * </pre>
+     */
+    boolean hasVersionQualifier();
+    /**
+     * <code>optional string versionQualifier = 8;</code>
+     *
+     * <pre>
+     * example: SNAPSHOT
+     * </pre>
+     */
+    java.lang.String getVersionQualifier();
+    /**
+     * <code>optional string versionQualifier = 8;</code>
+     *
+     * <pre>
+     * example: SNAPSHOT
+     * </pre>
+     */
+    com.google.protobuf.ByteString
+        getVersionQualifierBytes();
   }
   /**
    * Protobuf type {@code exec.user.RpcEndpointInfos}
@@ -2679,6 +2724,16 @@ public final class UserProtos {
               application_ = input.readBytes();
               break;
             }
+            case 56: {
+              bitField0_ |= 0x00000040;
+              buildNumber_ = input.readUInt32();
+              break;
+            }
+            case 66: {
+              bitField0_ |= 0x00000080;
+              versionQualifier_ = input.readBytes();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -2956,6 +3011,85 @@ public final class UserProtos {
       }
     }
 
+    // optional uint32 buildNumber = 7;
+    public static final int BUILDNUMBER_FIELD_NUMBER = 7;
+    private int buildNumber_;
+    /**
+     * <code>optional uint32 buildNumber = 7;</code>
+     *
+     * <pre>
+     * example: 32
+     * </pre>
+     */
+    public boolean hasBuildNumber() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>optional uint32 buildNumber = 7;</code>
+     *
+     * <pre>
+     * example: 32
+     * </pre>
+     */
+    public int getBuildNumber() {
+      return buildNumber_;
+    }
+
+    // optional string versionQualifier = 8;
+    public static final int VERSIONQUALIFIER_FIELD_NUMBER = 8;
+    private java.lang.Object versionQualifier_;
+    /**
+     * <code>optional string versionQualifier = 8;</code>
+     *
+     * <pre>
+     * example: SNAPSHOT
+     * </pre>
+     */
+    public boolean hasVersionQualifier() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional string versionQualifier = 8;</code>
+     *
+     * <pre>
+     * example: SNAPSHOT
+     * </pre>
+     */
+    public java.lang.String getVersionQualifier() {
+      java.lang.Object ref = versionQualifier_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          versionQualifier_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string versionQualifier = 8;</code>
+     *
+     * <pre>
+     * example: SNAPSHOT
+     * </pre>
+     */
+    public com.google.protobuf.ByteString
+        getVersionQualifierBytes() {
+      java.lang.Object ref = versionQualifier_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        versionQualifier_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
     private void initFields() {
       name_ = "";
       version_ = "";
@@ -2963,6 +3097,8 @@ public final class UserProtos {
       minorVersion_ = 0;
       patchVersion_ = 0;
       application_ = "";
+      buildNumber_ = 0;
+      versionQualifier_ = "";
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -2994,6 +3130,12 @@ public final class UserProtos {
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         output.writeBytes(6, getApplicationBytes());
       }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeUInt32(7, buildNumber_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeBytes(8, getVersionQualifierBytes());
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -3027,6 +3169,14 @@ public final class UserProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBytesSize(6, getApplicationBytes());
       }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(7, buildNumber_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(8, getVersionQualifierBytes());
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -3155,6 +3305,10 @@ public final class UserProtos {
         bitField0_ = (bitField0_ & ~0x00000010);
         application_ = "";
         bitField0_ = (bitField0_ & ~0x00000020);
+        buildNumber_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        versionQualifier_ = "";
+        bitField0_ = (bitField0_ & ~0x00000080);
         return this;
       }
 
@@ -3207,6 +3361,14 @@ public final class UserProtos {
           to_bitField0_ |= 0x00000020;
         }
         result.application_ = application_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.buildNumber_ = buildNumber_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.versionQualifier_ = versionQualifier_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -3247,6 +3409,14 @@ public final class UserProtos {
           application_ = other.application_;
           onChanged();
         }
+        if (other.hasBuildNumber()) {
+          setBuildNumber(other.getBuildNumber());
+        }
+        if (other.hasVersionQualifier()) {
+          bitField0_ |= 0x00000080;
+          versionQualifier_ = other.versionQualifier_;
+          onChanged();
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -3715,6 +3885,153 @@ public final class UserProtos {
         return this;
       }
 
+      // optional uint32 buildNumber = 7;
+      private int buildNumber_ ;
+      /**
+       * <code>optional uint32 buildNumber = 7;</code>
+       *
+       * <pre>
+       * example: 32
+       * </pre>
+       */
+      public boolean hasBuildNumber() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional uint32 buildNumber = 7;</code>
+       *
+       * <pre>
+       * example: 32
+       * </pre>
+       */
+      public int getBuildNumber() {
+        return buildNumber_;
+      }
+      /**
+       * <code>optional uint32 buildNumber = 7;</code>
+       *
+       * <pre>
+       * example: 32
+       * </pre>
+       */
+      public Builder setBuildNumber(int value) {
+        bitField0_ |= 0x00000040;
+        buildNumber_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint32 buildNumber = 7;</code>
+       *
+       * <pre>
+       * example: 32
+       * </pre>
+       */
+      public Builder clearBuildNumber() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        buildNumber_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // optional string versionQualifier = 8;
+      private java.lang.Object versionQualifier_ = "";
+      /**
+       * <code>optional string versionQualifier = 8;</code>
+       *
+       * <pre>
+       * example: SNAPSHOT
+       * </pre>
+       */
+      public boolean hasVersionQualifier() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>optional string versionQualifier = 8;</code>
+       *
+       * <pre>
+       * example: SNAPSHOT
+       * </pre>
+       */
+      public java.lang.String getVersionQualifier() {
+        java.lang.Object ref = versionQualifier_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          versionQualifier_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string versionQualifier = 8;</code>
+       *
+       * <pre>
+       * example: SNAPSHOT
+       * </pre>
+       */
+      public com.google.protobuf.ByteString
+          getVersionQualifierBytes() {
+        java.lang.Object ref = versionQualifier_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          versionQualifier_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string versionQualifier = 8;</code>
+       *
+       * <pre>
+       * example: SNAPSHOT
+       * </pre>
+       */
+      public Builder setVersionQualifier(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000080;
+        versionQualifier_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string versionQualifier = 8;</code>
+       *
+       * <pre>
+       * example: SNAPSHOT
+       * </pre>
+       */
+      public Builder clearVersionQualifier() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        versionQualifier_ = getDefaultInstance().getVersionQualifier();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string versionQualifier = 8;</code>
+       *
+       * <pre>
+       * example: SNAPSHOT
+       * </pre>
+       */
+      public Builder setVersionQualifierBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000080;
+        versionQualifier_ = value;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:exec.user.RpcEndpointInfos)
     }
 
@@ -7616,6 +7933,20 @@ public final class UserProtos {
      */
     com.google.protobuf.ByteString
         getAuthenticationMechanismsBytes(int index);
+
+    // repeated .exec.user.RpcType supported_methods = 8;
+    /**
+     * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+     */
+    java.util.List<org.apache.drill.exec.proto.UserProtos.RpcType> getSupportedMethodsList();
+    /**
+     * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+     */
+    int getSupportedMethodsCount();
+    /**
+     * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+     */
+    org.apache.drill.exec.proto.UserProtos.RpcType getSupportedMethods(int index);
   }
   /**
    * Protobuf type {@code exec.user.BitToUserHandshake}
@@ -7715,6 +8046,39 @@ public final class UserProtos {
               authenticationMechanisms_.add(input.readBytes());
               break;
             }
+            case 64: {
+              int rawValue = input.readEnum();
+              org.apache.drill.exec.proto.UserProtos.RpcType value = org.apache.drill.exec.proto.UserProtos.RpcType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(8, rawValue);
+              } else {
+                if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+                  supportedMethods_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserProtos.RpcType>();
+                  mutable_bitField0_ |= 0x00000040;
+                }
+                supportedMethods_.add(value);
+              }
+              break;
+            }
+            case 66: {
+              int length = input.readRawVarint32();
+              int oldLimit = input.pushLimit(length);
+              while(input.getBytesUntilLimit() > 0) {
+                int rawValue = input.readEnum();
+                org.apache.drill.exec.proto.UserProtos.RpcType value = org.apache.drill.exec.proto.UserProtos.RpcType.valueOf(rawValue);
+                if (value == null) {
+                  unknownFields.mergeVarintField(8, rawValue);
+                } else {
+                  if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+                    supportedMethods_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserProtos.RpcType>();
+                    mutable_bitField0_ |= 0x00000040;
+                  }
+                  supportedMethods_.add(value);
+                }
+              }
+              input.popLimit(oldLimit);
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -7726,6 +8090,9 @@ public final class UserProtos {
         if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
           authenticationMechanisms_ = new com.google.protobuf.UnmodifiableLazyStringList(authenticationMechanisms_);
         }
+        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+          supportedMethods_ = java.util.Collections.unmodifiableList(supportedMethods_);
+        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
@@ -7928,6 +8295,28 @@ public final class UserProtos {
       return authenticationMechanisms_.getByteString(index);
     }
 
+    // repeated .exec.user.RpcType supported_methods = 8;
+    public static final int SUPPORTED_METHODS_FIELD_NUMBER = 8;
+    private java.util.List<org.apache.drill.exec.proto.UserProtos.RpcType> supportedMethods_;
+    /**
+     * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+     */
+    public java.util.List<org.apache.drill.exec.proto.UserProtos.RpcType> getSupportedMethodsList() {
+      return supportedMethods_;
+    }
+    /**
+     * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+     */
+    public int getSupportedMethodsCount() {
+      return supportedMethods_.size();
+    }
+    /**
+     * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+     */
+    public org.apache.drill.exec.proto.UserProtos.RpcType getSupportedMethods(int index) {
+      return supportedMethods_.get(index);
+    }
+
     private void initFields() {
       rpcVersion_ = 0;
       status_ = org.apache.drill.exec.proto.UserProtos.HandshakeStatus.SUCCESS;
@@ -7935,6 +8324,7 @@ public final class UserProtos {
       errorMessage_ = "";
       serverInfos_ = org.apache.drill.exec.proto.UserProtos.RpcEndpointInfos.getDefaultInstance();
       authenticationMechanisms_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      supportedMethods_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -7966,6 +8356,9 @@ public final class UserProtos {
       for (int i = 0; i < authenticationMechanisms_.size(); i++) {
         output.writeBytes(7, authenticationMechanisms_.getByteString(i));
       }
+      for (int i = 0; i < supportedMethods_.size(); i++) {
+        output.writeEnum(8, supportedMethods_.get(i).getNumber());
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -8004,6 +8397,15 @@ public final class UserProtos {
         size += dataSize;
         size += 1 * getAuthenticationMechanismsList().size();
       }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < supportedMethods_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeEnumSizeNoTag(supportedMethods_.get(i).getNumber());
+        }
+        size += dataSize;
+        size += 1 * supportedMethods_.size();
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -8137,6 +8539,8 @@ public final class UserProtos {
         bitField0_ = (bitField0_ & ~0x00000010);
         authenticationMechanisms_ = com.google.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000020);
+        supportedMethods_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000040);
         return this;
       }
 
@@ -8195,6 +8599,11 @@ public final class UserProtos {
           bitField0_ = (bitField0_ & ~0x00000020);
         }
         result.authenticationMechanisms_ = authenticationMechanisms_;
+        if (((bitField0_ & 0x00000040) == 0x00000040)) {
+          supportedMethods_ = java.util.Collections.unmodifiableList(supportedMethods_);
+          bitField0_ = (bitField0_ & ~0x00000040);
+        }
+        result.supportedMethods_ = supportedMethods_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -8240,6 +8649,16 @@ public final class UserProtos {
           }
           onChanged();
         }
+        if (!other.supportedMethods_.isEmpty()) {
+          if (supportedMethods_.isEmpty()) {
+            supportedMethods_ = other.supportedMethods_;
+            bitField0_ = (bitField0_ & ~0x00000040);
+          } else {
+            ensureSupportedMethodsIsMutable();
+            supportedMethods_.addAll(other.supportedMethods_);
+          }
+          onChanged();
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -8694,6 +9113,78 @@ public final class UserProtos {
         return this;
       }
 
+      // repeated .exec.user.RpcType supported_methods = 8;
+      private java.util.List<org.apache.drill.exec.proto.UserProtos.RpcType> supportedMethods_ =
+        java.util.Collections.emptyList();
+      private void ensureSupportedMethodsIsMutable() {
+        if (!((bitField0_ & 0x00000040) == 0x00000040)) {
+          supportedMethods_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserProtos.RpcType>(supportedMethods_);
+          bitField0_ |= 0x00000040;
+        }
+      }
+      /**
+       * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+       */
+      public java.util.List<org.apache.drill.exec.proto.UserProtos.RpcType> getSupportedMethodsList() {
+        return java.util.Collections.unmodifiableList(supportedMethods_);
+      }
+      /**
+       * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+       */
+      public int getSupportedMethodsCount() {
+        return supportedMethods_.size();
+      }
+      /**
+       * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+       */
+      public org.apache.drill.exec.proto.UserProtos.RpcType getSupportedMethods(int index) {
+        return supportedMethods_.get(index);
+      }
+      /**
+       * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+       */
+      public Builder setSupportedMethods(
+          int index, org.apache.drill.exec.proto.UserProtos.RpcType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        ensureSupportedMethodsIsMutable();
+        supportedMethods_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+       */
+      public Builder addSupportedMethods(org.apache.drill.exec.proto.UserProtos.RpcType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        ensureSupportedMethodsIsMutable();
+        supportedMethods_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+       */
+      public Builder addAllSupportedMethods(
+          java.lang.Iterable<? extends org.apache.drill.exec.proto.UserProtos.RpcType> values) {
+        ensureSupportedMethodsIsMutable();
+        super.addAll(values, supportedMethods_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated .exec.user.RpcType supported_methods = 8;</code>
+       */
+      public Builder clearSupportedMethods() {
+        supportedMethods_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000040);
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:exec.user.BitToUserHandshake)
     }
 
@@ -29578,136 +30069,138 @@ public final class UserProtos {
       "tControl.proto\032\025ExecutionProtos.proto\"&\n" +
       "\010Property\022\013\n\003key\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\"9\n" +
       "\016UserProperties\022\'\n\nproperties\030\001 \003(\0132\023.ex" +
-      "ec.user.Property\"\210\001\n\020RpcEndpointInfos\022\014\n" +
+      "ec.user.Property\"\267\001\n\020RpcEndpointInfos\022\014\n" +
       "\004name\030\001 \001(\t\022\017\n\007version\030\002 \001(\t\022\024\n\014majorVer" +
       "sion\030\003 \001(\r\022\024\n\014minorVersion\030\004 \001(\r\022\024\n\014patc" +
-      "hVersion\030\005 \001(\r\022\023\n\013application\030\006 \001(\t\"\375\002\n\022" +
-      "UserToBitHandshake\022.\n\007channel\030\001 \001(\0162\027.ex",
-      "ec.shared.RpcChannel:\004USER\022\031\n\021support_li" +
-      "stening\030\002 \001(\010\022\023\n\013rpc_version\030\003 \001(\005\0221\n\013cr" +
-      "edentials\030\004 \001(\0132\034.exec.shared.UserCreden" +
-      "tials\022-\n\nproperties\030\005 \001(\0132\031.exec.user.Us" +
-      "erProperties\022$\n\025support_complex_types\030\006 " +
-      "\001(\010:\005false\022\036\n\017support_timeout\030\007 \001(\010:\005fal" +
-      "se\0221\n\014client_infos\030\010 \001(\0132\033.exec.user.Rpc" +
-      "EndpointInfos\022,\n\014sasl_support\030\t \001(\0162\026.ex" +
-      "ec.user.SaslSupport\"S\n\016RequestResults\022&\n" +
-      "\010query_id\030\001 \001(\0132\024.exec.shared.QueryId\022\031\n",
-      "\021maximum_responses\030\002 \001(\005\"g\n\025GetQueryPlan" +
-      "Fragments\022\r\n\005query\030\001 \002(\t\022$\n\004type\030\002 \001(\0162\026" +
-      ".exec.shared.QueryType\022\031\n\nsplit_plan\030\003 \001" +
-      "(\010:\005false\"\316\001\n\022QueryPlanFragments\0223\n\006stat" +
-      "us\030\001 \002(\0162#.exec.shared.QueryResult.Query" +
-      "State\022&\n\010query_id\030\002 \001(\0132\024.exec.shared.Qu" +
-      "eryId\0221\n\tfragments\030\003 \003(\0132\036.exec.bit.cont" +
-      "rol.PlanFragment\022(\n\005error\030\004 \001(\0132\031.exec.s" +
-      "hared.DrillPBError\"\321\001\n\022BitToUserHandshak" +
-      "e\022\023\n\013rpc_version\030\002 \001(\005\022*\n\006status\030\003 \001(\0162\032",
-      ".exec.user.HandshakeStatus\022\017\n\007errorId\030\004 " +
-      "\001(\t\022\024\n\014errorMessage\030\005 \001(\t\0221\n\014server_info" +
-      "s\030\006 \001(\0132\033.exec.user.RpcEndpointInfos\022 \n\030" +
-      "authenticationMechanisms\030\007 \003(\t\"-\n\nLikeFi" +
-      "lter\022\017\n\007pattern\030\001 \001(\t\022\016\n\006escape\030\002 \001(\t\"D\n" +
-      "\016GetCatalogsReq\0222\n\023catalog_name_filter\030\001" +
-      " \001(\0132\025.exec.user.LikeFilter\"M\n\017CatalogMe" +
-      "tadata\022\024\n\014catalog_name\030\001 \001(\t\022\023\n\013descript" +
-      "ion\030\002 \001(\t\022\017\n\007connect\030\003 \001(\t\"\223\001\n\017GetCatalo" +
-      "gsResp\022(\n\006status\030\001 \001(\0162\030.exec.user.Reque",
-      "stStatus\022,\n\010catalogs\030\002 \003(\0132\032.exec.user.C" +
-      "atalogMetadata\022(\n\005error\030\003 \001(\0132\031.exec.sha" +
-      "red.DrillPBError\"v\n\rGetSchemasReq\0222\n\023cat" +
-      "alog_name_filter\030\001 \001(\0132\025.exec.user.LikeF" +
-      "ilter\0221\n\022schema_name_filter\030\002 \001(\0132\025.exec" +
-      ".user.LikeFilter\"i\n\016SchemaMetadata\022\024\n\014ca" +
-      "talog_name\030\001 \001(\t\022\023\n\013schema_name\030\002 \001(\t\022\r\n" +
-      "\005owner\030\003 \001(\t\022\014\n\004type\030\004 \001(\t\022\017\n\007mutable\030\005 " +
-      "\001(\t\"\220\001\n\016GetSchemasResp\022(\n\006status\030\001 \001(\0162\030" +
-      ".exec.user.RequestStatus\022*\n\007schemas\030\002 \003(",
-      "\0132\031.exec.user.SchemaMetadata\022(\n\005error\030\003 " +
-      "\001(\0132\031.exec.shared.DrillPBError\"\302\001\n\014GetTa" +
-      "blesReq\0222\n\023catalog_name_filter\030\001 \001(\0132\025.e" +
+      "hVersion\030\005 \001(\r\022\023\n\013application\030\006 \001(\t\022\023\n\013b" +
+      "uildNumber\030\007 \001(\r\022\030\n\020versionQualifier\030\010 \001",
+      "(\t\"\375\002\n\022UserToBitHandshake\022.\n\007channel\030\001 \001" +
+      "(\0162\027.exec.shared.RpcChannel:\004USER\022\031\n\021sup" +
+      "port_listening\030\002 \001(\010\022\023\n\013rpc_version\030\003 \001(" +
+      "\005\0221\n\013credentials\030\004 \001(\0132\034.exec.shared.Use" +
+      "rCredentials\022-\n\nproperties\030\005 \001(\0132\031.exec." +
+      "user.UserProperties\022$\n\025support_complex_t" +
+      "ypes\030\006 \001(\010:\005false\022\036\n\017support_timeout\030\007 \001" +
+      "(\010:\005false\0221\n\014client_infos\030\010 \001(\0132\033.exec.u" +
+      "ser.RpcEndpointInfos\022,\n\014sasl_support\030\t \001" +
+      "(\0162\026.exec.user.SaslSupport\"S\n\016RequestRes",
+      "ults\022&\n\010query_id\030\001 \001(\0132\024.exec.shared.Que" +
+      "ryId\022\031\n\021maximum_responses\030\002 \001(\005\"g\n\025GetQu" +
+      "eryPlanFragments\022\r\n\005query\030\001 \002(\t\022$\n\004type\030" +
+      "\002 \001(\0162\026.exec.shared.QueryType\022\031\n\nsplit_p" +
+      "lan\030\003 \001(\010:\005false\"\316\001\n\022QueryPlanFragments\022" +
+      "3\n\006status\030\001 \002(\0162#.exec.shared.QueryResul" +
+      "t.QueryState\022&\n\010query_id\030\002 \001(\0132\024.exec.sh" +
+      "ared.QueryId\0221\n\tfragments\030\003 \003(\0132\036.exec.b" +
+      "it.control.PlanFragment\022(\n\005error\030\004 \001(\0132\031" +
+      ".exec.shared.DrillPBError\"\200\002\n\022BitToUserH",
+      "andshake\022\023\n\013rpc_version\030\002 \001(\005\022*\n\006status\030" +
+      "\003 \001(\0162\032.exec.user.HandshakeStatus\022\017\n\007err" +
+      "orId\030\004 \001(\t\022\024\n\014errorMessage\030\005 \001(\t\0221\n\014serv" +
+      "er_infos\030\006 \001(\0132\033.exec.user.RpcEndpointIn" +
+      "fos\022 \n\030authenticationMechanisms\030\007 \003(\t\022-\n" +
+      "\021supported_methods\030\010 \003(\0162\022.exec.user.Rpc" +
+      "Type\"-\n\nLikeFilter\022\017\n\007pattern\030\001 \001(\t\022\016\n\006e" +
+      "scape\030\002 \001(\t\"D\n\016GetCatalogsReq\0222\n\023catalog" +
+      "_name_filter\030\001 \001(\0132\025.exec.user.LikeFilte" +
+      "r\"M\n\017CatalogMetadata\022\024\n\014catalog_name\030\001 \001",
+      "(\t\022\023\n\013description\030\002 \001(\t\022\017\n\007connect\030\003 \001(\t" +
+      "\"\223\001\n\017GetCatalogsResp\022(\n\006status\030\001 \001(\0162\030.e" +
+      "xec.user.RequestStatus\022,\n\010catalogs\030\002 \003(\013" +
+      "2\032.exec.user.CatalogMetadata\022(\n\005error\030\003 " +
+      "\001(\0132\031.exec.shared.DrillPBError\"v\n\rGetSch" +
+      "emasReq\0222\n\023catalog_name_filter\030\001 \001(\0132\025.e" +
       "xec.user.LikeFilter\0221\n\022schema_name_filte" +
-      "r\030\002 \001(\0132\025.exec.user.LikeFilter\0220\n\021table_" +
-      "name_filter\030\003 \001(\0132\025.exec.user.LikeFilter" +
-      "\022\031\n\021table_type_filter\030\004 \003(\t\"\\\n\rTableMeta" +
-      "data\022\024\n\014catalog_name\030\001 \001(\t\022\023\n\013schema_nam" +
-      "e\030\002 \001(\t\022\022\n\ntable_name\030\003 \001(\t\022\014\n\004type\030\004 \001(" +
-      "\t\"\215\001\n\rGetTablesResp\022(\n\006status\030\001 \001(\0162\030.ex",
-      "ec.user.RequestStatus\022(\n\006tables\030\002 \003(\0132\030." +
-      "exec.user.TableMetadata\022(\n\005error\030\003 \001(\0132\031" +
-      ".exec.shared.DrillPBError\"\333\001\n\rGetColumns" +
-      "Req\0222\n\023catalog_name_filter\030\001 \001(\0132\025.exec." +
-      "user.LikeFilter\0221\n\022schema_name_filter\030\002 " +
-      "\001(\0132\025.exec.user.LikeFilter\0220\n\021table_name" +
-      "_filter\030\003 \001(\0132\025.exec.user.LikeFilter\0221\n\022" +
-      "column_name_filter\030\004 \001(\0132\025.exec.user.Lik" +
-      "eFilter\"\251\003\n\016ColumnMetadata\022\024\n\014catalog_na" +
-      "me\030\001 \001(\t\022\023\n\013schema_name\030\002 \001(\t\022\022\n\ntable_n",
-      "ame\030\003 \001(\t\022\023\n\013column_name\030\004 \001(\t\022\030\n\020ordina" +
-      "l_position\030\005 \001(\005\022\025\n\rdefault_value\030\006 \001(\t\022" +
-      "\023\n\013is_nullable\030\007 \001(\010\022\021\n\tdata_type\030\010 \001(\t\022" +
-      "\027\n\017char_max_length\030\t \001(\005\022\031\n\021char_octet_l" +
-      "ength\030\n \001(\005\022\031\n\021numeric_precision\030\013 \001(\005\022\037" +
-      "\n\027numeric_precision_radix\030\014 \001(\005\022\025\n\rnumer" +
-      "ic_scale\030\r \001(\005\022\033\n\023date_time_precision\030\016 " +
-      "\001(\005\022\025\n\rinterval_type\030\017 \001(\t\022\032\n\022interval_p" +
-      "recision\030\020 \001(\005\022\023\n\013column_size\030\021 \001(\005\"\220\001\n\016" +
-      "GetColumnsResp\022(\n\006status\030\001 \001(\0162\030.exec.us",
-      "er.RequestStatus\022*\n\007columns\030\002 \003(\0132\031.exec" +
-      ".user.ColumnMetadata\022(\n\005error\030\003 \001(\0132\031.ex" +
-      "ec.shared.DrillPBError\"/\n\032CreatePrepared" +
-      "StatementReq\022\021\n\tsql_query\030\001 \001(\t\"\326\003\n\024Resu" +
-      "ltColumnMetadata\022\024\n\014catalog_name\030\001 \001(\t\022\023" +
-      "\n\013schema_name\030\002 \001(\t\022\022\n\ntable_name\030\003 \001(\t\022" +
-      "\023\n\013column_name\030\004 \001(\t\022\r\n\005label\030\005 \001(\t\022\021\n\td" +
-      "ata_type\030\006 \001(\t\022\023\n\013is_nullable\030\007 \001(\010\022\021\n\tp" +
-      "recision\030\010 \001(\005\022\r\n\005scale\030\t \001(\005\022\016\n\006signed\030" +
-      "\n \001(\010\022\024\n\014display_size\030\013 \001(\005\022\022\n\nis_aliase",
-      "d\030\014 \001(\010\0225\n\rsearchability\030\r \001(\0162\036.exec.us" +
-      "er.ColumnSearchability\0223\n\014updatability\030\016" +
-      " \001(\0162\035.exec.user.ColumnUpdatability\022\026\n\016a" +
-      "uto_increment\030\017 \001(\010\022\030\n\020case_sensitivity\030" +
-      "\020 \001(\010\022\020\n\010sortable\030\021 \001(\010\022\022\n\nclass_name\030\022 " +
-      "\001(\t\022\023\n\013is_currency\030\024 \001(\010\".\n\027PreparedStat" +
-      "ementHandle\022\023\n\013server_info\030\001 \001(\014\"\200\001\n\021Pre" +
-      "paredStatement\0220\n\007columns\030\001 \003(\0132\037.exec.u" +
-      "ser.ResultColumnMetadata\0229\n\rserver_handl" +
-      "e\030\002 \001(\0132\".exec.user.PreparedStatementHan",
-      "dle\"\253\001\n\033CreatePreparedStatementResp\022(\n\006s" +
-      "tatus\030\001 \001(\0162\030.exec.user.RequestStatus\0228\n" +
-      "\022prepared_statement\030\002 \001(\0132\034.exec.user.Pr" +
-      "eparedStatement\022(\n\005error\030\003 \001(\0132\031.exec.sh" +
-      "ared.DrillPBError\"\353\001\n\010RunQuery\0221\n\014result" +
-      "s_mode\030\001 \001(\0162\033.exec.user.QueryResultsMod" +
-      "e\022$\n\004type\030\002 \001(\0162\026.exec.shared.QueryType\022" +
-      "\014\n\004plan\030\003 \001(\t\0221\n\tfragments\030\004 \003(\0132\036.exec." +
-      "bit.control.PlanFragment\022E\n\031prepared_sta" +
-      "tement_handle\030\005 \001(\0132\".exec.user.Prepared",
-      "StatementHandle*\332\003\n\007RpcType\022\r\n\tHANDSHAKE" +
-      "\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\r\n\tRUN_QUERY\020\003\022" +
-      "\020\n\014CANCEL_QUERY\020\004\022\023\n\017REQUEST_RESULTS\020\005\022\027" +
-      "\n\023RESUME_PAUSED_QUERY\020\013\022\034\n\030GET_QUERY_PLA" +
-      "N_FRAGMENTS\020\014\022\020\n\014GET_CATALOGS\020\016\022\017\n\013GET_S" +
-      "CHEMAS\020\017\022\016\n\nGET_TABLES\020\020\022\017\n\013GET_COLUMNS\020" +
-      "\021\022\035\n\031CREATE_PREPARED_STATEMENT\020\026\022\016\n\nQUER" +
-      "Y_DATA\020\006\022\020\n\014QUERY_HANDLE\020\007\022\030\n\024QUERY_PLAN" +
-      "_FRAGMENTS\020\r\022\014\n\010CATALOGS\020\022\022\013\n\007SCHEMAS\020\023\022" +
-      "\n\n\006TABLES\020\024\022\013\n\007COLUMNS\020\025\022\026\n\022PREPARED_STA",
-      "TEMENT\020\027\022\026\n\022REQ_META_FUNCTIONS\020\010\022\026\n\022RESP" +
-      "_FUNCTION_LIST\020\t\022\020\n\014QUERY_RESULT\020\n\022\020\n\014SA" +
-      "SL_MESSAGE\020\030*6\n\013SaslSupport\022\030\n\024UNKNOWN_S" +
-      "ASL_SUPPORT\020\000\022\r\n\tSASL_AUTH\020\001*#\n\020QueryRes" +
-      "ultsMode\022\017\n\013STREAM_FULL\020\001*q\n\017HandshakeSt" +
-      "atus\022\013\n\007SUCCESS\020\001\022\030\n\024RPC_VERSION_MISMATC" +
-      "H\020\002\022\017\n\013AUTH_FAILED\020\003\022\023\n\017UNKNOWN_FAILURE\020" +
-      "\004\022\021\n\rAUTH_REQUIRED\020\005*D\n\rRequestStatus\022\022\n" +
-      "\016UNKNOWN_STATUS\020\000\022\006\n\002OK\020\001\022\n\n\006FAILED\020\002\022\013\n" +
-      "\007TIMEOUT\020\003*Y\n\023ColumnSearchability\022\031\n\025UNK",
-      "NOWN_SEARCHABILITY\020\000\022\010\n\004NONE\020\001\022\010\n\004CHAR\020\002" +
-      "\022\n\n\006NUMBER\020\003\022\007\n\003ALL\020\004*K\n\022ColumnUpdatabil" +
-      "ity\022\030\n\024UNKNOWN_UPDATABILITY\020\000\022\r\n\tREAD_ON" +
-      "LY\020\001\022\014\n\010WRITABLE\020\002B+\n\033org.apache.drill.e" +
-      "xec.protoB\nUserProtosH\001"
+      "r\030\002 \001(\0132\025.exec.user.LikeFilter\"i\n\016Schema" +
+      "Metadata\022\024\n\014catalog_name\030\001 \001(\t\022\023\n\013schema" +
+      "_name\030\002 \001(\t\022\r\n\005owner\030\003 \001(\t\022\014\n\004type\030\004 \001(\t",
+      "\022\017\n\007mutable\030\005 \001(\t\"\220\001\n\016GetSchemasResp\022(\n\006" +
+      "status\030\001 \001(\0162\030.exec.user.RequestStatus\022*" +
+      "\n\007schemas\030\002 \003(\0132\031.exec.user.SchemaMetada" +
+      "ta\022(\n\005error\030\003 \001(\0132\031.exec.shared.DrillPBE" +
+      "rror\"\302\001\n\014GetTablesReq\0222\n\023catalog_name_fi" +
+      "lter\030\001 \001(\0132\025.exec.user.LikeFilter\0221\n\022sch" +
+      "ema_name_filter\030\002 \001(\0132\025.exec.user.LikeFi" +
+      "lter\0220\n\021table_name_filter\030\003 \001(\0132\025.exec.u" +
+      "ser.LikeFilter\022\031\n\021table_type_filter\030\004 \003(" +
+      "\t\"\\\n\rTableMetadata\022\024\n\014catalog_name\030\001 \001(\t",
+      "\022\023\n\013schema_name\030\002 \001(\t\022\022\n\ntable_name\030\003 \001(" +
+      "\t\022\014\n\004type\030\004 \001(\t\"\215\001\n\rGetTablesResp\022(\n\006sta" +
+      "tus\030\001 \001(\0162\030.exec.user.RequestStatus\022(\n\006t" +
+      "ables\030\002 \003(\0132\030.exec.user.TableMetadata\022(\n" +
+      "\005error\030\003 \001(\0132\031.exec.shared.DrillPBError\"" +
+      "\333\001\n\rGetColumnsReq\0222\n\023catalog_name_filter" +
+      "\030\001 \001(\0132\025.exec.user.LikeFilter\0221\n\022schema_" +
+      "name_filter\030\002 \001(\0132\025.exec.user.LikeFilter" +
+      "\0220\n\021table_name_filter\030\003 \001(\0132\025.exec.user." +
+      "LikeFilter\0221\n\022column_name_filter\030\004 \001(\0132\025",
+      ".exec.user.LikeFilter\"\251\003\n\016ColumnMetadata" +
+      "\022\024\n\014catalog_name\030\001 \001(\t\022\023\n\013schema_name\030\002 " +
+      "\001(\t\022\022\n\ntable_name\030\003 \001(\t\022\023\n\013column_name\030\004" +
+      " \001(\t\022\030\n\020ordinal_position\030\005 \001(\005\022\025\n\rdefaul" +
+      "t_value\030\006 \001(\t\022\023\n\013is_nullable\030\007 \001(\010\022\021\n\tda" +
+      "ta_type\030\010 \001(\t\022\027\n\017char_max_length\030\t \001(\005\022\031" +
+      "\n\021char_octet_length\030\n \001(\005\022\031\n\021numeric_pre" +
+      "cision\030\013 \001(\005\022\037\n\027numeric_precision_radix\030" +
+      "\014 \001(\005\022\025\n\rnumeric_scale\030\r \001(\005\022\033\n\023date_tim" +
+      "e_precision\030\016 \001(\005\022\025\n\rinterval_type\030\017 \001(\t",
+      "\022\032\n\022interval_precision\030\020 \001(\005\022\023\n\013column_s" +
+      "ize\030\021 \001(\005\"\220\001\n\016GetColumnsResp\022(\n\006status\030\001" +
+      " \001(\0162\030.exec.user.RequestStatus\022*\n\007column" +
+      "s\030\002 \003(\0132\031.exec.user.ColumnMetadata\022(\n\005er" +
+      "ror\030\003 \001(\0132\031.exec.shared.DrillPBError\"/\n\032" +
+      "CreatePreparedStatementReq\022\021\n\tsql_query\030" +
+      "\001 \001(\t\"\326\003\n\024ResultColumnMetadata\022\024\n\014catalo" +
+      "g_name\030\001 \001(\t\022\023\n\013schema_name\030\002 \001(\t\022\022\n\ntab" +
+      "le_name\030\003 \001(\t\022\023\n\013column_name\030\004 \001(\t\022\r\n\005la" +
+      "bel\030\005 \001(\t\022\021\n\tdata_type\030\006 \001(\t\022\023\n\013is_nulla",
+      "ble\030\007 \001(\010\022\021\n\tprecision\030\010 \001(\005\022\r\n\005scale\030\t " +
+      "\001(\005\022\016\n\006signed\030\n \001(\010\022\024\n\014display_size\030\013 \001(" +
+      "\005\022\022\n\nis_aliased\030\014 \001(\010\0225\n\rsearchability\030\r" +
+      " \001(\0162\036.exec.user.ColumnSearchability\0223\n\014" +
+      "updatability\030\016 \001(\0162\035.exec.user.ColumnUpd" +
+      "atability\022\026\n\016auto_increment\030\017 \001(\010\022\030\n\020cas" +
+      "e_sensitivity\030\020 \001(\010\022\020\n\010sortable\030\021 \001(\010\022\022\n" +
+      "\nclass_name\030\022 \001(\t\022\023\n\013is_currency\030\024 \001(\010\"." +
+      "\n\027PreparedStatementHandle\022\023\n\013server_info" +
+      "\030\001 \001(\014\"\200\001\n\021PreparedStatement\0220\n\007columns\030",
+      "\001 \003(\0132\037.exec.user.ResultColumnMetadata\0229" +
+      "\n\rserver_handle\030\002 \001(\0132\".exec.user.Prepar" +
+      "edStatementHandle\"\253\001\n\033CreatePreparedStat" +
+      "ementResp\022(\n\006status\030\001 \001(\0162\030.exec.user.Re" +
+      "questStatus\0228\n\022prepared_statement\030\002 \001(\0132" +
+      "\034.exec.user.PreparedStatement\022(\n\005error\030\003" +
+      " \001(\0132\031.exec.shared.DrillPBError\"\353\001\n\010RunQ" +
+      "uery\0221\n\014results_mode\030\001 \001(\0162\033.exec.user.Q" +
+      "ueryResultsMode\022$\n\004type\030\002 \001(\0162\026.exec.sha" +
+      "red.QueryType\022\014\n\004plan\030\003 \001(\t\0221\n\tfragments",
+      "\030\004 \003(\0132\036.exec.bit.control.PlanFragment\022E" +
+      "\n\031prepared_statement_handle\030\005 \001(\0132\".exec" +
+      ".user.PreparedStatementHandle*\332\003\n\007RpcTyp" +
+      "e\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\r" +
+      "\n\tRUN_QUERY\020\003\022\020\n\014CANCEL_QUERY\020\004\022\023\n\017REQUE" +
+      "ST_RESULTS\020\005\022\027\n\023RESUME_PAUSED_QUERY\020\013\022\034\n" +
+      "\030GET_QUERY_PLAN_FRAGMENTS\020\014\022\020\n\014GET_CATAL" +
+      "OGS\020\016\022\017\n\013GET_SCHEMAS\020\017\022\016\n\nGET_TABLES\020\020\022\017" +
+      "\n\013GET_COLUMNS\020\021\022\035\n\031CREATE_PREPARED_STATE" +
+      "MENT\020\026\022\016\n\nQUERY_DATA\020\006\022\020\n\014QUERY_HANDLE\020\007",
+      "\022\030\n\024QUERY_PLAN_FRAGMENTS\020\r\022\014\n\010CATALOGS\020\022" +
+      "\022\013\n\007SCHEMAS\020\023\022\n\n\006TABLES\020\024\022\013\n\007COLUMNS\020\025\022\026" +
+      "\n\022PREPARED_STATEMENT\020\027\022\026\n\022REQ_META_FUNCT" +
+      "IONS\020\010\022\026\n\022RESP_FUNCTION_LIST\020\t\022\020\n\014QUERY_" +
+      "RESULT\020\n\022\020\n\014SASL_MESSAGE\020\030*6\n\013SaslSuppor" +
+      "t\022\030\n\024UNKNOWN_SASL_SUPPORT\020\000\022\r\n\tSASL_AUTH" +
+      "\020\001*#\n\020QueryResultsMode\022\017\n\013STREAM_FULL\020\001*" +
+      "q\n\017HandshakeStatus\022\013\n\007SUCCESS\020\001\022\030\n\024RPC_V" +
+      "ERSION_MISMATCH\020\002\022\017\n\013AUTH_FAILED\020\003\022\023\n\017UN" +
+      "KNOWN_FAILURE\020\004\022\021\n\rAUTH_REQUIRED\020\005*D\n\rRe",
+      "questStatus\022\022\n\016UNKNOWN_STATUS\020\000\022\006\n\002OK\020\001\022" +
+      "\n\n\006FAILED\020\002\022\013\n\007TIMEOUT\020\003*Y\n\023ColumnSearch" +
+      "ability\022\031\n\025UNKNOWN_SEARCHABILITY\020\000\022\010\n\004NO" +
+      "NE\020\001\022\010\n\004CHAR\020\002\022\n\n\006NUMBER\020\003\022\007\n\003ALL\020\004*K\n\022C" +
+      "olumnUpdatability\022\030\n\024UNKNOWN_UPDATABILIT" +
+      "Y\020\000\022\r\n\tREAD_ONLY\020\001\022\014\n\010WRITABLE\020\002B+\n\033org." +
+      "apache.drill.exec.protoB\nUserProtosH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -29731,7 +30224,7 @@ public final class UserProtos {
           internal_static_exec_user_RpcEndpointInfos_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_user_RpcEndpointInfos_descriptor,
-              new java.lang.String[] { "Name", "Version", "MajorVersion", "MinorVersion", "PatchVersion", "Application", });
+              new java.lang.String[] { "Name", "Version", "MajorVersion", "MinorVersion", "PatchVersion", "Application", "BuildNumber", "VersionQualifier", });
           internal_static_exec_user_UserToBitHandshake_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_exec_user_UserToBitHandshake_fieldAccessorTable = new
@@ -29761,7 +30254,7 @@ public final class UserProtos {
           internal_static_exec_user_BitToUserHandshake_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_user_BitToUserHandshake_descriptor,
-              new java.lang.String[] { "RpcVersion", "Status", "ErrorId", "ErrorMessage", "ServerInfos", "AuthenticationMechanisms", });
+              new java.lang.String[] { "RpcVersion", "Status", "ErrorId", "ErrorMessage", "ServerInfos", "AuthenticationMechanisms", "SupportedMethods", });
           internal_static_exec_user_LikeFilter_descriptor =
             getDescriptor().getMessageTypes().get(8);
           internal_static_exec_user_LikeFilter_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/protocol/src/main/java/org/apache/drill/exec/proto/beans/BitToUserHandshake.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/BitToUserHandshake.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/BitToUserHandshake.java
index 0025049..0cad1c0 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/BitToUserHandshake.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/BitToUserHandshake.java
@@ -55,6 +55,7 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
     private String errorMessage;
     private RpcEndpointInfos serverInfos;
     private List<String> authenticationMechanisms;
+    private List<RpcType> supportedMethods;
 
     public BitToUserHandshake()
     {
@@ -141,6 +142,19 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
         return this;
     }
 
+    // supportedMethods
+
+    public List<RpcType> getSupportedMethodsList()
+    {
+        return supportedMethods;
+    }
+
+    public BitToUserHandshake setSupportedMethodsList(List<RpcType> supportedMethods)
+    {
+        this.supportedMethods = supportedMethods;
+        return this;
+    }
+
     // java serialization
 
     public void readExternal(ObjectInput in) throws IOException
@@ -216,6 +230,11 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
                         message.authenticationMechanisms = new ArrayList<String>();
                     message.authenticationMechanisms.add(input.readString());
                     break;
+                case 8:
+                    if(message.supportedMethods == null)
+                        message.supportedMethods = new ArrayList<RpcType>();
+                    message.supportedMethods.add(RpcType.valueOf(input.readEnum()));
+                    break;
                 default:
                     input.handleUnknownField(number, this);
             }   
@@ -249,6 +268,15 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
                     output.writeString(7, authenticationMechanisms, true);
             }
         }
+
+        if(message.supportedMethods != null)
+        {
+            for(RpcType supportedMethods : message.supportedMethods)
+            {
+                if(supportedMethods != null)
+                    output.writeEnum(8, supportedMethods.number, true);
+            }
+        }
     }
 
     public String getFieldName(int number)
@@ -261,6 +289,7 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
             case 5: return "errorMessage";
             case 6: return "serverInfos";
             case 7: return "authenticationMechanisms";
+            case 8: return "supportedMethods";
             default: return null;
         }
     }
@@ -280,6 +309,7 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
         __fieldMap.put("errorMessage", 5);
         __fieldMap.put("serverInfos", 6);
         __fieldMap.put("authenticationMechanisms", 7);
+        __fieldMap.put("supportedMethods", 8);
     }
     
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/protocol/src/main/java/org/apache/drill/exec/proto/beans/RpcEndpointInfos.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/RpcEndpointInfos.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/RpcEndpointInfos.java
index 9849e56..8a8ffa7 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/RpcEndpointInfos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/RpcEndpointInfos.java
@@ -53,6 +53,8 @@ public final class RpcEndpointInfos implements Externalizable, Message<RpcEndpoi
     private int minorVersion;
     private int patchVersion;
     private String application;
+    private int buildNumber;
+    private String versionQualifier;
 
     public RpcEndpointInfos()
     {
@@ -139,6 +141,32 @@ public final class RpcEndpointInfos implements Externalizable, Message<RpcEndpoi
         return this;
     }
 
+    // buildNumber
+
+    public int getBuildNumber()
+    {
+        return buildNumber;
+    }
+
+    public RpcEndpointInfos setBuildNumber(int buildNumber)
+    {
+        this.buildNumber = buildNumber;
+        return this;
+    }
+
+    // versionQualifier
+
+    public String getVersionQualifier()
+    {
+        return versionQualifier;
+    }
+
+    public RpcEndpointInfos setVersionQualifier(String versionQualifier)
+    {
+        this.versionQualifier = versionQualifier;
+        return this;
+    }
+
     // java serialization
 
     public void readExternal(ObjectInput in) throws IOException
@@ -211,6 +239,12 @@ public final class RpcEndpointInfos implements Externalizable, Message<RpcEndpoi
                 case 6:
                     message.application = input.readString();
                     break;
+                case 7:
+                    message.buildNumber = input.readUInt32();
+                    break;
+                case 8:
+                    message.versionQualifier = input.readString();
+                    break;
                 default:
                     input.handleUnknownField(number, this);
             }   
@@ -237,6 +271,12 @@ public final class RpcEndpointInfos implements Externalizable, Message<RpcEndpoi
 
         if(message.application != null)
             output.writeString(6, message.application, false);
+
+        if(message.buildNumber != 0)
+            output.writeUInt32(7, message.buildNumber, false);
+
+        if(message.versionQualifier != null)
+            output.writeString(8, message.versionQualifier, false);
     }
 
     public String getFieldName(int number)
@@ -249,6 +289,8 @@ public final class RpcEndpointInfos implements Externalizable, Message<RpcEndpoi
             case 4: return "minorVersion";
             case 5: return "patchVersion";
             case 6: return "application";
+            case 7: return "buildNumber";
+            case 8: return "versionQualifier";
             default: return null;
         }
     }
@@ -268,6 +310,8 @@ public final class RpcEndpointInfos implements Externalizable, Message<RpcEndpoi
         __fieldMap.put("minorVersion", 4);
         __fieldMap.put("patchVersion", 5);
         __fieldMap.put("application", 6);
+        __fieldMap.put("buildNumber", 7);
+        __fieldMap.put("versionQualifier", 8);
     }
     
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/16aa0810/protocol/src/main/protobuf/User.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/User.proto b/protocol/src/main/protobuf/User.proto
index 971248e..ef993cc 100644
--- a/protocol/src/main/protobuf/User.proto
+++ b/protocol/src/main/protobuf/User.proto
@@ -57,12 +57,14 @@ message UserProperties {
 }
 
 message RpcEndpointInfos {
-    optional string name = 1;           // example: Apache Drill Server, Apache Drill C++ client
-    optional string version = 2;        // example: 1.9.0
-    optional uint32 majorVersion = 3;   // example: 1
-    optional uint32 minorVersion = 4;   // example: 9
-    optional uint32 patchVersion = 5;   // example: 0
-    optional string application = 6;    // example: Tableau 9.3
+    optional string name = 1;             // example: Apache Drill Server, Apache Drill C++ client
+    optional string version = 2;          // example: 1.9.0
+    optional uint32 majorVersion = 3;     // example: 1
+    optional uint32 minorVersion = 4;     // example: 9
+    optional uint32 patchVersion = 5;     // example: 0
+    optional string application = 6;      // example: Tableau 9.3
+    optional uint32 buildNumber = 7;      // example: 32
+    optional string versionQualifier = 8; // example: SNAPSHOT
 }
 
 enum SaslSupport {
@@ -121,6 +123,7 @@ message BitToUserHandshake {
   optional string errorMessage = 5;
   optional RpcEndpointInfos server_infos = 6;
   repeated string authenticationMechanisms = 7;
+  repeated RpcType supported_methods = 8;
 }
 
 /*