You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2015/10/23 04:38:54 UTC

[05/11] incubator-calcite git commit: [CALCITE-645] Implement AvaticaSqlException to pass server-side exception information to clients (Josh Elser)

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/main/java/org/apache/calcite/avatica/remote/MockProtobufService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/MockProtobufService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/MockProtobufService.java
index fe357fd..8cca9d9 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/MockProtobufService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/MockProtobufService.java
@@ -34,23 +34,30 @@ import java.util.Map;
  */
 public class MockProtobufService extends ProtobufService {
 
-  private static final Map<Request, Response> MAPPING;
-  static {
+  private final String connectionId;
+  private final Map<Request, Response> mapping;
+
+  public MockProtobufService(String connectionId) {
+    this.connectionId = connectionId;
+    this.mapping = createMapping();
+  }
+
+  private Map<Request, Response> createMapping() {
     HashMap<Request, Response> mappings = new HashMap<>();
 
     // Add in mappings
 
     mappings.put(
-        new OpenConnectionRequest("0", new HashMap<String, String>()),
+        new OpenConnectionRequest(connectionId, new HashMap<String, String>()),
         new OpenConnectionResponse());
 
     // Get the schema, no.. schema..?
     mappings.put(
-        new SchemasRequest("0", null, null),
+        new SchemasRequest(connectionId, null, null),
         new ResultSetResponse(null, 1, true, null, Meta.Frame.EMPTY, -1));
 
     // Get the tables, no tables exist
-    mappings.put(new TablesRequest("0", null, null, null, Collections.<String>emptyList()),
+    mappings.put(new TablesRequest(connectionId, null, null, null, Collections.<String>emptyList()),
         new ResultSetResponse(null, 150, true, null, Meta.Frame.EMPTY, -1));
 
     // Create a statement, get back an id
@@ -58,7 +65,7 @@ public class MockProtobufService extends ProtobufService {
 
     // Prepare and execute a query. Values and schema are returned
     mappings.put(
-        new PrepareAndExecuteRequest("0", 1,
+        new PrepareAndExecuteRequest(connectionId, 1,
             "select * from (\\n values (1, 'a'), (null, 'b'), (3, 'c')) as t (c1, c2)", -1),
         new ResultSetResponse("0", 1, true,
             Meta.Signature.create(
@@ -72,7 +79,7 @@ public class MockProtobufService extends ProtobufService {
 
     // Prepare a query. Schema for results are returned, but no values
     mappings.put(
-        new PrepareRequest("0",
+        new PrepareRequest(connectionId,
             "select * from (\\n values(1, 'a'), (null, 'b'), (3, 'c')), as t (c1, c2)", -1),
         new ResultSetResponse("0", 1, true,
             Meta.Signature.create(
@@ -84,7 +91,7 @@ public class MockProtobufService extends ProtobufService {
             null, -1));
 
     mappings.put(
-        new ColumnsRequest(null, null, "my_table", null),
+        new ColumnsRequest(connectionId, null, null, "my_table", null),
         new ResultSetResponse("00000000-0000-0000-0000-000000000000", -1, true,
             Meta.Signature.create(
                 Arrays.<ColumnMetaData>asList(
@@ -94,7 +101,7 @@ public class MockProtobufService extends ProtobufService {
             Meta.Frame.create(0, true,
                 Arrays.<Object>asList(new Object[] {new Object[]{"my_table", 10}})), -1));
 
-    MAPPING = Collections.unmodifiableMap(mappings);
+    return Collections.unmodifiableMap(mappings);
   }
 
   @Override public Response _apply(Request request) {
@@ -113,20 +120,7 @@ public class MockProtobufService extends ProtobufService {
    * @throws RuntimeException if no mapping is found for the request
    */
   private Response dispatch(Request request) {
-    // Canonicalize connectionId's to 0
-    if (request instanceof OpenConnectionRequest) {
-      OpenConnectionRequest req = (OpenConnectionRequest) request;
-      request = new OpenConnectionRequest("0", req.info);
-    } else if (request instanceof TablesRequest) {
-      TablesRequest req = (TablesRequest) request;
-      request = new TablesRequest("0", req.catalog, req.schemaPattern,
-          req.tableNamePattern, req.typeList);
-    } else if (request instanceof SchemasRequest) {
-      SchemasRequest req = (SchemasRequest) request;
-      request = new SchemasRequest("0", req.catalog, req.schemaPattern);
-    }
-
-    Response response = MAPPING.get(request);
+    Response response = mapping.get(request);
 
     if (null == response) {
       throw new RuntimeException("Had no response mapping for " + request);
@@ -140,7 +134,7 @@ public class MockProtobufService extends ProtobufService {
    */
   public static class MockProtobufServiceFactory implements Service.Factory {
     @Override public Service create(AvaticaConnection connection) {
-      return new MockProtobufService();
+      return new MockProtobufService(connection.handle.id);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufHandler.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufHandler.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufHandler.java
index 52f0a03..d77b52d 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufHandler.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufHandler.java
@@ -18,8 +18,6 @@ package org.apache.calcite.avatica.remote;
 
 import org.apache.calcite.avatica.remote.Service.Response;
 
-import com.google.protobuf.InvalidProtocolBufferException;
-
 import java.io.IOException;
 
 /**
@@ -27,36 +25,25 @@ import java.io.IOException;
  * by converting them to the POJO Request. Returns back the serialized protocol
  * buffer response.
  */
-public class ProtobufHandler implements Handler<byte[]> {
+public class ProtobufHandler extends AbstractHandler<byte[]> {
 
-  private final Service service;
   private final ProtobufTranslation translation;
 
   public ProtobufHandler(Service service, ProtobufTranslation translation) {
-    this.service = service;
+    super(service);
     this.translation = translation;
   }
 
-  @Override public byte[] apply(byte[] requestBytes) {
-    // Transform the protocol buffer bytes into a POJO
-    // Encapsulate the task of transforming this since
-    // the bytes also contain the PB request class name.
-    Service.Request requestPojo;
-    try {
-      requestPojo = translation.parseRequest(requestBytes);
-    } catch (InvalidProtocolBufferException e) {
-      throw new RuntimeException(e);
-    }
-
-    // Get the response for the request
-    Response response = requestPojo.accept(service);
-
-    try {
-      // Serialize it into bytes for the wire.
-      return translation.serializeResponse(response);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
+  @Override public HandlerResponse<byte[]> apply(byte[] requestBytes) {
+    return super.apply(requestBytes);
+  }
+
+  @Override Service.Request decode(byte[] serializedRequest) throws IOException {
+    return translation.parseRequest(serializedRequest);
+  }
+
+  @Override byte[] encode(Response response) throws IOException {
+    return translation.serializeResponse(response);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
index e41e9f6..35cb35a 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
@@ -74,8 +74,7 @@ public abstract class ProtobufService extends AbstractService {
     return (CloseStatementResponse) _apply(request);
   }
 
-  @Override
-  public OpenConnectionResponse apply(OpenConnectionRequest request) {
+  @Override public OpenConnectionResponse apply(OpenConnectionRequest request) {
     return (OpenConnectionResponse) _apply(request);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteProtobufService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteProtobufService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteProtobufService.java
index 8cf0f0d..fa447de 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteProtobufService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteProtobufService.java
@@ -38,6 +38,8 @@ public class RemoteProtobufService extends ProtobufService {
   }
 
   @Override public Response _apply(Request request) {
+    final InputStream inputStream;
+
     try {
       final HttpURLConnection connection =
           (HttpURLConnection) url.openConnection();
@@ -52,21 +54,29 @@ public class RemoteProtobufService extends ProtobufService {
       }
       final int responseCode = connection.getResponseCode();
       if (responseCode != HttpURLConnection.HTTP_OK) {
-        InputStream errorStream = connection.getErrorStream();
-        if (errorStream != null) {
-          byte[] errorResponse = AvaticaUtils.readFullyToBytes(errorStream);
-          ErrorResponse response = (ErrorResponse) translation.parseResponse(errorResponse);
-          throw new RuntimeException("Remote driver error: " + response.message);
-        } else {
-          throw new RuntimeException("response code " + responseCode);
-        }
+        inputStream = connection.getErrorStream();
+      } else {
+        inputStream = connection.getInputStream();
       }
-      final InputStream inputStream = connection.getInputStream();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    Response resp;
+    try {
       // Read the (serialized protobuf) response off the wire and convert it back to a Response
-      return translation.parseResponse(AvaticaUtils.readFullyToBytes(inputStream));
+      resp = translation.parseResponse(AvaticaUtils.readFullyToBytes(inputStream));
     } catch (IOException e) {
+      // Not a protobuf that we could parse.
       throw new RuntimeException(e);
     }
+
+    // The server had an error, throw an Exception for that.
+    if (resp instanceof ErrorResponse) {
+      throw ((ErrorResponse) resp).toException();
+    }
+
+    return resp;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteService.java
index 298aa95..f661cbd 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteService.java
@@ -53,22 +53,19 @@ public class RemoteService extends JsonService {
           wr.close();
         }
       }
-      int responseCode = connection.getResponseCode();
+      final int responseCode = connection.getResponseCode();
+      final InputStream inputStream;
       if (responseCode != HttpURLConnection.HTTP_OK) {
-        InputStream errorStream = connection.getErrorStream();
-        if (errorStream != null) {
-          String errorResponse = AvaticaUtils.readFully(errorStream);
-          ErrorResponse response = decode(errorResponse, ErrorResponse.class);
-          throw new RuntimeException("Remote driver error: " + response.message);
-        } else {
-          throw new RuntimeException("response code " + responseCode);
-        }
+        inputStream = connection.getErrorStream();
+      } else {
+        inputStream = connection.getInputStream();
       }
-      final InputStream inputStream = connection.getInputStream();
+
       return AvaticaUtils.readFully(inputStream);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
+
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
index 3717ab5..2309661 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
@@ -16,7 +16,9 @@
  */
 package org.apache.calcite.avatica.remote;
 
+import org.apache.calcite.avatica.AvaticaClientRuntimeException;
 import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaSeverity;
 import org.apache.calcite.avatica.ConnectionPropertiesImpl;
 import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.proto.Common;
@@ -30,11 +32,16 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import com.google.protobuf.Descriptors.Descriptor;
 import com.google.protobuf.Message;
 
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Objects;
 
 /**
  * API for request-response calls to an Avatica server.
@@ -113,7 +120,8 @@ public interface Service {
           name = "closeConnection"),
       @JsonSubTypes.Type(value = ConnectionSyncResponse.class, name = "connectionSync"),
       @JsonSubTypes.Type(value = DatabasePropertyResponse.class, name = "databaseProperties"),
-      @JsonSubTypes.Type(value = ExecuteResponse.class, name = "executeResults") })
+      @JsonSubTypes.Type(value = ExecuteResponse.class, name = "executeResults"),
+      @JsonSubTypes.Type(value = ErrorResponse.class, name = "error") })
   abstract class Response {
     abstract Response deserialize(Message genericMsg);
     abstract Message serialize();
@@ -1924,8 +1932,7 @@ public interface Service {
       return service.apply(this);
     }
 
-    @Override
-    Request deserialize(Message genericMsg) {
+    @Override Request deserialize(Message genericMsg) {
       if (!(genericMsg instanceof Requests.OpenConnectionRequest)) {
         throw new IllegalArgumentException(
             "Expected OpenConnectionRequest, but got" + genericMsg.getClass().getName());
@@ -1948,8 +1955,7 @@ public interface Service {
       return new OpenConnectionRequest(connectionId, info);
     }
 
-    @Override
-    Message serialize() {
+    @Override Message serialize() {
       Requests.OpenConnectionRequest.Builder builder = Requests.OpenConnectionRequest.newBuilder();
       if (null != connectionId) {
         builder.setConnectionId(connectionId);
@@ -2112,6 +2118,7 @@ public interface Service {
   /** Response from
    * {@link org.apache.calcite.avatica.remote.Service.CloseConnectionRequest}. */
   class CloseConnectionResponse extends Response {
+
     @JsonCreator
     public CloseConnectionResponse() {}
 
@@ -2438,74 +2445,149 @@ public interface Service {
   }
 
   /**
-   * ErrorResponse can be used in response to any kind of request. It is used internally
-   * by the transport layers to format errors for transport over the wire.
-   * Thus, {@link Request#apply} will never return an ErrorResponse.
+   * Response for any request that the server failed to successfully perform.
+   * It is used internally by the transport layers to format errors for
+   * transport over the wire. Thus, {@link Request#apply} will never return
+   * an ErrorResponse.
    */
-  class ErrorResponse extends Response {
-    public final String message;
+  public class ErrorResponse extends Response {
+    public static final int UNKNOWN_ERROR_CODE = -1;
+    public static final String UNKNOWN_SQL_STATE = "00000";
+
+    public final List<String> exceptions;
+    public final String errorMessage;
+    public final int errorCode;
+    public final String sqlState;
+    public final AvaticaSeverity severity;
 
-    public ErrorResponse() {
-      message = null;
+    ErrorResponse() {
+      exceptions = Collections.singletonList("Unhandled exception");
+      errorMessage = "Unknown message";
+      errorCode = -1;
+      sqlState = UNKNOWN_SQL_STATE;
+      severity = AvaticaSeverity.UNKNOWN;
     }
 
     @JsonCreator
-    public ErrorResponse(@JsonProperty("message") String message) {
-      this.message = message;
+    public ErrorResponse(@JsonProperty("exceptions") List<String> exceptions,
+        @JsonProperty("errorMessage") String errorMessage,
+        @JsonProperty("errorCode") int errorCode,
+        @JsonProperty("sqlState") String sqlState,
+        @JsonProperty("severity") AvaticaSeverity severity) {
+      this.exceptions = exceptions;
+      this.errorMessage = errorMessage;
+      this.errorCode = errorCode;
+      this.sqlState = sqlState;
+      this.severity = severity;
+    }
+
+    protected ErrorResponse(Exception e, String errorMessage, int code, String sqlState,
+        AvaticaSeverity severity) {
+      this(errorMessage, code, sqlState, severity, toStackTraces(e));
+    }
+
+    protected ErrorResponse(String errorMessage, int code, String sqlState,
+        AvaticaSeverity severity, List<String> exceptions) {
+      this.exceptions = exceptions;
+      this.errorMessage = errorMessage;
+      this.errorCode = code;
+      this.sqlState = sqlState;
+      this.severity = severity;
+    }
+
+    static List<String> toStackTraces(Exception e) {
+      List<String> stackTraces = new ArrayList<>();
+      stackTraces.add(toString(e));
+      if (e instanceof SQLException) {
+        SQLException next = ((SQLException) e).getNextException();
+        while (null != next) {
+          stackTraces.add(toString(next));
+          next = next.getNextException();
+        }
+      }
+      return stackTraces;
+    }
+
+    static String toString(Exception e) {
+      StringWriter sw = new StringWriter();
+      Objects.requireNonNull(e).printStackTrace(new PrintWriter(sw));
+      return sw.toString();
     }
 
     @Override ErrorResponse deserialize(Message genericMsg) {
       if (!(genericMsg instanceof Responses.ErrorResponse)) {
-        throw new IllegalArgumentException(
-            "Expected ErrorResponse, but got " + genericMsg.getClass().getName());
-      }
-
-      final Responses.ErrorResponse msg = (Responses.ErrorResponse) genericMsg;
-      final Descriptor desc = msg.getDescriptorForType();
-
-      String message = null;
-      if (ProtobufService.hasField(msg, desc,
-          Responses.ErrorResponse.MESSAGE_FIELD_NUMBER)) {
-        message = msg.getMessage();
+        throw new IllegalArgumentException("Expected ErrorResponse, but got "
+          + genericMsg.getClass());
       }
 
-      return new ErrorResponse(message);
+      Responses.ErrorResponse msg = (Responses.ErrorResponse) genericMsg;
+      return new ErrorResponse(msg.getExceptionsList(), msg.getErrorMessage(),
+          msg.getErrorCode(), msg.getSqlState(), AvaticaSeverity.fromProto(msg.getSeverity()));
     }
 
     @Override Responses.ErrorResponse serialize() {
       Responses.ErrorResponse.Builder builder = Responses.ErrorResponse.newBuilder();
-
-      if (null != message) {
-        builder.setMessage(message);
-      }
-
-      return builder.build();
+      return builder.addAllExceptions(exceptions).setErrorMessage(errorMessage)
+          .setErrorCode(errorCode).setSqlState(sqlState).setSeverity(severity.toProto()).build();
     }
 
     @Override public int hashCode() {
-      return message == null ? 0 : message.hashCode();
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((exceptions == null) ? 0 : exceptions.hashCode());
+      result = prime * result + errorCode;
+      result = prime * result + ((sqlState == null) ? 0 : sqlState.hashCode());
+      result = prime * result + ((severity == null) ? 0 : severity.hashCode());
+      return result;
     }
 
-    @Override public boolean equals(Object o) {
-      if (o == this) {
+    @Override public boolean equals(Object obj) {
+      if (this == obj) {
         return true;
       }
+      if (!(obj instanceof ErrorResponse)) {
+        return false;
+      }
 
-      if (o instanceof ErrorResponse) {
-        ErrorResponse other = (ErrorResponse) o;
+      ErrorResponse other = (ErrorResponse) obj;
+      if (exceptions == null) {
+        if (other.exceptions != null) {
+          return false;
+        }
+      } else if (!exceptions.equals(other.exceptions)) {
+        return false;
+      }
 
-        if (null == message) {
-          if (null != other.message) {
-            return false;
-          }
-        } else if (!message.equals(other.message)) {
+      if (errorMessage == null) {
+        if (other.errorMessage != null) {
           return false;
         }
+      } else if (!errorMessage.equals(other.errorMessage)) {
+        return false;
+      }
 
-        return true;
+      if (errorCode != other.errorCode) {
+        return false;
       }
 
-      return false;
+      if (sqlState == null) {
+        if (other.sqlState != null) {
+          return false;
+        }
+      } else if (!sqlState.equals(other.sqlState)) {
+        return false;
+      }
+
+      if (severity != other.severity) {
+        return false;
+      }
+
+      return true;
+    }
+
+    public AvaticaClientRuntimeException toException() {
+      return new AvaticaClientRuntimeException("Remote driver error: " + errorMessage, errorCode,
+          sqlState, severity, exceptions);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/main/protobuf/common.proto
----------------------------------------------------------------------
diff --git a/avatica/src/main/protobuf/common.proto b/avatica/src/main/protobuf/common.proto
index 05bb7c2..3716c7e 100644
--- a/avatica/src/main/protobuf/common.proto
+++ b/avatica/src/main/protobuf/common.proto
@@ -189,3 +189,12 @@ message TypedValue {
   double double_value = 6; // big numbers
   bool null = 7; // a null object
 }
+
+// The severity of some unexpected outcome to an operation.
+// Protobuf enum values must be unique across all other enums
+enum Severity {
+  UNKNOWN_SEVERITY = 0;
+  FATAL_SEVERITY = 1;
+  ERROR_SEVERITY = 2;
+  WARNING_SEVERITY = 3;
+}

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/main/protobuf/responses.proto
----------------------------------------------------------------------
diff --git a/avatica/src/main/protobuf/responses.proto b/avatica/src/main/protobuf/responses.proto
index e4f52a3..7c52be7 100644
--- a/avatica/src/main/protobuf/responses.proto
+++ b/avatica/src/main/protobuf/responses.proto
@@ -82,6 +82,11 @@ message DatabasePropertyResponse {
   repeated DatabasePropertyElement props = 1;
 }
 
+// Send contextual information about some error over the wire from the server.
 message ErrorResponse {
-  string message = 1;
+  repeated string exceptions = 1; // exception stacktraces, many for linked exceptions.
+  string error_message = 2; // human readable description
+  Severity severity = 3;
+  uint32 error_code = 4; // numeric identifier for error
+  string sql_state = 5; // five-character standard-defined value
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/test/java/org/apache/calcite/avatica/remote/AbstractHandlerTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/remote/AbstractHandlerTest.java b/avatica/src/test/java/org/apache/calcite/avatica/remote/AbstractHandlerTest.java
new file mode 100644
index 0000000..012cccc
--- /dev/null
+++ b/avatica/src/test/java/org/apache/calcite/avatica/remote/AbstractHandlerTest.java
@@ -0,0 +1,165 @@
+/*
+ * 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.calcite.avatica.remote;
+
+import org.apache.calcite.avatica.AvaticaSeverity;
+import org.apache.calcite.avatica.remote.Handler.HandlerResponse;
+import org.apache.calcite.avatica.remote.Service.ErrorResponse;
+import org.apache.calcite.avatica.remote.Service.Request;
+import org.apache.calcite.avatica.remote.Service.Response;
+
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Arrays;
+import java.util.Objects;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test for common functionality across {@link Handler} implementations.
+ */
+public class AbstractHandlerTest {
+
+  private String exceptionToString(Exception e) {
+    StringWriter sw = new StringWriter();
+    PrintWriter pw = new PrintWriter(sw);
+    Objects.requireNonNull(e).printStackTrace(pw);
+    return sw.toString();
+  }
+
+  @Test public void testExceptionUnwrappingWithoutContext() {
+    @SuppressWarnings("unchecked")
+    AbstractHandler<String> handler = Mockito.mock(AbstractHandler.class);
+
+    Mockito.when(handler.unwrapException(Mockito.any(Exception.class))).thenCallRealMethod();
+
+    Exception e = new RuntimeException();
+    Response resp = handler.unwrapException(e);
+    assertTrue("Response should be ErrorResponse, but was " + resp.getClass(),
+        resp instanceof ErrorResponse);
+    ErrorResponse errorResp = (ErrorResponse) resp;
+    assertEquals(ErrorResponse.UNKNOWN_ERROR_CODE, errorResp.errorCode);
+    assertEquals(AvaticaSeverity.UNKNOWN, errorResp.severity);
+    assertEquals(Arrays.asList(exceptionToString(e)), errorResp.exceptions);
+
+    e = new AvaticaRuntimeException();
+    resp = handler.unwrapException(e);
+    assertTrue("Response should be ErrorResponse, but was " + resp.getClass(),
+        resp instanceof ErrorResponse);
+    errorResp = (ErrorResponse) resp;
+    assertEquals(ErrorResponse.UNKNOWN_ERROR_CODE, errorResp.errorCode);
+    assertEquals(AvaticaSeverity.UNKNOWN, errorResp.severity);
+    assertEquals(Arrays.asList(exceptionToString(e)), errorResp.exceptions);
+  }
+
+  @Test public void testExceptionUnwrappingWithContext() {
+    @SuppressWarnings("unchecked")
+    AbstractHandler<String> handler = Mockito.mock(AbstractHandler.class);
+
+    Mockito.when(handler.unwrapException(Mockito.any(Exception.class))).thenCallRealMethod();
+
+    final String msg = "Something failed!";
+    AvaticaRuntimeException e = new AvaticaRuntimeException(msg,
+        ErrorResponse.UNKNOWN_ERROR_CODE, ErrorResponse.UNKNOWN_SQL_STATE, AvaticaSeverity.FATAL);
+    Response resp = handler.unwrapException(e);
+    assertTrue("Response should be ErrorResponse, but was " + resp.getClass(),
+        resp instanceof ErrorResponse);
+    ErrorResponse errorResp = (ErrorResponse) resp;
+    assertEquals(ErrorResponse.UNKNOWN_ERROR_CODE, errorResp.errorCode);
+    assertEquals(AvaticaSeverity.FATAL, errorResp.severity);
+    assertEquals(Arrays.asList(exceptionToString(e)), errorResp.exceptions);
+    assertEquals(msg, errorResp.errorMessage);
+  }
+
+  @Test public void testFailedResponseSerialization() throws IOException {
+    @SuppressWarnings("unchecked")
+    final AbstractHandler<String> handler = Mockito.mock(AbstractHandler.class);
+    final Request request = Mockito.mock(Request.class);
+    final Response response = Mockito.mock(Response.class);
+    final IOException exception = new IOException();
+    final ErrorResponse errorResponse = Mockito.mock(ErrorResponse.class);
+    final String serializedErrorResponse = "An ErrorResponse";
+
+    // Accept a serialized request
+    Mockito.when(handler.apply(Mockito.anyString())).thenCallRealMethod();
+    // Deserialize it back into a POJO
+    Mockito.when(handler.decode(Mockito.anyString())).thenReturn(request);
+    // Construct the Response for that Request
+    Mockito.when(request.accept(Mockito.any(Service.class))).thenReturn(response);
+    // Throw an IOException when serializing the Response.
+    Mockito.when(handler.encode(response)).thenThrow(exception);
+    // Convert the IOException into an ErrorResponse
+    Mockito.when(handler.unwrapException(exception)).thenReturn(errorResponse);
+    Mockito.when(handler.encode(errorResponse)).thenReturn(serializedErrorResponse);
+
+    HandlerResponse<String> handlerResp = handler.apply("this is mocked out");
+    assertEquals(500, handlerResp.getStatusCode());
+    assertEquals(serializedErrorResponse, handlerResp.getResponse());
+  }
+
+  @Test public void testFailedErrorResponseSerialization() throws IOException {
+    @SuppressWarnings("unchecked")
+    final AbstractHandler<String> handler = Mockito.mock(AbstractHandler.class);
+    final Request request = Mockito.mock(Request.class);
+    final Response response = Mockito.mock(Response.class);
+    final IOException exception = new IOException();
+    final ErrorResponse errorResponse = Mockito.mock(ErrorResponse.class);
+
+    // Accept a serialized request
+    Mockito.when(handler.apply(Mockito.anyString())).thenCallRealMethod();
+    // Deserialize it back into a POJO
+    Mockito.when(handler.decode(Mockito.anyString())).thenReturn(request);
+    // Construct the Response for that Request
+    Mockito.when(request.accept(Mockito.any(Service.class))).thenReturn(response);
+    // Throw an IOException when serializing the Response.
+    Mockito.when(handler.encode(response)).thenThrow(exception);
+    // Convert the IOException into an ErrorResponse
+    Mockito.when(handler.unwrapException(exception)).thenReturn(errorResponse);
+    // Fail to serialize the ErrorResponse
+    Mockito.when(handler.encode(errorResponse)).thenThrow(exception);
+
+    try {
+      handler.apply("this is mocked out");
+    } catch (RuntimeException e) {
+      assertEquals(exception, e.getCause());
+    }
+  }
+
+  @Test public void testFailedRequestDeserialization() throws IOException {
+    @SuppressWarnings("unchecked")
+    final AbstractHandler<String> handler = Mockito.mock(AbstractHandler.class);
+    final IOException exception = new IOException();
+
+    // Accept a serialized request
+    Mockito.when(handler.apply(Mockito.anyString())).thenCallRealMethod();
+    // Throw an Exception trying to convert it back into a POJO
+    Mockito.when(handler.decode(Mockito.anyString())).thenThrow(exception);
+
+    try {
+      handler.apply("this is mocked out");
+    } catch (RuntimeException e) {
+      assertEquals(exception, e.getCause());
+    }
+  }
+}
+
+// End AbstractHandlerTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/test/java/org/apache/calcite/avatica/remote/ErrorResponseTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/remote/ErrorResponseTest.java b/avatica/src/test/java/org/apache/calcite/avatica/remote/ErrorResponseTest.java
new file mode 100644
index 0000000..7ee6fcf
--- /dev/null
+++ b/avatica/src/test/java/org/apache/calcite/avatica/remote/ErrorResponseTest.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.avatica.remote;
+
+import org.apache.calcite.avatica.AvaticaClientRuntimeException;
+import org.apache.calcite.avatica.AvaticaSeverity;
+import org.apache.calcite.avatica.remote.Service.ErrorResponse;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * A test class for ErrorResponse.
+ */
+public class ErrorResponseTest {
+
+  @Test public void testEquality() {
+    final String message = "There was an error";
+    final int code = 23;
+    final String state = "a1b2c";
+    final AvaticaSeverity severity = AvaticaSeverity.ERROR;
+    final List<String> exceptions = Arrays.asList("Server Stacktrace 1", "Server Stacktace 2");
+    assertEquals(new ErrorResponse(message, code, state, severity, exceptions),
+        new ErrorResponse(message, code, state, severity, exceptions));
+  }
+
+  @Test public void testToClientRTE() {
+    final String message = "There was an error";
+    final int code = 23;
+    final String state = "a1b2c";
+    final AvaticaSeverity severity = AvaticaSeverity.ERROR;
+    final List<String> exceptions = Arrays.asList("Server Stacktrace 1", "Server Stacktace 2");
+    final ErrorResponse resp = new ErrorResponse(message, code, state, severity, exceptions);
+    AvaticaClientRuntimeException exception = resp.toException();
+    assertTrue("Expected error message to end with '" + resp.errorMessage + "', but was '"
+        + exception.getMessage() + "'", exception.getMessage().endsWith(resp.errorMessage));
+    assertEquals(resp.errorCode, exception.getErrorCode());
+    assertEquals(resp.severity, exception.getSeverity());
+    assertEquals(resp.sqlState, exception.getSqlState());
+    assertEquals(resp.exceptions, exception.getServerExceptions());
+  }
+}
+
+// End ErrorResponseTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
index 2e307ca..aa03c3c 100644
--- a/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
+++ b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
@@ -21,6 +21,7 @@ import org.apache.calcite.avatica.Meta.Frame;
 import org.apache.calcite.avatica.proto.Common;
 import org.apache.calcite.avatica.proto.Requests;
 import org.apache.calcite.avatica.proto.Responses;
+import org.apache.calcite.avatica.remote.Handler.HandlerResponse;
 import org.apache.calcite.avatica.remote.Service.FetchRequest;
 import org.apache.calcite.avatica.remote.Service.FetchResponse;
 
@@ -90,7 +91,9 @@ public class ProtobufHandlerTest {
     when(translation.serializeResponse(response))
         .thenReturn(response.serialize().toByteArray());
 
-    byte[] serializedResponse = handler.apply(serializedRequest);
+    HandlerResponse<byte[]> handlerResponse = handler.apply(serializedRequest);
+    byte[] serializedResponse = handlerResponse.getResponse();
+    assertEquals(200, handlerResponse.getStatusCode());
 
     Responses.FetchResponse protoResponse = Responses.FetchResponse.parseFrom(serializedResponse);
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
index 337932e..d385054 100644
--- a/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
+++ b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.avatica.remote;
 
 import org.apache.calcite.avatica.AvaticaParameter;
+import org.apache.calcite.avatica.AvaticaSeverity;
 import org.apache.calcite.avatica.ColumnMetaData;
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.calcite.avatica.ConnectionPropertiesImpl;
@@ -60,6 +61,8 @@ import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
 import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
 import java.sql.Types;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -273,7 +276,11 @@ public class ProtobufTranslationImplTest<T> {
             new Meta.StatementHandle("connectionId", Integer.MAX_VALUE,
                 signature)));
 
-    responses.add(new ErrorResponse("an error occurred"));
+    StringWriter sw = new StringWriter();
+    new Exception().printStackTrace(new PrintWriter(sw));
+    responses.add(new ErrorResponse(Collections.singletonList(sw.toString()), "Test Error Message",
+        ErrorResponse.UNKNOWN_ERROR_CODE, ErrorResponse.UNKNOWN_SQL_STATE,
+        AvaticaSeverity.WARNING));
 
     return responses;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaClientRuntimeExceptionTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaClientRuntimeExceptionTest.java b/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaClientRuntimeExceptionTest.java
new file mode 100644
index 0000000..d457aa3
--- /dev/null
+++ b/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaClientRuntimeExceptionTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.calcite.avatica.test;
+
+import org.apache.calcite.avatica.AvaticaClientRuntimeException;
+import org.apache.calcite.avatica.AvaticaSeverity;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test class for {@link AvaticaClientRuntimeException}.
+ */
+public class AvaticaClientRuntimeExceptionTest {
+
+  @Test public void testGetters() {
+    final String errorMsg = "My error message";
+    final int errorCode = 10;
+    final String sqlState = "abc12";
+    final AvaticaSeverity severity = AvaticaSeverity.ERROR;
+    final List<String> stacktraces = Arrays.asList("my stack trace");
+    AvaticaClientRuntimeException e = new AvaticaClientRuntimeException(errorMsg, errorCode,
+        sqlState, severity, stacktraces);
+    assertEquals(errorMsg, e.getMessage());
+    assertEquals(errorCode, e.getErrorCode());
+    assertEquals(severity, e.getSeverity());
+    assertEquals(stacktraces, e.getServerExceptions());
+  }
+
+}
+
+// End AvaticaClientRuntimeExceptionTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSeverityTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSeverityTest.java b/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSeverityTest.java
new file mode 100644
index 0000000..945f959
--- /dev/null
+++ b/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSeverityTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.calcite.avatica.test;
+
+import org.apache.calcite.avatica.AvaticaSeverity;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link AvaticaSeverity}.
+ */
+public class AvaticaSeverityTest {
+
+  @Test
+  public void testProtobufSerialization() {
+    for (AvaticaSeverity severity : AvaticaSeverity.values()) {
+      assertEquals(severity, AvaticaSeverity.fromProto(severity.toProto()));
+    }
+  }
+
+}
+
+// End AvaticaSeverityTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSqlExceptionTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSqlExceptionTest.java b/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSqlExceptionTest.java
new file mode 100644
index 0000000..fb10c45
--- /dev/null
+++ b/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSqlExceptionTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.calcite.avatica.test;
+
+import org.apache.calcite.avatica.AvaticaSqlException;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link AvaticaSqlException}.
+ */
+public class AvaticaSqlExceptionTest {
+
+  @Test public void testGetters() {
+    final String msg = "My query failed!";
+    final int code = 42;
+    final String sql = "SELECT foo FROM bar;";
+    final String stacktrace = "My Stack Trace";
+
+    AvaticaSqlException e = new AvaticaSqlException(msg, sql, code, Arrays.asList(stacktrace));
+    assertTrue(e.getMessage().contains(msg));
+    assertEquals(code, e.getErrorCode());
+    assertEquals(sql, e.getSQLState());
+    assertEquals(1, e.getStackTraces().size());
+    assertEquals(stacktrace, e.getStackTraces().get(0));
+  }
+
+}
+
+// End AvaticaSqlExceptionTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5be93fb4/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java b/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
index 4147bcd..59a3bfa 100644
--- a/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
@@ -109,7 +109,7 @@ public class ExceptionMessageTest {
       fail("Query badEntries should result in an exception");
     } catch (SQLException e) {
       assertThat(e.getMessage(),
-          equalTo("error while executing SQL \"select * from \"badEntries\"\": "
+          equalTo("Error while executing SQL \"select * from \"badEntries\"\": "
               + "Can't iterate over badEntries"));
     }
   }
@@ -120,7 +120,7 @@ public class ExceptionMessageTest {
       fail("Query should fail");
     } catch (SQLException e) {
       assertThat(e.getMessage(),
-          equalTo("error while executing SQL \"invalid sql\": parse failed: "
+          equalTo("Error while executing SQL \"invalid sql\": parse failed: "
               + "Non-query expression encountered in illegal context"));
     }
   }