You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2022/02/22 14:30:17 UTC

[hive] branch branch-3 updated: HIVE-25844: Exception deserialization error-s may cause beeline to terminate immediately (backport HIVE-24772) (#2918) (Zoltan Haindrich reviewed by Krisztian Kasa)

This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch branch-3
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/branch-3 by this push:
     new 1087c4e  HIVE-25844: Exception deserialization error-s may cause beeline to terminate immediately (backport HIVE-24772) (#2918) (Zoltan Haindrich reviewed by Krisztian Kasa)
1087c4e is described below

commit 1087c4e14a2668eab914939f1f92dae2f049cf7e
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Tue Feb 22 15:29:52 2022 +0100

    HIVE-25844: Exception deserialization error-s may cause beeline to terminate immediately (backport HIVE-24772) (#2918) (Zoltan Haindrich reviewed by Krisztian Kasa)
---
 .../apache/hive/minikdc/TestJdbcWithMiniKdc.java   |  28 +---
 .../apache/hive/service/cli/HiveSQLException.java  | 167 +++++----------------
 .../hive/service/cli/thrift/ThriftCLIService.java  |  67 +++++----
 .../hive/service/cli/TestHiveSQLException.java     | 112 +-------------
 4 files changed, 79 insertions(+), 295 deletions(-)

diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdc.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdc.java
index e526b48..353d157 100644
--- a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdc.java
+++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdc.java
@@ -221,22 +221,15 @@ public class TestJdbcWithMiniKdc {
    * it's not allowed to impersonate
    * @throws Exception
    */
-  @Test
+  @Test(expected = HiveSQLException.class)
   public void testNegativeTokenAuth() throws Exception {
     miniHiveKdc.loginUser(MiniHiveKdc.HIVE_TEST_SUPER_USER);
     hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL());
 
     try {
       // retrieve token and store in the cache
-      String token = ((HiveConnection)hs2Conn).getDelegationToken(
+      ((HiveConnection)hs2Conn).getDelegationToken(
           MiniHiveKdc.HIVE_TEST_USER_2, MiniHiveKdc.HIVE_SERVICE_PRINCIPAL);
-
-      fail(MiniHiveKdc.HIVE_TEST_SUPER_USER + " shouldn't be allowed to retrieve token for " +
-          MiniHiveKdc.HIVE_TEST_USER_2);
-    } catch (SQLException e) {
-      // Expected error
-      assertEquals("Unexpected type of exception class thrown", HiveSQLException.class, e.getClass());
-      assertTrue(e.getCause().getCause().getMessage().contains("is not allowed to impersonate"));
     } finally {
       hs2Conn.close();
     }
@@ -260,22 +253,11 @@ public class TestJdbcWithMiniKdc {
    * impersonate the given user
    * @throws Exception
    */
-  @Test
+  @Test(expected = SQLException.class)
   public void testNegativeProxyAuth() throws Exception {
     miniHiveKdc.loginUser(MiniHiveKdc.HIVE_TEST_SUPER_USER);
-    try {
-      hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL("default",
-          ";hive.server2.proxy.user=" + MiniHiveKdc.HIVE_TEST_USER_2));
-      verifyProperty(SESSION_USER_NAME, MiniHiveKdc.HIVE_TEST_USER_2);
-      fail(MiniHiveKdc.HIVE_TEST_SUPER_USER + " shouldn't be allowed proxy connection for "
-          + MiniHiveKdc.HIVE_TEST_USER_2);
-    } catch (SQLException e) {
-      // Expected error
-      e.printStackTrace();
-      assertTrue(e.getMessage().contains("Failed to validate proxy privilege"));
-      assertTrue(e.getCause().getCause().getCause().getMessage()
-              .contains("is not allowed to impersonate"));
-    }
+    hs2Conn = DriverManager
+        .getConnection(miniHS2.getJdbcURL("default", ";hive.server2.proxy.user=" + MiniHiveKdc.HIVE_TEST_USER_2));
   }
 
   /**
diff --git a/service/src/java/org/apache/hive/service/cli/HiveSQLException.java b/service/src/java/org/apache/hive/service/cli/HiveSQLException.java
index 5f9ff43..f180830 100644
--- a/service/src/java/org/apache/hive/service/cli/HiveSQLException.java
+++ b/service/src/java/org/apache/hive/service/cli/HiveSQLException.java
@@ -19,39 +19,52 @@
 package org.apache.hive.service.cli;
 
 import java.sql.SQLException;
-import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.hive.service.rpc.thrift.TStatus;
 import org.apache.hive.service.rpc.thrift.TStatusCode;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
- * HiveSQLException.
- *
+ * An exception that provides information on a Hive access error or other
+ * errors.
  */
 public class HiveSQLException extends SQLException {
 
-  /**
-   *
-   */
-  private static final long serialVersionUID = -6095254671958748094L;
+  private static final long serialVersionUID = -6095254671958748095L;
+
+  @VisibleForTesting
+  public static final List<String> DEFAULT_INFO =
+      Collections.singletonList("Server-side error; please check HS2 logs.");
 
   /**
-   *
+   * Constructor.
    */
   public HiveSQLException() {
     super();
   }
 
   /**
-   * @param reason
+   * Constructs a SQLException object with a given reason. The SQLState is
+   * initialized to null and the vendor code is initialized to 0. The cause is
+   * not initialized, and may subsequently be initialized by a call to the
+   * Throwable.initCause(java.lang.Throwable) method.
+   *
+   * @param reason a description of the exception
    */
   public HiveSQLException(String reason) {
     super(reason);
   }
 
   /**
-   * @param cause
+   * Constructs a SQLException object with a given cause. The SQLState is
+   * initialized to null and the vendor code is initialized to 0. The reason is
+   * initialized to null if cause==null or to cause.toString() if cause!=null.
+   *
+   * @param cause the underlying reason for this SQLException - may be null
+   *          indicating the cause is non-existent or unknown
    */
   public HiveSQLException(Throwable cause) {
     super(cause);
@@ -102,18 +115,13 @@ public class HiveSQLException extends SQLException {
   }
 
   public HiveSQLException(TStatus status) {
-    // TODO: set correct vendorCode field
     super(status.getErrorMessage(), status.getSqlState(), status.getErrorCode());
-    if (status.getInfoMessages() != null) {
-      initCause(toCause(status.getInfoMessages()));
-    }
   }
 
-
-
-/**
-   * Converts current object to a {@link TStatus} object
-   * @return	a {@link TStatus} object
+  /**
+   * Converts current object to a {@link TStatus} object.
+   *
+   * @return a {@link TStatus} object
    */
   public TStatus toTStatus() {
     // TODO: convert sqlState, etc.
@@ -121,130 +129,25 @@ public class HiveSQLException extends SQLException {
     tStatus.setSqlState(getSQLState());
     tStatus.setErrorCode(getErrorCode());
     tStatus.setErrorMessage(getMessage());
-    tStatus.setInfoMessages(toString(this));
+    tStatus.setInfoMessages(DEFAULT_INFO);
     return tStatus;
   }
 
   /**
-   * Converts the specified {@link Exception} object into a {@link TStatus} object
-   * @param e	a {@link Exception} object
-   * @return	a {@link TStatus} object
+   * Converts the specified {@link Exception} object into a {@link TStatus}
+   * object.
+   *
+   * @param e a {@link Exception} object
+   * @return a {@link TStatus} object
    */
   public static TStatus toTStatus(Exception e) {
     if (e instanceof HiveSQLException) {
-      return ((HiveSQLException)e).toTStatus();
+      return ((HiveSQLException) e).toTStatus();
     }
     TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS);
     tStatus.setErrorMessage(e.getMessage());
-    tStatus.setInfoMessages(toString(e));
+    tStatus.setInfoMessages(DEFAULT_INFO);
     return tStatus;
   }
 
-  /**
-   * Converts a {@link Throwable} object into a flattened list of texts including its stack trace
-   * and the stack traces of the nested causes.
-   * @param ex  a {@link Throwable} object
-   * @return    a flattened list of texts including the {@link Throwable} object's stack trace
-   *            and the stack traces of the nested causes.
-   */
-  public static List<String> toString(Throwable ex) {
-    return toString(ex, null);
-  }
-
-  private static List<String> toString(Throwable cause, StackTraceElement[] parent) {
-    StackTraceElement[] trace = cause.getStackTrace();
-    int m = trace.length - 1;
-    if (parent != null) {
-      int n = parent.length - 1;
-      while (m >= 0 && n >= 0 && trace[m].equals(parent[n])) {
-        m--; n--;
-      }
-    }
-    List<String> detail = enroll(cause, trace, m);
-    cause = cause.getCause();
-    if (cause != null) {
-      detail.addAll(toString(cause, trace));
-    }
-    return detail;
-  }
-
-  private static List<String> enroll(Throwable ex, StackTraceElement[] trace, int max) {
-    List<String> details = new ArrayList<String>();
-    StringBuilder builder = new StringBuilder();
-    builder.append('*').append(ex.getClass().getName()).append(':');
-    builder.append(ex.getMessage()).append(':');
-    builder.append(trace.length).append(':').append(max);
-    details.add(builder.toString());
-    for (int i = 0; i <= max; i++) {
-      builder.setLength(0);
-      builder.append(trace[i].getClassName()).append(':');
-      builder.append(trace[i].getMethodName()).append(':');
-      String fileName = trace[i].getFileName();
-      builder.append(fileName == null ? "" : fileName).append(':');
-      builder.append(trace[i].getLineNumber());
-      details.add(builder.toString());
-    }
-    return details;
-  }
-
-  /**
-   * Converts a flattened list of texts including the stack trace and the stack
-   * traces of the nested causes into a {@link Throwable} object.
-   * @param details a flattened list of texts including the stack trace and the stack
-   *                traces of the nested causes
-   * @return        a {@link Throwable} object
-   */
-  public static Throwable toCause(List<String> details) {
-    return toStackTrace(details, null, 0);
-  }
-
-  private static Throwable toStackTrace(List<String> details, StackTraceElement[] parent, int index) {
-    String detail = details.get(index++);
-    if (!detail.startsWith("*")) {
-      return null;  // should not be happened. ignore remaining
-    }
-    int i1 = detail.indexOf(':');
-    int i3 = detail.lastIndexOf(':');
-    int i2 = detail.substring(0, i3).lastIndexOf(':');
-    String exceptionClass = detail.substring(1, i1);
-    String exceptionMessage = detail.substring(i1 + 1, i2);
-    Throwable ex = newInstance(exceptionClass, exceptionMessage);
-
-    int length = Integer.parseInt(detail.substring(i2 + 1, i3));
-    int unique = Integer.parseInt(detail.substring(i3 + 1));
-
-    int i = 0;
-    StackTraceElement[] trace = new StackTraceElement[length];
-    for (; i <= unique; i++) {
-      detail = details.get(index++);
-      int j1 = detail.indexOf(':');
-      int j3 = detail.lastIndexOf(':');
-      int j2 = detail.substring(0, j3).lastIndexOf(':');
-      String className = detail.substring(0, j1);
-      String methodName = detail.substring(j1 + 1, j2);
-      String fileName = detail.substring(j2 + 1, j3);
-      if (fileName.isEmpty()) {
-        fileName = null;
-      }
-      int lineNumber = Integer.parseInt(detail.substring(j3 + 1));
-      trace[i] = new StackTraceElement(className, methodName, fileName, lineNumber);
-    }
-    int common = trace.length - i;
-    if (common > 0) {
-      System.arraycopy(parent, parent.length - common, trace, trace.length - common, common);
-    }
-    if (details.size() > index) {
-      ex.initCause(toStackTrace(details, trace, index));
-    }
-    ex.setStackTrace(trace);
-    return ex;
-  }
-
-  private static Throwable newInstance(String className, String message) {
-    try {
-      return (Throwable)Class.forName(className).getConstructor(String.class).newInstance(message);
-    } catch (Exception e) {
-      return new RuntimeException(className + ":" + message);
-    }
-  }
 }
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
index 5481b90..9dbf494 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
@@ -124,7 +124,7 @@ import org.slf4j.LoggerFactory;
  */
 public abstract class ThriftCLIService extends AbstractService implements TCLIService.Iface, Runnable {
 
-  public static final Logger LOG = LoggerFactory.getLogger(ThriftCLIService.class.getName());
+  protected static final Logger LOG = LoggerFactory.getLogger(ThriftCLIService.class);
 
   protected CLIService cliService;
   private static final TStatus OK_STATUS = new TStatus(TStatusCode.SUCCESS_STATUS);
@@ -259,7 +259,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
         resp.setDelegationToken(token);
         resp.setStatus(OK_STATUS);
       } catch (HiveSQLException e) {
-        LOG.error("Error obtaining delegation token", e);
+        LOG.error("Failed to get delegation token [request: {}]", req, e);
         TStatus tokenErrorStatus = HiveSQLException.toTStatus(e);
         tokenErrorStatus.setSqlState("42000");
         resp.setStatus(tokenErrorStatus);
@@ -281,7 +281,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
             hiveAuthFactory, req.getDelegationToken());
         resp.setStatus(OK_STATUS);
       } catch (HiveSQLException e) {
-        LOG.error("Error canceling delegation token", e);
+        LOG.error("Failed to cancel delegation token [request: {}]", req, e);
         resp.setStatus(HiveSQLException.toTStatus(e));
       }
     }
@@ -300,7 +300,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
             hiveAuthFactory, req.getDelegationToken());
         resp.setStatus(OK_STATUS);
       } catch (HiveSQLException e) {
-        LOG.error("Error obtaining renewing token", e);
+        LOG.error("Failed to renew delegation token [request: {}]", e);
         resp.setStatus(HiveSQLException.toTStatus(e));
       }
     }
@@ -318,7 +318,9 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
   public TOpenSessionResp OpenSession(TOpenSessionReq req) throws TException {
     LOG.info("Client protocol version: " + req.getClient_protocol());
     TOpenSessionResp resp = new TOpenSessionResp();
+    String userName = null;
     try {
+      userName = getUserName(req);
       SessionHandle sessionHandle = getSessionHandle(req, resp);
       resp.setSessionHandle(sessionHandle.toTSessionHandle());
       Map<String, String> configurationMap = new HashMap<String, String>();
@@ -337,7 +339,8 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
         context.setSessionHandle(sessionHandle);
       }
     } catch (Exception e) {
-      LOG.warn("Error opening session: ", e);
+      // Do not log request as it contains password information
+      LOG.error("Login attempt failed for user : {}", userName, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -363,7 +366,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
           try {
             cliService.setApplicationName(sh, e.getValue());
           } catch (Exception ex) {
-            LOG.warn("Error setting application name", ex);
+            LOG.error("Failed setting application name", ex);
             resp = new TSetClientInfoResp(HiveSQLException.toTStatus(ex));
           }
         }
@@ -392,7 +395,6 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
         clientIpAddress = TSetIpAddressProcessor.getUserIpAddress();
       }
     }
-    LOG.debug("Client's IP Address: " + clientIpAddress);
     return clientIpAddress;
   }
 
@@ -428,7 +430,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
 
     userName = getShortName(userName);
     String effectiveClientUser = getProxyUser(userName, req.getConfiguration(), getIpAddress());
-    LOG.debug("Client's username: " + effectiveClientUser);
+    LOG.debug("Client's username: {}", effectiveClientUser);
     return effectiveClientUser;
   }
 
@@ -463,6 +465,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       throws HiveSQLException, LoginException, IOException {
     String userName = getUserName(req);
     String ipAddress = getIpAddress();
+    LOG.info("Creating Hive session handle for user [{}] from IP {}", userName, ipAddress);
     TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION,
         req.getClient_protocol());
     SessionHandle sessionHandle;
@@ -480,7 +483,8 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
   }
 
   private double getProgressedPercentage(OperationHandle opHandle) throws HiveSQLException {
-    checkArgument(OperationType.EXECUTE_STATEMENT.equals(opHandle.getOperationType()));
+    checkArgument(OperationType.EXECUTE_STATEMENT == opHandle.getOperationType());
+
     Operation operation = cliService.getSessionManager().getOperationManager().getOperation(opHandle);
     SessionState state = operation.getParentSession().getSessionState();
     ProgressMonitor monitor = state.getProgressMonitor();
@@ -527,7 +531,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
         context.setSessionHandle(null);
       }
     } catch (Exception e) {
-      LOG.warn("Error closing session: ", e);
+      LOG.error("Failed to close the session", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -543,7 +547,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       resp.setInfoValue(getInfoValue.toTGetInfoValue());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting info: ", e);
+      LOG.error("Failed to get info", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -568,7 +572,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       // Note: it's rather important that this (and other methods) catch Exception, not Throwable;
       // in combination with HiveSessionProxy.invoke code, perhaps unintentionally, it used
       // to also catch all errors; and now it allows OOMs only to propagate.
-      LOG.warn("Error executing statement: ", e);
+      LOG.error("Failed to execute statement [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -582,7 +586,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       resp.setOperationHandle(operationHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting type info: ", e);
+      LOG.error("Failed to get type info [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -596,7 +600,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting catalogs: ", e);
+      LOG.error("Failed getting catalogs [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -611,7 +615,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting schemas: ", e);
+      LOG.error("Failed to get schemas [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -627,7 +631,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting tables: ", e);
+      LOG.error("Failed to get tables [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -641,7 +645,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting table types: ", e);
+      LOG.error("Failed to get table types [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -660,7 +664,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting columns: ", e);
+      LOG.error("Failed to get column types [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -676,7 +680,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting functions: ", e);
+      LOG.error("Failed to get function: [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -726,7 +730,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       }
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting operation status: ", e);
+      LOG.error("Failed to get operation status [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -739,7 +743,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       cliService.cancelOperation(new OperationHandle(req.getOperationHandle()));
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error cancelling operation: ", e);
+      LOG.error("Failed to get cancel operation [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -752,7 +756,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       cliService.closeOperation(new OperationHandle(req.getOperationHandle()));
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error closing operation: ", e);
+      LOG.error("Failed to close operation [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -767,7 +771,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       resp.setSchema(schema.toTTableSchema());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting result set metadata: ", e);
+      LOG.error("Failed to get result set metadata [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -792,7 +796,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       resp.setHasMoreRows(false);
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error fetching results: ", e);
+      LOG.error("Failed fetch results [request: {}]", req, e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -809,7 +813,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-     LOG.warn("Error getting functions: ", e);
+     LOG.error("Failed to get primary keys [request: {}]", req, e);
      resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -827,9 +831,9 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
           resp.setOperationHandle(opHandle.toTOperationHandle());
           resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting functions: ", e);
-	  resp.setStatus(HiveSQLException.toTStatus(e));
-	}
+      LOG.error("Failed to get cross reference [request: {}]", req, e);
+      resp.setStatus(HiveSQLException.toTStatus(e));
+    }
     return resp;
   }
 
@@ -838,6 +842,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
     try {
       return new TGetQueryIdResp(cliService.getQueryId(req.getOperationHandle()));
     } catch (HiveSQLException e) {
+      LOG.error("Failed to get query ID [request: {}]", req, e);
       throw new TException(e);
     }
   }
@@ -861,12 +866,12 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
     if (cliService.getHiveConf().getVar(
         ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) {
       proxyUser = SessionManager.getProxyUserName();
-      LOG.debug("Proxy user from query string: " + proxyUser);
+      LOG.debug("Proxy user from query string: {}", proxyUser);
     }
 
     if (proxyUser == null && sessionConf != null && sessionConf.containsKey(HiveAuthConstants.HS2_PROXY_USER)) {
       String proxyUserFromThriftBody = sessionConf.get(HiveAuthConstants.HS2_PROXY_USER);
-      LOG.debug("Proxy user from thrift body: " + proxyUserFromThriftBody);
+      LOG.debug("Proxy user from thrift body: {}", proxyUserFromThriftBody);
       proxyUser = proxyUserFromThriftBody;
     }
 
@@ -887,7 +892,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
 
     // Verify proxy user privilege of the realUser for the proxyUser
     HiveAuthFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hiveConf);
-    LOG.debug("Verified proxy user: " + proxyUser);
+    LOG.debug("Verified proxy user: {}", proxyUser);
     return proxyUser;
   }
 }
diff --git a/service/src/test/org/apache/hive/service/cli/TestHiveSQLException.java b/service/src/test/org/apache/hive/service/cli/TestHiveSQLException.java
index ac7fec0..7e774ad 100644
--- a/service/src/test/org/apache/hive/service/cli/TestHiveSQLException.java
+++ b/service/src/test/org/apache/hive/service/cli/TestHiveSQLException.java
@@ -20,11 +20,10 @@ package org.apache.hive.service.cli;
 
 import java.util.List;
 
-import junit.framework.Assert;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.hive.service.rpc.thrift.TStatus;
 import org.apache.hive.service.rpc.thrift.TStatusCode;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class TestHiveSQLException {
@@ -40,7 +39,7 @@ public class TestHiveSQLException {
 
     Assert.assertEquals(TStatusCode.ERROR_STATUS, status.getStatusCode());
     Assert.assertEquals(ex1.getMessage(), status.getErrorMessage());
-    Assert.assertEquals(HiveSQLException.toString(ex1), status.getInfoMessages());
+    Assert.assertEquals(HiveSQLException.DEFAULT_INFO, status.getInfoMessages());
   }
 
   /**
@@ -58,109 +57,7 @@ public class TestHiveSQLException {
     Assert.assertEquals(TStatusCode.ERROR_STATUS, status.getStatusCode());
     Assert.assertEquals(expectedSqlState, status.getSqlState());
     Assert.assertEquals(expectedMessage, status.getErrorMessage());
-    Assert.assertEquals(HiveSQLException.toString(ex1), status.getInfoMessages());
-  }
-
-  /**
-   * Tests the conversion between the exception text with the simple cause and the
-   * Throwable object
-   */
-  @Test
-  public void testExceptionMarshalling() throws Exception {
-    Exception ex1 = createException();
-    ex1.initCause(createSimpleCause());
-    Throwable ex = HiveSQLException.toCause(HiveSQLException.toString(ex1));
-
-    Assert.assertSame(RuntimeException.class, ex.getClass());
-    Assert.assertEquals("exception1", ex.getMessage());
-    Assert.assertSame(UnsupportedOperationException.class, ex.getCause().getClass());
-    Assert.assertEquals("exception2", ex.getCause().getMessage());
-  }
-
-  /**
-   * Tests the conversion between the exception text with nested cause and
-   * the Throwable object
-   */
-  @Test
-  public void testNestedException() {
-    Exception ex1 = createException();
-    ex1.initCause(createNestedCause());
-    Throwable ex = HiveSQLException.toCause(HiveSQLException.toString(ex1));
-
-    Assert.assertSame(RuntimeException.class, ex.getClass());
-    Assert.assertEquals("exception1", ex.getMessage());
-
-    Assert.assertSame(UnsupportedOperationException.class, ex.getCause().getClass());
-    Assert.assertEquals("exception2", ex.getCause().getMessage());
-
-    Assert.assertSame(Exception.class, ex.getCause().getCause().getClass());
-    Assert.assertEquals("exception3", ex.getCause().getCause().getMessage());
-  }
-
-  /**
-   * Tests the conversion of the exception with unknown source
-   */
-  @Test
-  public void testExceptionWithUnknownSource() {
-    Exception ex1 = createException();
-    ex1.initCause(createSimpleCause());
-    List<String> details = HiveSQLException.toString(ex1);
-
-    // Simulate the unknown source
-    String[] tokens = details.get(1).split(":");
-    tokens[2] = null;
-    tokens[3] = "-1";
-    details.set(1, StringUtils.join(tokens, ":"));
-
-    Throwable ex = HiveSQLException.toCause(details);
-
-    Assert.assertSame(RuntimeException.class, ex.getClass());
-    Assert.assertEquals("exception1", ex.getMessage());
-    Assert.assertSame(UnsupportedOperationException.class, ex.getCause().getClass());
-    Assert.assertEquals("exception2", ex.getCause().getMessage());
-  }
-
-  /**
-   * Tests the conversion of the exception that the class type of one of the causes
-   * doesn't exist. The stack trace text is generated on the server and passed to JDBC
-   * client. It's possible that some cause types don't exist on the client and HiveSQLException
-   * can't convert them and use RunTimeException instead.
-   */
-  @Test
-  public void testExceptionWithMissingTypeOnClient() {
-    Exception ex1 = new UnsupportedOperationException();
-    ex1.initCause(createSimpleCause());
-    List<String> details = HiveSQLException.toString(ex1);
-
-    // Simulate an unknown type
-    String[] tokens = details.get(0).split(":");
-    tokens[0] = "*DummyException";
-    details.set(0, StringUtils.join(tokens, ":"));
-
-    Throwable ex = HiveSQLException.toCause(details);
-    Assert.assertEquals(RuntimeException.class, ex.getClass());
-  }
-
-  /**
-   * Tests the conversion of the exception from anonymous class
-   */
-  @Test
-  public void testExceptionFromAnonymousClass() {
-    Dummy d = new Dummy() {
-
-      public void testExceptionConversion() {
-        Exception ex1 = createException();
-        ex1.initCause(createSimpleCause());
-        Throwable ex = HiveSQLException.toCause(HiveSQLException.toString(ex1));
-
-        Assert.assertSame(RuntimeException.class, ex.getClass());
-        Assert.assertEquals("exception1", ex.getMessage());
-        Assert.assertSame(UnsupportedOperationException.class, ex.getCause().getClass());
-        Assert.assertEquals("exception2", ex.getCause().getMessage());
-      }
-    };
-
-    d.testExceptionConversion();
+    Assert.assertEquals(HiveSQLException.DEFAULT_INFO, status.getInfoMessages());
   }
 
   interface Dummy {
@@ -175,7 +72,4 @@ public class TestHiveSQLException {
     return new UnsupportedOperationException("exception2");
   }
 
-  private static Exception createNestedCause() {
-    return new UnsupportedOperationException("exception2", new Exception("exception3"));
-  }
 }