You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by jd...@apache.org on 2016/07/21 22:40:15 UTC

[2/3] incubator-kudu git commit: [java client] Redo how we manage exceptions

[java client] Redo how we manage exceptions

Right now the exceptions are hard to handle in the Java client. They're all
generic and you need to do a lot of introspection. For example, if you try
to create a table that already exists, you need to start searching the
exception's message to know if it's that or some other problem that gave
you the error.

With this patch we now only one main kind of public exception: KuduException.
We still have Recoverable/NonRecoverableException but those are now
package-private and only used internally. PleaseThrottleException is kept public
for the async API.

KuduException has a new field, `status`, which is your regular Kudu Status
object. Wherever we can we try to recreate the Status objects that are sent
to us from the servers, else we create our own. Now for the example above we
can just query the exception's status with `isNotFound()`.

The sync APIs is also modified to only throw KuduExceptions instead of plain
Exceptions.

Change-Id: Iba6e8a022d7a5391c3657cbdc9d3f06f951be048
Reviewed-on: http://gerrit.cloudera.org:8080/3055
Reviewed-by: Dan Burkert <da...@cloudera.com>
Tested-by: Kudu Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kudu/commit/0a792366
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kudu/tree/0a792366
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kudu/diff/0a792366

Branch: refs/heads/master
Commit: 0a792366ea44beff17f3bec659b3def8cb669e68
Parents: 83d6923
Author: Jean-Daniel Cryans <jd...@apache.org>
Authored: Thu May 12 16:09:54 2016 -0700
Committer: Jean-Daniel Cryans <jd...@apache.org>
Committed: Thu Jul 21 22:37:59 2016 +0000

----------------------------------------------------------------------
 .../java/org/kududb/client/AsyncKuduClient.java |  33 +++---
 .../org/kududb/client/AsyncKuduScanner.java     |  11 +-
 .../org/kududb/client/AsyncKuduSession.java     |  25 ++--
 .../kududb/client/ConnectionResetException.java |  49 --------
 .../client/GetMasterRegistrationReceived.java   |  23 +++-
 .../kududb/client/InvalidResponseException.java |  80 -------------
 .../main/java/org/kududb/client/KuduClient.java | 114 +++++++++++--------
 .../java/org/kududb/client/KuduException.java   |  64 ++++++++---
 .../main/java/org/kududb/client/KuduRpc.java    |  11 +-
 .../java/org/kududb/client/KuduScanner.java     |  18 ++-
 .../org/kududb/client/KuduServerException.java  |  47 --------
 .../java/org/kududb/client/KuduSession.java     |  32 ++++--
 .../org/kududb/client/MasterErrorException.java |  39 -------
 .../client/NoLeaderMasterFoundException.java    |  36 +-----
 .../kududb/client/NonCoveredRangeException.java |   7 +-
 .../kududb/client/NonRecoverableException.java  |  17 ++-
 .../kududb/client/PleaseThrottleException.java  |  20 ++--
 .../org/kududb/client/RecoverableException.java |  23 ++--
 .../org/kududb/client/RowResultIterator.java    |  50 ++++----
 .../src/main/java/org/kududb/client/Status.java |  40 ++++++-
 .../java/org/kududb/client/TabletClient.java    | 100 +++++++++-------
 .../client/TabletServerErrorException.java      |  40 -------
 .../org/kududb/client/TestAsyncKuduSession.java |  10 +-
 .../java/org/kududb/client/TestKuduSession.java |   4 +-
 .../java/org/kududb/client/TestKuduTable.java   |  17 +++
 .../java/org/kududb/client/TestTimeouts.java    |  16 +--
 26 files changed, 406 insertions(+), 520 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduClient.java b/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduClient.java
index 52a5fdb..2cf9efa 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduClient.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduClient.java
@@ -364,7 +364,7 @@ public class AsyncKuduClient implements AutoCloseable {
    * @param name the table's name, if the table was renamed then that name must be checked against
    * @return a deferred object to track the progress of the isAlterTableDone command
    */
-  public Deferred<IsAlterTableDoneResponse> isAlterTableDone(String name) throws Exception {
+  public Deferred<IsAlterTableDoneResponse> isAlterTableDone(String name) {
     checkIsClosed();
     IsAlterTableDoneRequest request = new IsAlterTableDoneRequest(this.masterTable, name);
     request.setTimeoutMillis(defaultAdminOperationTimeoutMs);
@@ -1000,12 +1000,10 @@ public class AsyncKuduClient implements AutoCloseable {
   }
 
   /**
-   * Checks whether or not an RPC can be retried once more.
-   * @param rpc The RPC we're going to attempt to execute.
+   * Checks whether or not an RPC can be retried once more
+   * @param rpc The RPC we're going to attempt to execute
    * @return {@code true} if this RPC already had too many attempts,
-   * {@code false} otherwise (in which case it's OK to retry once more).
-   * @throws NonRecoverableException if the request has had too many attempts
-   * already.
+   * {@code false} otherwise (in which case it's OK to retry once more)
    */
   static boolean cannotRetryRequest(final KuduRpc<?> rpc) {
     return rpc.deadlineTracker.timedOut() || rpc.attempt > MAX_RPC_ATTEMPTS;
@@ -1026,7 +1024,8 @@ public class AsyncKuduClient implements AutoCloseable {
     } else {
       message = "RPC can not complete before timeout: ";
     }
-    final Exception e = new NonRecoverableException(message + request, cause);
+    Status statusTimedOut = Status.TimedOut(message + request);
+    final Exception e = new NonRecoverableException(statusTimedOut, cause);
     request.errback(e);
     LOG.debug("Cannot continue with this RPC: {} because of: {}", request, message, e);
     return Deferred.fromError(e);
@@ -1091,7 +1090,8 @@ public class AsyncKuduClient implements AutoCloseable {
       if (clientForHostAndPort == null) {
         String message = "Couldn't resolve this master's address " + hostAndPort.toString();
         LOG.warn(message);
-        d = Deferred.fromError(new NonRecoverableException(message));
+        Status statusIOE = Status.IOError(message);
+        d = Deferred.fromError(new NonRecoverableException(statusIOE));
       } else {
         d = getMasterRegistration(clientForHostAndPort);
       }
@@ -1157,8 +1157,9 @@ public class AsyncKuduClient implements AutoCloseable {
       }
 
       if (deadlineTracker.timedOut()) {
-        return Deferred.fromError(new NonRecoverableException(
-            "Took too long getting the list of tablets, " + deadlineTracker));
+        Status statusTimedOut = Status.TimedOut("Took too long getting the list of tablets, " +
+            deadlineTracker);
+        return Deferred.fromError(new NonRecoverableException(statusTimedOut));
       }
 
       // If the partition key location isn't cached, and the request hasn't timed out,
@@ -1273,10 +1274,15 @@ public class AsyncKuduClient implements AutoCloseable {
           LOG.debug("Table {} has a non-running tablet", table.getName());
           tablesNotServed.add(table.getTableId());
         } else {
-          return new MasterErrorException("GetTableLocations error", response.getError());
+          Status status = Status.fromMasterErrorPB(response.getError());
+          return new NonRecoverableException(status);
         }
       } else {
-        discoverTablets(table, response.getTabletLocationsList());
+        try {
+          discoverTablets(table, response.getTabletLocationsList());
+        } catch (NonRecoverableException e) {
+          return e;
+        }
         if (partitionKey != null) {
           discoverNonCoveredRangePartitions(table.getTableId(), partitionKey,
                                             response.getTabletLocationsList());
@@ -2058,8 +2064,9 @@ public class AsyncKuduClient implements AutoCloseable {
         // no point in retrying.
         if (!lookupExceptions.isEmpty() &&
             lookupExceptions.size() == tabletLocations.getReplicasCount()) {
-          throw new NonRecoverableException("Couldn't find any valid locations, exceptions: " +
+          Status statusIOE = Status.IOError("Couldn't find any valid locations, exceptions: " +
               lookupExceptions);
+          throw new NonRecoverableException(statusIOE);
         }
 
       }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduScanner.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduScanner.java b/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduScanner.java
index 5e2d18c..e94ad7d 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduScanner.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduScanner.java
@@ -823,24 +823,27 @@ public final class AsyncKuduScanner {
       ScanResponsePB resp = builder.build();
       final byte[] id = resp.getScannerId().toByteArray();
       TabletServerErrorPB error = resp.hasError() ? resp.getError() : null;
+
       if (error != null && error.getCode().equals(TabletServerErrorPB.Code.TABLET_NOT_FOUND)) {
         if (state == State.OPENING) {
           // Doing this will trigger finding the new location.
           return new Pair<Response, Object>(null, error);
         } else {
-          throw new NonRecoverableException("Cannot continue scanning, " +
+          Status statusIncomplete = Status.Incomplete("Cannot continue scanning, " +
               "the tablet has moved and this isn't a fault tolerant scan");
+          throw new NonRecoverableException(statusIncomplete);
         }
       }
-      RowResultIterator iterator = new RowResultIterator(
+      RowResultIterator iterator = RowResultIterator.makeRowResultIterator(
           deadlineTracker.getElapsedMillis(), tsUUID, schema, resp.getData(),
           callResponse);
 
       boolean hasMore = resp.getHasMoreResults();
       if (id.length  != 0 && scannerId != null && !Bytes.equals(scannerId, id)) {
-        throw new InvalidResponseException("Scan RPC response was for scanner"
+        Status statusIllegalState = Status.IllegalState("Scan RPC response was for scanner"
             + " ID " + Bytes.pretty(id) + " but we expected "
-            + Bytes.pretty(scannerId), resp);
+            + Bytes.pretty(scannerId));
+        throw new NonRecoverableException(statusIllegalState);
       }
       Response response = new Response(id, iterator, hasMore);
       if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduSession.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduSession.java b/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduSession.java
index 400e46e..0290ee7 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduSession.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/AsyncKuduSession.java
@@ -487,8 +487,9 @@ public class AsyncKuduSession implements SessionConfiguration {
    * for the call (e.g looking up the tablet, etc).
    * @param operation operation to apply
    * @return a Deferred to track this operation
+   * @throws KuduException if an error happens or {@link PleaseThrottleException} is triggered
    */
-  public Deferred<OperationResponse> apply(final Operation operation) {
+  public Deferred<OperationResponse> apply(final Operation operation) throws KuduException {
     Preconditions.checkNotNull(operation, "Can not apply a null operation");
 
     // Freeze the row so that the client can not concurrently modify it while it is in flight.
@@ -519,9 +520,11 @@ public class AsyncKuduSession implements SessionConfiguration {
           if (inactiveBufferAvailable()) {
             refreshActiveBuffer();
           } else {
+            Status statusServiceUnavailable =
+                Status.ServiceUnavailable("All buffers are currently flushing");
             // This can happen if the user writes into a buffer, flushes it, writes
             // into the second, flushes it, and immediately tries to write again.
-            throw new PleaseThrottleException("All buffers are currently flushing",
+            throw new PleaseThrottleException(statusServiceUnavailable,
                                               null, operation, flushNotification.get());
           }
         }
@@ -530,8 +533,9 @@ public class AsyncKuduSession implements SessionConfiguration {
           if (activeBuffer.getOperations().size() < mutationBufferSpace) {
             activeBuffer.getOperations().add(new BufferedOperation(tablet, operation));
           } else {
-            throw new NonRecoverableException(
-                "MANUAL_FLUSH mode is enabled but the buffer is full");
+            Status statusIllegalState =
+                Status.IllegalState("MANUAL_FLUSH is enabled but the buffer is too big");
+            throw new NonRecoverableException(statusIllegalState);
           }
         } else {
           assert flushMode == FlushMode.AUTO_FLUSH_BACKGROUND;
@@ -546,7 +550,9 @@ public class AsyncKuduSession implements SessionConfiguration {
             if (inactiveBufferAvailable()) {
               refreshActiveBuffer();
             } else {
-              throw new PleaseThrottleException("All buffers are currently flushing",
+              Status statusServiceUnavailable =
+                  Status.ServiceUnavailable("All buffers are currently flushing");
+              throw new PleaseThrottleException(statusServiceUnavailable,
                                                 null, operation, flushNotification.get());
             }
           }
@@ -561,10 +567,11 @@ public class AsyncKuduSession implements SessionConfiguration {
                                                      mutationBufferLowWatermark);
 
             if (randomWatermark > mutationBufferSpace) {
-              throw new PleaseThrottleException(
-                  "The previous buffer hasn't been flushed and the " +
-                      "current buffer is over the low watermark, please retry later",
-                  null, operation, flushNotification.get());
+              Status statusServiceUnavailable =
+                  Status.ServiceUnavailable("The previous buffer hasn't been flushed and the " +
+                      "current buffer is over the low watermark, please retry later");
+              throw new PleaseThrottleException(statusServiceUnavailable,
+                                                null, operation, flushNotification.get());
             }
           }
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/ConnectionResetException.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/ConnectionResetException.java b/java/kudu-client/src/main/java/org/kududb/client/ConnectionResetException.java
deleted file mode 100644
index 2fa66a4..0000000
--- a/java/kudu-client/src/main/java/org/kududb/client/ConnectionResetException.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Copyright (C) 2010-2012  The Async HBase Authors.  All rights reserved.
- *
- * Redistribution and use in source and binary forms, with or without
- * modification, are permitted provided that the following conditions are met:
- *   - Redistributions of source code must retain the above copyright notice,
- *     this list of conditions and the following disclaimer.
- *   - Redistributions in binary form must reproduce the above copyright notice,
- *     this list of conditions and the following disclaimer in the documentation
- *     and/or other materials provided with the distribution.
- *   - Neither the name of the StumbleUpon nor the names of its contributors
- *     may be used to endorse or promote products derived from this software
- *     without specific prior written permission.
- * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
- * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
- * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
- * ARE DISCLAIMED.  IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
- * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
- * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
- * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
- * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
- * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
- * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
- * POSSIBILITY OF SUCH DAMAGE.
- */
-package org.kududb.client;
-
-import org.kududb.annotations.InterfaceAudience;
-import org.kududb.annotations.InterfaceStability;
-
-/**
- * Exception thrown when an RPC was in flight while we got disconnected.
- */
-@SuppressWarnings("serial")
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public final class ConnectionResetException extends RecoverableException {
-
-  ConnectionResetException(final String msg) {
-    super(msg);
-  }
-
-  /**
-   * Constructor.
-   */
-  ConnectionResetException(final String msg, final Exception cause) {
-    super(msg, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/GetMasterRegistrationReceived.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/GetMasterRegistrationReceived.java b/java/kudu-client/src/main/java/org/kududb/client/GetMasterRegistrationReceived.java
index c7159b3..f1a9075 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/GetMasterRegistrationReceived.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/GetMasterRegistrationReceived.java
@@ -16,6 +16,9 @@
 // under the License.
 package org.kududb.client;
 
+import com.google.common.base.Functions;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
 import com.google.common.net.HostAndPort;
 import com.google.protobuf.ByteString;
 import com.stumbleupon.async.Callback;
@@ -115,18 +118,28 @@ final class GetMasterRegistrationReceived {
         String allHosts = NetUtil.hostsAndPortsToString(masterAddrs);
         // Doing a negative check because allUnrecoverable stays true if there are no exceptions.
         if (!allUnrecoverable) {
+          String message = "Master config (" + allHosts + ") has no leader.";
+          Exception ex;
           if (exceptionsReceived.isEmpty()) {
             LOG.warn("None of the provided masters (" + allHosts + ") is a leader, will retry.");
+            ex = new NoLeaderMasterFoundException(Status.ServiceUnavailable(message));
           } else {
             LOG.warn("Unable to find the leader master (" + allHosts + "), will retry");
+            String joinedMsg = message + ". Exceptions received: " +
+                Joiner.on(",").join(
+                    Lists.transform(exceptionsReceived, Functions.toStringFunction()));
+            Status statusServiceUnavailable = Status.ServiceUnavailable(joinedMsg);
+            ex = new NoLeaderMasterFoundException(
+                statusServiceUnavailable,
+                exceptionsReceived.get(exceptionsReceived.size() - 1));
           }
-          responseD.callback(NoLeaderMasterFoundException.create(
-              "Master config (" + allHosts + ") has no leader.",
-              exceptionsReceived));
+          responseD.callback(ex);
         } else {
+          Status statusConfigurationError = Status.ConfigurationError(
+              "Couldn't find a valid master in (" + allHosts +
+                  "), exceptions: " + exceptionsReceived);
           // This will stop retries.
-          responseD.callback(new NonRecoverableException("Couldn't find a valid master in (" +
-              allHosts + "), exceptions: " + exceptionsReceived));
+          responseD.callback(new NonRecoverableException(statusConfigurationError));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/InvalidResponseException.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/InvalidResponseException.java b/java/kudu-client/src/main/java/org/kududb/client/InvalidResponseException.java
deleted file mode 100644
index 4221075..0000000
--- a/java/kudu-client/src/main/java/org/kududb/client/InvalidResponseException.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Copyright (C) 2010-2012  The Async HBase Authors.  All rights reserved.
- *
- * Redistribution and use in source and binary forms, with or without
- * modification, are permitted provided that the following conditions are met:
- *   - Redistributions of source code must retain the above copyright notice,
- *     this list of conditions and the following disclaimer.
- *   - Redistributions in binary form must reproduce the above copyright notice,
- *     this list of conditions and the following disclaimer in the documentation
- *     and/or other materials provided with the distribution.
- *   - Neither the name of the StumbleUpon nor the names of its contributors
- *     may be used to endorse or promote products derived from this software
- *     without specific prior written permission.
- * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
- * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
- * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
- * ARE DISCLAIMED.  IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
- * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
- * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
- * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
- * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
- * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
- * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
- * POSSIBILITY OF SUCH DAMAGE.
- */
-package org.kududb.client;
-
-import org.kududb.annotations.InterfaceAudience;
-import org.kududb.annotations.InterfaceStability;
-
-/**
- * Exception used when the server sends an invalid response to an RPC.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-@SuppressWarnings("serial")
-public final class InvalidResponseException extends NonRecoverableException {
-
-  private final Object response;
-
-  /**
-   * Constructor.
-   * @param msg The message of the exception, potentially including a stack
-   * trace.
-   * @param response The response that was received from the server.
-   */
-  InvalidResponseException(final String msg, final Object response) {
-    super(msg);
-    this.response = response;
-  }
-
-  /**
-   * Constructor.
-   * @param msg The message of the exception.
-   * @param cause The exception explaining why the response is invalid.
-   */
-  InvalidResponseException(final String msg, final Exception cause) {
-    super(msg, cause);
-    this.response = null;
-  }
-
-  /**
-   * Constructor for unexpected response types.
-   * @param expected The type of the response that was expected.
-   * @param response The response that was received from the server.
-   */
-  InvalidResponseException(final Class<?> expected, final Object response) {
-    super("Unexpected response type.  Expected: " + expected.getName()
-        + ", got: " + (response == null ? "null"
-        : response.getClass() + ", value=" + response));
-    this.response = response;
-  }
-
-  /**
-   * Returns the possibly {@code null} response received from the server.
-   */
-  public Object getResponse() {
-    return response;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/KuduClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/KuduClient.java b/java/kudu-client/src/main/java/org/kududb/client/KuduClient.java
index db432c8..cfd4662 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/KuduClient.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/KuduClient.java
@@ -53,21 +53,23 @@ public class KuduClient implements AutoCloseable {
    * @param schema the table's schema
    * @param builder a builder containing the table's configurations
    * @return an object to communicate with the created table
+   * @throws KuduException if anything went wrong
    */
   public KuduTable createTable(String name, Schema schema, CreateTableOptions builder)
-      throws Exception {
+      throws KuduException {
     Deferred<KuduTable> d = asyncClient.createTable(name, schema, builder);
-    return d.join(getDefaultAdminOperationTimeoutMs());
+    return joinAndHandleException(d);
   }
 
   /**
    * Delete a table on the cluster with the specified name.
    * @param name the table's name
    * @return an rpc response object
+   * @throws KuduException if anything went wrong
    */
-  public DeleteTableResponse deleteTable(String name) throws Exception {
+  public DeleteTableResponse deleteTable(String name) throws KuduException {
     Deferred<DeleteTableResponse> d = asyncClient.deleteTable(name);
-    return d.join(getDefaultAdminOperationTimeoutMs());
+    return joinAndHandleException(d);
   }
 
   /**
@@ -78,10 +80,11 @@ public class KuduClient implements AutoCloseable {
    * @param name the table's name, if this is a table rename then the old table name must be passed
    * @param ato the alter table builder
    * @return an rpc response object
+   * @throws KuduException if anything went wrong
    */
-  public AlterTableResponse alterTable(String name, AlterTableOptions ato) throws Exception {
+  public AlterTableResponse alterTable(String name, AlterTableOptions ato) throws KuduException {
     Deferred<AlterTableResponse> d = asyncClient.alterTable(name, ato);
-    return d.join(getDefaultAdminOperationTimeoutMs());
+    return joinAndHandleException(d);
   }
 
   /**
@@ -89,38 +92,39 @@ public class KuduClient implements AutoCloseable {
    * It will block until the alter command is done or the timeout is reached.
    * @param name Table's name, if the table was renamed then that name must be checked against
    * @return a boolean indicating if the table is done being altered
+   * @throws KuduException for any error returned by sending RPCs to the master
    */
-  public boolean isAlterTableDone(String name) throws Exception {
+  public boolean isAlterTableDone(String name) throws KuduException {
     long totalSleepTime = 0;
     while (totalSleepTime < getDefaultAdminOperationTimeoutMs()) {
       long start = System.currentTimeMillis();
 
-      Deferred<IsAlterTableDoneResponse> d = asyncClient.isAlterTableDone(name);
-      IsAlterTableDoneResponse response;
       try {
+        Deferred<IsAlterTableDoneResponse> d = asyncClient.isAlterTableDone(name);
+        IsAlterTableDoneResponse response;
+
         response = d.join(AsyncKuduClient.SLEEP_TIME);
+        if (response.isDone()) {
+          return true;
+        }
+
+        // Count time that was slept and see if we need to wait a little more.
+        long elapsed = System.currentTimeMillis() - start;
+        // Don't oversleep the deadline.
+        if (totalSleepTime + AsyncKuduClient.SLEEP_TIME > getDefaultAdminOperationTimeoutMs()) {
+          return false;
+        }
+        // elapsed can be bigger if we slept about 500ms
+        if (elapsed <= AsyncKuduClient.SLEEP_TIME) {
+          LOG.debug("Alter not done, sleep " + (AsyncKuduClient.SLEEP_TIME - elapsed) +
+              " and slept " + totalSleepTime);
+          Thread.sleep(AsyncKuduClient.SLEEP_TIME - elapsed);
+          totalSleepTime += AsyncKuduClient.SLEEP_TIME;
+        } else {
+          totalSleepTime += elapsed;
+        }
       } catch (Exception ex) {
-        throw ex;
-      }
-
-      if (response.isDone()) {
-        return true;
-      }
-
-      // Count time that was slept and see if we need to wait a little more.
-      long elapsed = System.currentTimeMillis() - start;
-      // Don't oversleep the deadline.
-      if (totalSleepTime + AsyncKuduClient.SLEEP_TIME > getDefaultAdminOperationTimeoutMs()) {
-        return false;
-      }
-      // elapsed can be bigger if we slept about 500ms
-      if (elapsed <= AsyncKuduClient.SLEEP_TIME) {
-        LOG.debug("Alter not done, sleep " + (AsyncKuduClient.SLEEP_TIME - elapsed) +
-            " and slept " + totalSleepTime);
-        Thread.sleep(AsyncKuduClient.SLEEP_TIME - elapsed);
-        totalSleepTime += AsyncKuduClient.SLEEP_TIME;
-      } else {
-        totalSleepTime += elapsed;
+        throw KuduException.transformException(ex);
       }
     }
     return false;
@@ -129,17 +133,19 @@ public class KuduClient implements AutoCloseable {
   /**
    * Get the list of running tablet servers.
    * @return a list of tablet servers
+   * @throws KuduException if anything went wrong
    */
-  public ListTabletServersResponse listTabletServers() throws Exception {
+  public ListTabletServersResponse listTabletServers() throws KuduException {
     Deferred<ListTabletServersResponse> d = asyncClient.listTabletServers();
-    return d.join(getDefaultAdminOperationTimeoutMs());
+    return joinAndHandleException(d);
   }
 
   /**
    * Get the list of all the tables.
    * @return a list of all the tables
+   * @throws KuduException if anything went wrong
    */
-  public ListTablesResponse getTablesList() throws Exception {
+  public ListTablesResponse getTablesList() throws KuduException {
     return getTablesList(null);
   }
 
@@ -148,31 +154,34 @@ public class KuduClient implements AutoCloseable {
    * specified, it only returns tables that satisfy a substring match.
    * @param nameFilter an optional table name filter
    * @return a deferred that contains the list of table names
+   * @throws KuduException if anything went wrong
    */
-  public ListTablesResponse getTablesList(String nameFilter) throws Exception {
+  public ListTablesResponse getTablesList(String nameFilter) throws KuduException {
     Deferred<ListTablesResponse> d = asyncClient.getTablesList(nameFilter);
-    return d.join(getDefaultAdminOperationTimeoutMs());
+    return joinAndHandleException(d);
   }
 
   /**
    * Test if a table exists.
    * @param name a non-null table name
    * @return true if the table exists, else false
+   * @throws KuduException if anything went wrong
    */
-  public boolean tableExists(String name) throws Exception {
+  public boolean tableExists(String name) throws KuduException {
     Deferred<Boolean> d = asyncClient.tableExists(name);
-    return d.join(getDefaultAdminOperationTimeoutMs());
+    return joinAndHandleException(d);
   }
 
   /**
    * Open the table with the given name. If the table was just created, this method will block until
    * all its tablets have also been created.
    * @param name table to open
-   * @return a KuduTable if the table exists, else a MasterErrorException
+   * @return a KuduTable if the table exists
+   * @throws KuduException if anything went wrong
    */
-  public KuduTable openTable(final String name) throws Exception {
+  public KuduTable openTable(final String name) throws KuduException {
     Deferred<KuduTable> d = asyncClient.openTable(name);
-    return d.join(getDefaultAdminOperationTimeoutMs());
+    return joinAndHandleException(d);
   }
 
   /**
@@ -226,20 +235,24 @@ public class KuduClient implements AutoCloseable {
 
   /**
    * Analogous to {@link #shutdown()}.
-   * @throws Exception if an error happens while closing the connections
+   * @throws KuduException if an error happens while closing the connections
    */
   @Override
-  public void close() throws Exception {
-    asyncClient.close();
+  public void close() throws KuduException {
+    try {
+      asyncClient.close();
+    } catch (Exception e) {
+      KuduException.transformException(e);
+    }
   }
 
   /**
    * Performs a graceful shutdown of this instance.
-   * @throws Exception
+   * @throws KuduException if anything went wrong
    */
-  public void shutdown() throws Exception {
+  public void shutdown() throws KuduException {
     Deferred<ArrayList<Void>> d = asyncClient.shutdown();
-    d.join(getDefaultAdminOperationTimeoutMs());
+    joinAndHandleException(d);
   }
 
   /**
@@ -258,6 +271,15 @@ public class KuduClient implements AutoCloseable {
     return asyncClient.getDefaultAdminOperationTimeoutMs();
   }
 
+  // Helper method to handle joining and transforming the Exception we receive.
+  private <R> R joinAndHandleException(Deferred<R> deferred) throws KuduException {
+    try {
+      return deferred.join(getDefaultAdminOperationTimeoutMs());
+    } catch (Exception e) {
+      throw KuduException.transformException(e);
+    }
+  }
+
   /**
    * Builder class to use in order to connect to Kudu.
    * All the parameters beyond those in the constructors are optional.

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/KuduException.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/KuduException.java b/java/kudu-client/src/main/java/org/kududb/client/KuduException.java
index 726649b..4bd2eaf 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/KuduException.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/KuduException.java
@@ -25,45 +25,75 @@
  */
 package org.kududb.client;
 
+import com.stumbleupon.async.DeferredGroupException;
+import com.stumbleupon.async.TimeoutException;
 import org.kududb.annotations.InterfaceAudience;
 import org.kududb.annotations.InterfaceStability;
 
+import java.io.IOException;
+
 /**
- * The parent class of all {@link RuntimeException} created by this package.
+ * The parent class of all exceptions sent by the Kudu client. This is the only exception you will
+ * see if you're using the non-async API, such as {@link KuduSession} instead of
+ * {@link AsyncKuduSession}.
+ *
+ * Each instance of this class has a {@link Status} which gives more information about the error.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 @SuppressWarnings("serial")
-public abstract class KuduException extends RuntimeException {
+public abstract class KuduException extends IOException {
+
+  private final Status status;
 
   /**
    * Constructor.
-   * @param msg The message of the exception, potentially including a stack
+   * @param status object containing the reason for the exception
    * trace.
    */
-  KuduException(final String msg) {
-    super(msg);
+  KuduException(Status status) {
+    super(status.getMessage());
+    this.status = status;
   }
 
   /**
    * Constructor.
-   * @param msg The message of the exception, potentially including a stack
-   * trace.
+   * @param status object containing the reason for the exception
    * @param cause The exception that caused this one to be thrown.
    */
-  KuduException(final String msg, final Throwable cause) {
-    super(msg, cause);
+  KuduException(Status status, Throwable cause) {
+    super(status.getMessage(), cause);
+    this.status = status;
+  }
+
+  /**
+   * Get the Status object for this exception.
+   * @return a status object indicating the reason for the exception
+   */
+  public Status getStatus() {
+    return status;
   }
 
   /**
-   * Factory method to make it possible to create an exception from another
-   * one without having to resort to reflection, which is annoying to use.
-   * Sub-classes that want to provide this internal functionality should
-   * implement this method.
-   * @param arg Some arbitrary parameter to help build the new instance.
-   * @param rpc The RPC that failed, if any.  Can be {@code null}.
+   * Inspects the given exception and transforms it into a KuduException.
+   * @param e generic exception we want to transform
+   * @return a KuduException that's easier to handle
    */
-  KuduException make(final Object arg, final KuduRpc<?> rpc) {
-    throw new AssertionError("Must not be used.");
+  static KuduException transformException(Exception e) {
+    if (e instanceof KuduException) {
+      return (KuduException) e;
+    } else if (e instanceof DeferredGroupException) {
+      // TODO anything we can do to improve on that kind of exception?
+    } else if (e instanceof TimeoutException) {
+      Status statusTimeout = Status.TimedOut(e.getMessage());
+      return new NonRecoverableException(statusTimeout, e);
+    } else if (e instanceof InterruptedException) {
+      // Need to reset the interrupt flag since we caught it but aren't handling it.
+      Thread.currentThread().interrupt();
+      Status statusAborted = Status.Aborted(e.getMessage());
+      return new NonRecoverableException(statusAborted, e);
+    }
+    Status status = Status.IOError(e.getMessage());
+    return new NonRecoverableException(status, e);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/KuduRpc.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/KuduRpc.java b/java/kudu-client/src/main/java/org/kududb/client/KuduRpc.java
index 1fe2e6e..e7493a2 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/KuduRpc.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/KuduRpc.java
@@ -269,13 +269,12 @@ public abstract class KuduRpc<R> {
     try {
       builder.mergeFrom(payload, offset, length);
       if (!builder.isInitialized()) {
-        throw new InvalidResponseException("Could not deserialize the response," +
-            " incompatible RPC? Error is: " + builder.getInitializationErrorString(), null);
+        throw new RuntimeException("Could not deserialize the response," +
+            " incompatible RPC? Error is: " + builder.getInitializationErrorString());
       }
     } catch (InvalidProtocolBufferException e) {
-      final String msg = "Invalid RPC response: length=" + length
-          + ", payload=" + Bytes.pretty(payload);
-      throw new InvalidResponseException(msg, e);
+      throw new RuntimeException("Invalid RPC response: length=" + length +
+            ", payload=" + Bytes.pretty(payload));
     }
   }
 
@@ -294,7 +293,7 @@ public abstract class KuduRpc<R> {
       pb.writeTo(out);
       out.checkNoSpaceLeft();
     } catch (IOException e) {
-      throw new NonRecoverableException("Cannot serialize the following message " + pb, e);
+      throw new RuntimeException("Cannot serialize the following message " + pb);
     }
     chanBuf.writerIndex(buf.length);
     return chanBuf;

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/KuduScanner.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/KuduScanner.java b/java/kudu-client/src/main/java/org/kududb/client/KuduScanner.java
index d2d2e8e..87db768 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/KuduScanner.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/KuduScanner.java
@@ -51,10 +51,15 @@ public class KuduScanner {
    * {@code Scanner} is done scanning), calling it again leads to an undefined
    * behavior.
    * @return a list of rows.
+   * @throws KuduException if anything went wrong
    */
-  public RowResultIterator nextRows() throws Exception {
+  public RowResultIterator nextRows() throws KuduException {
     Deferred<RowResultIterator> d = asyncScanner.nextRows();
-    return d.join(asyncScanner.scanRequestTimeout);
+    try {
+      return d.join(asyncScanner.scanRequestTimeout);
+    } catch (Exception e) {
+      throw KuduException.transformException(e);
+    }
   }
 
   /**
@@ -62,10 +67,15 @@ public class KuduScanner {
    * <p>
    * Closing a scanner already closed has no effect.
    * @return a deferred object that indicates the completion of the request
+   * @throws KuduException if anything went wrong
    */
-  public RowResultIterator close() throws Exception {
+  public RowResultIterator close() throws KuduException {
     Deferred<RowResultIterator> d = asyncScanner.close();
-    return d.join(asyncScanner.scanRequestTimeout);
+    try {
+      return d.join(asyncScanner.scanRequestTimeout);
+    } catch (Exception e) {
+      throw KuduException.transformException(e);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/KuduServerException.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/KuduServerException.java b/java/kudu-client/src/main/java/org/kududb/client/KuduServerException.java
deleted file mode 100644
index db12c94..0000000
--- a/java/kudu-client/src/main/java/org/kududb/client/KuduServerException.java
+++ /dev/null
@@ -1,47 +0,0 @@
-// 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.kududb.client;
-
-import org.kududb.WireProtocol;
-import org.kududb.annotations.InterfaceAudience;
-import org.kududb.annotations.InterfaceStability;
-import org.kududb.rpc.RpcHeader;
-
-/**
- * This class is used for errors sent in response to a RPC.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-@SuppressWarnings("serial")
-public class KuduServerException extends KuduException {
-
-  KuduServerException(String serverUuid, RpcHeader.ErrorStatusPB errorStatus) {
-    this(serverUuid, errorStatus.getMessage(), errorStatus.getCode().toString(),
-        errorStatus.getCode().getNumber(), null);
-  }
-
-  KuduServerException(String serverUuid, WireProtocol.AppStatusPB appStatus) {
-    this(serverUuid, appStatus.getMessage(), appStatus.getCode().toString(),
-        appStatus.getCode().getNumber(), null);
-  }
-
-  KuduServerException(String serverUuid, String message, String errorDesc,
-                      int errCode, Throwable cause) {
-    super("Server[" + serverUuid + "] "
-        + errorDesc + "[code " + errCode + "]: "  + message, cause);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/KuduSession.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/KuduSession.java b/java/kudu-client/src/main/java/org/kududb/client/KuduSession.java
index f13f9c6..61718b2 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/KuduSession.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/KuduSession.java
@@ -16,6 +16,8 @@
 // under the License.
 package org.kududb.client;
 
+import com.stumbleupon.async.DeferredGroupException;
+import com.stumbleupon.async.TimeoutException;
 import org.kududb.annotations.*;
 
 import com.stumbleupon.async.Deferred;
@@ -59,14 +61,14 @@ public class KuduSession implements SessionConfiguration {
    * errors can be checked by calling {@link #countPendingErrors()} and can be retrieved by calling
    * {@link #getPendingErrors()}.
    * <li>MANUAL_FLUSH: the call returns when the operation has been added to the buffer,
-   * else it throws an exception such as a NonRecoverableException if the buffer is full.
+   * else it throws a KuduException if the buffer is full.
    * </ul>
    *
    * @param operation operation to apply
    * @return an OperationResponse for the applied Operation
-   * @throws Exception if anything went wrong
+   * @throws KuduException if anything went wrong
    */
-  public OperationResponse apply(Operation operation) throws Exception {
+  public OperationResponse apply(Operation operation) throws KuduException {
     while (true) {
       try {
         Deferred<OperationResponse> d = session.apply(operation);
@@ -83,7 +85,7 @@ public class KuduSession implements SessionConfiguration {
           LOG.error("Previous batch had this exception", e);
         }
       } catch (Exception e) {
-        throw e;
+        throw KuduException.transformException(e);
       }
     }
     return null;
@@ -93,21 +95,27 @@ public class KuduSession implements SessionConfiguration {
    * Blocking call that force flushes this session's buffers. Data is persisted when this call
    * returns, else it will throw an exception.
    * @return a list of OperationResponse, one per operation that was flushed
-   * @throws Exception if anything went wrong. If it's an issue with some or all batches,
-   * it will be of type DeferredGroupException.
+   * @throws KuduException if anything went wrong
    */
-  public List<OperationResponse> flush() throws Exception {
-    return session.flush().join(getTimeoutMillis());
+  public List<OperationResponse> flush() throws KuduException {
+    try {
+      return session.flush().join(getTimeoutMillis());
+    } catch (Exception e) {
+      throw KuduException.transformException(e);
+    }
   }
 
   /**
    * Blocking call that flushes the buffers (see {@link #flush()} and closes the sessions.
    * @return List of OperationResponse, one per operation that was flushed
-   * @throws Exception if anything went wrong. If it's an issue with some or all batches,
-   * it will be of type DeferredGroupException.
+   * @throws KuduException if anything went wrong
    */
-  public List<OperationResponse> close() throws Exception {
-    return session.close().join(getTimeoutMillis());
+  public List<OperationResponse> close() throws KuduException {
+    try {
+      return session.close().join(getTimeoutMillis());
+    } catch (Exception e) {
+      throw KuduException.transformException(e);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/MasterErrorException.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/MasterErrorException.java b/java/kudu-client/src/main/java/org/kududb/client/MasterErrorException.java
deleted file mode 100644
index 2bbb7a5..0000000
--- a/java/kudu-client/src/main/java/org/kududb/client/MasterErrorException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-// 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.kududb.client;
-
-import org.kududb.annotations.InterfaceAudience;
-import org.kududb.annotations.InterfaceStability;
-import org.kududb.master.Master;
-import org.kududb.rpc.RpcHeader;
-
-/**
- * This exception is thrown when a Master responds to an RPC with an error message
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-@SuppressWarnings("serial")
-public class MasterErrorException extends KuduServerException {
-
-  MasterErrorException(String serverUuid, RpcHeader.ErrorStatusPB errorStatus) {
-    super(serverUuid, errorStatus);
-  }
-
-  MasterErrorException(String serverUuid, Master.MasterErrorPB error) {
-    super(serverUuid, error.getStatus());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/NoLeaderMasterFoundException.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/NoLeaderMasterFoundException.java b/java/kudu-client/src/main/java/org/kududb/client/NoLeaderMasterFoundException.java
index 3cc502e..1cde694 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/NoLeaderMasterFoundException.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/NoLeaderMasterFoundException.java
@@ -16,9 +16,6 @@
 // under the License.
 package org.kududb.client;
 
-import com.google.common.base.Functions;
-import com.google.common.base.Joiner;
-import com.google.common.collect.Lists;
 import org.kududb.annotations.InterfaceAudience;
 import org.kududb.annotations.InterfaceStability;
 
@@ -27,35 +24,14 @@ import java.util.List;
 /**
  * Indicates that the request failed because we couldn't find a leader master server.
  */
-@InterfaceAudience.Public
+@InterfaceAudience.Private
 @InterfaceStability.Evolving
-public final class NoLeaderMasterFoundException extends RecoverableException {
+final class NoLeaderMasterFoundException extends RecoverableException {
 
-  NoLeaderMasterFoundException(final String msg) {
-    super(msg);
+  NoLeaderMasterFoundException(Status status) {
+    super(status);
   }
-  NoLeaderMasterFoundException(final String msg, final Exception cause) {
-    super(msg, cause);
-  }
-
-  /**
-   * Factory method that creates a NoLeaderException given a message and a list
-   * (which may be empty, but must be initialized) of exceptions encountered: they indicate
-   * why {@link GetMasterRegistrationRequest} calls to the masters in the config
-   * have failed, to aid in debugging the issue. If the list is non-empty, each exception's
-   * 'toString()' message is appended to 'msg' and the last exception is used as the
-   * cause for the exception.
-   * @param msg A message detailing why this exception occured.
-   * @param causes List of exceptions encountered when retrieving registration from individual
-   *               masters.
-   * @return An instantiated NoLeaderMasterFoundException which can be thrown.
-   */
-  static NoLeaderMasterFoundException create(String msg, List<Exception> causes) {
-    if (causes.isEmpty()) {
-      return new NoLeaderMasterFoundException(msg);
-    }
-    String joinedMsg = msg + ". Exceptions received: " +
-        Joiner.on(",").join(Lists.transform(causes, Functions.toStringFunction()));
-    return new NoLeaderMasterFoundException(joinedMsg, causes.get(causes.size() - 1));
+  NoLeaderMasterFoundException(Status status, Exception cause) {
+    super(status, cause);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/NonCoveredRangeException.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/NonCoveredRangeException.java b/java/kudu-client/src/main/java/org/kududb/client/NonCoveredRangeException.java
index 3400a51..b704441 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/NonCoveredRangeException.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/NonCoveredRangeException.java
@@ -22,14 +22,13 @@ import org.kududb.annotations.InterfaceStability;
 /**
  * Exception indicating that an operation attempted to access a non-covered range partition.
  */
-@InterfaceAudience.Public
-@InterfaceStability.Unstable
-public class NonCoveredRangeException extends KuduException {
+@InterfaceAudience.Private
+class NonCoveredRangeException extends NonRecoverableException {
   private final byte[] nonCoveredRangeStart;
   private final byte[] nonCoveredRangeEnd;
 
   public NonCoveredRangeException(byte[] nonCoveredRangeStart, byte[] nonCoveredRangeEnd) {
-    super("non-covered range");
+    super(Status.NotFound("non-covered range"));
     this.nonCoveredRangeStart = nonCoveredRangeStart;
     this.nonCoveredRangeEnd = nonCoveredRangeEnd;
   }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/NonRecoverableException.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/NonRecoverableException.java b/java/kudu-client/src/main/java/org/kududb/client/NonRecoverableException.java
index a11a05e..7bcb81d 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/NonRecoverableException.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/NonRecoverableException.java
@@ -28,27 +28,26 @@ package org.kududb.client;
 import org.kududb.annotations.InterfaceAudience;
 import org.kududb.annotations.InterfaceStability;
 
-@InterfaceAudience.Public
+@InterfaceAudience.Private
 @InterfaceStability.Evolving
 @SuppressWarnings("serial")
-public class NonRecoverableException extends KuduException {
+class NonRecoverableException extends KuduException {
 
   /**
    * Constructor.
-   * @param msg The message of the exception, potentially including a stack
+   * @param status status object containing the reason for the exception
    * trace.
    */
-  NonRecoverableException(final String msg) {
-    super(msg);
+  NonRecoverableException(Status status) {
+    super(status);
   }
 
   /**
    * Constructor.
-   * @param msg The message of the exception, potentially including a stack
-   * trace.
+   * @param status status object containing the reason for the exception
    * @param cause The exception that caused this one to be thrown.
    */
-  NonRecoverableException(final String msg, final Throwable cause) {
-    super(msg, cause);
+  NonRecoverableException(Status status, Throwable cause) {
+    super(status, cause);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/PleaseThrottleException.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/PleaseThrottleException.java b/java/kudu-client/src/main/java/org/kududb/client/PleaseThrottleException.java
index dc84e60..3ca98e2 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/PleaseThrottleException.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/PleaseThrottleException.java
@@ -61,7 +61,7 @@ import org.kududb.annotations.InterfaceStability;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 @SuppressWarnings("serial")
-public final class PleaseThrottleException extends NonRecoverableException
+public final class PleaseThrottleException extends RecoverableException
     implements HasFailedRpcException {
 
   /** The RPC that was failed with this exception.  */
@@ -72,17 +72,17 @@ public final class PleaseThrottleException extends NonRecoverableException
 
   /**
    * Constructor.
-   * @param msg A message explaining why the application has to throttle.
+   * @param status status object containing the reason for the exception
    * @param cause The exception that requires the application to throttle
-   * itself (can be {@code null}).
-   * @param rpc The RPC that was made to fail with this exception.
-   * @param deferred A deferred one can wait on before retrying the failed RPC.
+   * itself (can be {@code null})
+   * @param rpc The RPC that was made to fail with this exception
+   * @param deferred A deferred one can wait on before retrying the failed RPC
    */
-  PleaseThrottleException(final String msg,
-                          final KuduException cause,
-                          final Operation rpc,
-                          final Deferred deferred) {
-    super(msg, cause);
+  PleaseThrottleException(Status status,
+                          KuduException cause,
+                          Operation rpc,
+                          Deferred deferred) {
+    super(status, cause);
     this.rpc = rpc;
     this.deferred = deferred;
   }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/RecoverableException.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/RecoverableException.java b/java/kudu-client/src/main/java/org/kududb/client/RecoverableException.java
index 50ff2ac..25c0fe0 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/RecoverableException.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/RecoverableException.java
@@ -29,33 +29,28 @@ import org.kududb.annotations.InterfaceAudience;
 import org.kududb.annotations.InterfaceStability;
 
 /**
- * An exception for which it's typically useful to retry
- * <p>
- * The retry strategy is up to you, but it's typically recommended to put an
- * upper bound on the number of retries and to use some kind of an exponential
- * backoff.
+ * An exception that's possible to retry.
  */
-@InterfaceAudience.Public
+@InterfaceAudience.Private
 @InterfaceStability.Evolving
 @SuppressWarnings("serial")
-public abstract class RecoverableException extends KuduException {
+class RecoverableException extends KuduException {
 
   /**
    * Constructor.
-   * @param msg The message of the exception, potentially including a stack
+   * @param status status object containing the reason for the exception
    * trace.
    */
-  RecoverableException(final String msg) {
-    super(msg);
+  RecoverableException(Status status) {
+    super(status);
   }
 
   /**
    * Constructor.
-   * @param msg The message of the exception, potentially including a stack
-   * trace.
+   * @param status status object containing the reason for the exception
    * @param cause The exception that caused this one to be thrown.
    */
-  RecoverableException(final String msg, final Exception cause) {
-    super(msg, cause);
+  RecoverableException(Status status, Throwable cause) {
+    super(status, cause);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/RowResultIterator.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/RowResultIterator.java b/java/kudu-client/src/main/java/org/kududb/client/RowResultIterator.java
index a3c6941..5705ea3 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/RowResultIterator.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/RowResultIterator.java
@@ -32,7 +32,8 @@ import org.kududb.util.Slice;
 public class RowResultIterator extends KuduRpcResponse implements Iterator<RowResult>,
     Iterable<RowResult> {
 
-  private static final RowResultIterator EMPTY = new RowResultIterator(0, null, null, null, null);
+  private static final RowResultIterator EMPTY =
+      new RowResultIterator(0, null, null, 0, null, null);
 
   private final Schema schema;
   private final Slice bs;
@@ -43,35 +44,46 @@ public class RowResultIterator extends KuduRpcResponse implements Iterator<RowRe
 
   /**
    * Package private constructor, only meant to be instantiated from AsyncKuduScanner.
-   * @param ellapsedMillis Time in milliseconds since RPC creation to now.
-   * @param schema Schema used to parse the rows
-   * @param data PB containing the data
-   * @param callResponse the call response received from the server for this
-   * RPC.
+   * @param ellapsedMillis ime in milliseconds since RPC creation to now
+   * @param tsUUID UUID of the tablet server that handled our request
+   * @param schema schema used to parse the rows
+   * @param numRows how many rows are contained in the bs slice
+   * @param bs normal row data
+   * @param indirectBs indirect row data
    */
-  RowResultIterator(long ellapsedMillis, String tsUUID, Schema schema,
-                    WireProtocol.RowwiseRowBlockPB data,
-                    final CallResponse callResponse) {
+  private RowResultIterator(long ellapsedMillis, String tsUUID, Schema schema,
+                            int numRows, Slice bs, Slice indirectBs) {
     super(ellapsedMillis, tsUUID);
     this.schema = schema;
+    this.bs = bs;
+    this.indirectBs = indirectBs;
+    this.numRows = numRows;
+
+    this.rowResult = numRows == 0 ? null : new RowResult(this.schema, this.bs, this.indirectBs);
+  }
+
+  static RowResultIterator makeRowResultIterator(long ellapsedMillis, String tsUUID,
+                                                 Schema schema,
+                                                 WireProtocol.RowwiseRowBlockPB data,
+                                                 final CallResponse callResponse)
+      throws KuduException {
     if (data == null || data.getNumRows() == 0) {
-      this.bs = this.indirectBs = null;
-      this.rowResult = null;
-      this.numRows = 0;
-      return;
+      return new RowResultIterator(ellapsedMillis, tsUUID, schema, 0, null, null);
     }
-    this.bs = callResponse.getSidecar(data.getRowsSidecar());
-    this.indirectBs = callResponse.getSidecar(data.getIndirectDataSidecar());
-    this.numRows = data.getNumRows();
+
+    Slice bs = callResponse.getSidecar(data.getRowsSidecar());
+    Slice indirectBs = callResponse.getSidecar(data.getIndirectDataSidecar());
+    int numRows = data.getNumRows();
 
     // Integrity check
     int rowSize = schema.getRowSize();
     int expectedSize = numRows * rowSize;
     if (expectedSize != bs.length()) {
-      throw new NonRecoverableException("RowResult block has " + bs.length() + " bytes of data " +
-          "but expected " + expectedSize + " for " + numRows + " rows");
+      Status statusIllegalState = Status.IllegalState("RowResult block has " + bs.length() +
+          " bytes of data but expected " + expectedSize + " for " + numRows + " rows");
+      throw new NonRecoverableException(statusIllegalState);
     }
-    this.rowResult = new RowResult(this.schema, this.bs, this.indirectBs);
+    return new RowResultIterator(ellapsedMillis, tsUUID, schema, numRows, bs, indirectBs);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/Status.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/Status.java b/java/kudu-client/src/main/java/org/kududb/client/Status.java
index d2ce300..cd0a17d 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/Status.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/Status.java
@@ -19,16 +19,20 @@ package org.kududb.client;
 import org.kududb.WireProtocol;
 import org.kududb.annotations.InterfaceAudience;
 import org.kududb.annotations.InterfaceStability;
+import org.kududb.master.Master;
+import org.kududb.tserver.Tserver;
 
 /**
- * Representation of an error code and message. Wraps {@link org.kududb.WireProtocol.AppStatusPB}.
+ * Representation of an error code and message.
  * See also {@code src/kudu/util/status.h} in the C++ codebase.
- *
- * <p>Do not use the {@code @deprecated} methods in this class.</p>
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class Status {
+
+  // Keep a single OK status object else we'll end up instantiating tons of them.
+  private static final Status STATIC_OK = new Status(WireProtocol.AppStatusPB.ErrorCode.OK);
+
   private final WireProtocol.AppStatusPB appStatusPB;
 
   private Status(WireProtocol.AppStatusPB appStatusPB) {
@@ -55,7 +59,33 @@ public class Status {
   // Factory methods.
 
   /**
-   * Create a Status object from a {@link org.kududb.WireProtocol.AppStatusPB} protobuf object.
+   * Create a status object from a master error.
+   * @param masterErrorPB pb object received via RPC from the master
+   * @return status object equivalent to the pb
+   */
+  static Status fromMasterErrorPB(Master.MasterErrorPB masterErrorPB) {
+    if (masterErrorPB == Master.MasterErrorPB.getDefaultInstance()) {
+      return Status.OK();
+    } else {
+      return new Status(masterErrorPB.getStatus());
+    }
+  }
+
+  /**
+   * Create a status object from a tablet server error.
+   * @param tserverErrorPB pb object received via RPC from the TS
+   * @return status object equivalent to the pb
+   */
+  static Status fromTabletServerErrorPB(Tserver.TabletServerErrorPB tserverErrorPB) {
+    if (tserverErrorPB == Tserver.TabletServerErrorPB.getDefaultInstance()) {
+      return Status.OK();
+    } else {
+      return new Status(tserverErrorPB.getStatus());
+    }
+  }
+
+  /**
+   * Create a Status object from a {@link WireProtocol.AppStatusPB} protobuf object.
    * Package-private because we shade Protobuf and this is not usable outside this package.
    */
   static Status fromPB(WireProtocol.AppStatusPB pb) {
@@ -63,7 +93,7 @@ public class Status {
   }
 
   public static Status OK() {
-    return new Status(WireProtocol.AppStatusPB.ErrorCode.OK);
+    return STATIC_OK;
   }
 
   public static Status NotFound(String msg) {

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/TabletClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/TabletClient.java b/java/kudu-client/src/main/java/org/kududb/client/TabletClient.java
index d3ecf8f..8f6a91f 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/TabletClient.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/TabletClient.java
@@ -160,8 +160,9 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
       if (!rpc.getRequiredFeatures().isEmpty() &&
           !secureRpcHelper.getServerFeatures().contains(
               RpcHeader.RpcFeatureFlag.APPLICATION_FEATURE_FLAGS)) {
-        rpc.errback(new NonRecoverableException(
-            "the server does not support the APPLICATION_FEATURE_FLAGS RPC feature"));
+        Status statusNotSupported = Status.NotSupported("the server does not support the" +
+            "APPLICATION_FEATURE_FLAGS RPC feature");
+        rpc.errback(new NonRecoverableException(statusNotSupported));
       }
 
       encodedRpcAndId = encode(rpc);
@@ -200,7 +201,9 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
     }
 
     if (failRpc) {
-      failOrRetryRpc(rpc, new ConnectionResetException(null));
+      Status statusNetworkError =
+          Status.NetworkError(getPeerUuidLoggingString() + "Connection reset on " + chan);
+      failOrRetryRpc(rpc, new RecoverableException(statusNetworkError));
     } else if (tryAgain) {
       // This recursion will not lead to a loop because we only get here if we
       // connected while entering the synchronized block above. So when trying
@@ -208,7 +211,6 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
       // connected, or fail through to the code below if we got disconnected
       // in the mean time.
       sendRpc(rpc);
-      return;
     }
   }
 
@@ -253,8 +255,9 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
           + " rpcid=" + rpcid + ": " + oldrpc
           + ".  This happened when sending out: " + rpc;
       LOG.error(wtf);
+      Status statusIllegalState = Status.IllegalState(wtf);
       // Make it fail. This isn't an expected failure mode.
-      oldrpc.errback(new NonRecoverableException(wtf));
+      oldrpc.errback(new NonRecoverableException(statusIllegalState));
     }
 
     if (LOG.isDebugEnabled()) {
@@ -284,11 +287,13 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
    * @return deferred object to use to track the shutting down of this connection
    */
   public Deferred<Void> shutdown() {
+    Status statusNetworkError =
+        Status.NetworkError(getPeerUuidLoggingString() + "Client is shutting down");
+    NonRecoverableException exception = new NonRecoverableException(statusNetworkError);
     // First, check whether we have RPCs in flight and cancel them.
     for (Iterator<KuduRpc<?>> ite = rpcs_inflight.values().iterator(); ite
         .hasNext();) {
-      KuduRpc<?> rpc = ite.next();
-      rpc.errback(new ConnectionResetException(null));
+      ite.next().errback(exception);
       ite.remove();
     }
 
@@ -296,7 +301,7 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
     synchronized (this) {
       if (pending_rpcs != null) {
         for (Iterator<KuduRpc<?>> ite = pending_rpcs.iterator(); ite.hasNext();) {
-          ite.next().errback(new ConnectionResetException(null));
+          ite.next().errback(exception);
           ite.remove();
         }
       }
@@ -335,8 +340,9 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
           } else {
             // Wrap the Throwable because Deferred doesn't handle Throwables,
             // it only uses Exception.
-            d.callback(new NonRecoverableException("Failed to shutdown: "
-                + TabletClient.this, t));
+            Status statusIllegalState = Status.IllegalState("Failed to shutdown: " +
+                TabletClient.this);
+            d.callback(new NonRecoverableException(statusIllegalState, t));
           }
         }
       });
@@ -353,7 +359,7 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
   @Override
   @SuppressWarnings("unchecked")
   protected Object decode(ChannelHandlerContext ctx, Channel chan, ChannelBuffer buf,
-                              VoidEnum voidEnum) {
+                              VoidEnum voidEnum) throws NonRecoverableException {
     final long start = System.nanoTime();
     final int rdx = buf.readerIndex();
     LOG.debug("------------------>> ENTERING DECODE >>------------------");
@@ -363,7 +369,8 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
     } catch (SaslException e) {
       String message = getPeerUuidLoggingString() + "Couldn't complete the SASL handshake";
       LOG.error(message);
-      throw new NonRecoverableException(message, e);
+      Status statusIOE = Status.IOError(message);
+      throw new NonRecoverableException(statusIOE, e);
     }
     if (buf == null) {
       return null;
@@ -377,7 +384,8 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
       final String msg = getPeerUuidLoggingString() + "RPC response (size: " + size + ") doesn't"
           + " have a call ID: " + header + ", buf=" + Bytes.pretty(buf);
       LOG.error(msg);
-      throw new NonRecoverableException(msg);
+      Status statusIncomplete = Status.Incomplete(msg);
+      throw new NonRecoverableException(statusIncomplete);
     }
     final int rpcid = header.getCallId();
 
@@ -388,6 +396,7 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
       final String msg = getPeerUuidLoggingString() + "Invalid rpcid: " + rpcid + " found in "
           + buf + '=' + Bytes.pretty(buf);
       LOG.error(msg);
+      Status statusIllegalState = Status.IllegalState(msg);
       // The problem here is that we don't know which Deferred corresponds to
       // this RPC, since we don't have a valid ID.  So we're hopeless, we'll
       // never be able to recover because responses are not framed, we don't
@@ -395,24 +404,25 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
       // and throw this outside of our Netty handler, so Netty will call our
       // exception handler where we'll close this channel, which will cause
       // all RPCs in flight to be failed.
-      throw new NonRecoverableException(msg);
+      throw new NonRecoverableException(statusIllegalState);
     }
 
     Pair<Object, Object> decoded = null;
     Exception exception = null;
-    KuduException retryableHeaderException = null;
+    Status retryableHeaderError = Status.OK();
     if (header.hasIsError() && header.getIsError()) {
       RpcHeader.ErrorStatusPB.Builder errorBuilder = RpcHeader.ErrorStatusPB.newBuilder();
       KuduRpc.readProtobuf(response.getPBMessage(), errorBuilder);
       RpcHeader.ErrorStatusPB error = errorBuilder.build();
       if (error.getCode().equals(RpcHeader.ErrorStatusPB.RpcErrorCodePB.ERROR_SERVER_TOO_BUSY)) {
         // We can't return right away, we still need to remove ourselves from 'rpcs_inflight', so we
-        // populate 'retryableHeaderException'.
-        retryableHeaderException = new TabletServerErrorException(uuid, error);
+        // populate 'retryableHeaderError'.
+        retryableHeaderError = Status.ServiceUnavailable(error.getMessage());
       } else {
         String message = getPeerUuidLoggingString() +
             "Tablet server sent error " + error.getMessage();
-        exception = new NonRecoverableException(message);
+        Status status = Status.RemoteError(message);
+        exception = new NonRecoverableException(status);
         LOG.error(message); // can be useful
       }
     } else {
@@ -433,13 +443,14 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
       final KuduRpc<?> removed = rpcs_inflight.remove(rpcid);
       if (removed == null) {
         // The RPC we were decoding was cleaned up already, give up.
-        throw new NonRecoverableException("RPC not found");
+        Status statusIllegalState = Status.IllegalState("RPC not found");
+        throw new NonRecoverableException(statusIllegalState);
       }
     }
 
     // This check is specifically for the ERROR_SERVER_TOO_BUSY case above.
-    if (retryableHeaderException != null) {
-      kuduClient.handleRetryableError(rpc, retryableHeaderException);
+    if (!retryableHeaderError.ok()) {
+      kuduClient.handleRetryableError(rpc, new RecoverableException(retryableHeaderError));
       return null;
     }
 
@@ -504,18 +515,18 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
   private Exception dispatchTSErrorOrReturnException(KuduRpc rpc,
                                                      Tserver.TabletServerErrorPB error) {
     WireProtocol.AppStatusPB.ErrorCode code = error.getStatus().getCode();
-    TabletServerErrorException ex = new TabletServerErrorException(uuid, error.getStatus());
+    Status status = Status.fromTabletServerErrorPB(error);
     if (error.getCode() == Tserver.TabletServerErrorPB.Code.TABLET_NOT_FOUND) {
-      kuduClient.handleTabletNotFound(rpc, ex, this);
+      kuduClient.handleTabletNotFound(rpc, new RecoverableException(status), this);
       // we're not calling rpc.callback() so we rely on the client to retry that RPC
     } else if (code == WireProtocol.AppStatusPB.ErrorCode.SERVICE_UNAVAILABLE) {
-      kuduClient.handleRetryableError(rpc, ex);
+      kuduClient.handleRetryableError(rpc, new RecoverableException(status));
       // The following two error codes are an indication that the tablet isn't a leader.
     } else if (code == WireProtocol.AppStatusPB.ErrorCode.ILLEGAL_STATE ||
         code == WireProtocol.AppStatusPB.ErrorCode.ABORTED) {
-      kuduClient.handleNotLeader(rpc, ex, this);
+      kuduClient.handleNotLeader(rpc, new RecoverableException(status), this);
     } else {
-      return ex;
+      return new NonRecoverableException(status);
     }
     return null;
   }
@@ -530,16 +541,22 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
   private Exception dispatchMasterErrorOrReturnException(KuduRpc rpc,
                                                          Master.MasterErrorPB error) {
     WireProtocol.AppStatusPB.ErrorCode code = error.getStatus().getCode();
-    MasterErrorException ex = new MasterErrorException(uuid, error);
+    Status status = Status.fromMasterErrorPB(error);
     if (error.getCode() == Master.MasterErrorPB.Code.NOT_THE_LEADER) {
-      kuduClient.handleNotLeader(rpc, ex, this);
-    } else if (code == WireProtocol.AppStatusPB.ErrorCode.SERVICE_UNAVAILABLE &&
-        (!(rpc instanceof GetMasterRegistrationRequest))) {
-      // TODO: This is a crutch until we either don't have to retry RPCs going to the
-      // same server or use retry policies.
-      kuduClient.handleRetryableError(rpc, ex);
+      kuduClient.handleNotLeader(rpc, new RecoverableException(status), this);
+    } else if (code == WireProtocol.AppStatusPB.ErrorCode.SERVICE_UNAVAILABLE) {
+      if (rpc instanceof GetMasterRegistrationRequest) {
+        // Special case:
+        // We never want to retry this RPC, we only use it to poke masters to learn where the leader
+        // is. If the error is truly non recoverable, it'll be handled later.
+        return new RecoverableException(status);
+      } else {
+        // TODO: This is a crutch until we either don't have to retry RPCs going to the
+        // same server or use retry policies.
+        kuduClient.handleRetryableError(rpc, new RecoverableException(status));
+      }
     } else {
-      return ex;
+      return new NonRecoverableException(status);
     }
     return null;
   }
@@ -559,7 +576,7 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
   protected Object decodeLast(final ChannelHandlerContext ctx,
                               final Channel chan,
                               final ChannelBuffer buf,
-                              final VoidEnum unused) {
+                              final VoidEnum unused) throws NonRecoverableException {
     // When we disconnect, decodeLast is called instead of decode.
     // We simply check whether there's any data left in the buffer, in which
     // case we attempt to process it.  But if there's no data left, then we
@@ -595,7 +612,7 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
    * from the server and sending an RPC (via {@link #sendRpc} or any other
    * indirect means such as {@code GetTableLocations()}) will fail immediately
    * by having the RPC's {@link Deferred} called back immediately with a
-   * {@link ConnectionResetException}.  This typically means that you got a
+   * {@link RecoverableException}.  This typically means that you got a
    * stale reference (or that the reference to this instance is just about to
    * be invalidated) and that you shouldn't use this instance.
    */
@@ -660,7 +677,7 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
   /**
    * Cleans up any outstanding or lingering RPC (used when shutting down).
    * <p>
-   * All RPCs in flight will fail with a {@link ConnectionResetException} and
+   * All RPCs in flight will fail with a {@link RecoverableException} and
    * all edits buffered will be re-scheduled.
    */
   private void cleanup(final Channel chan) {
@@ -688,8 +705,9 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
 
       pending_rpcs = null;
     }
-    final ConnectionResetException exception =
-        new ConnectionResetException(getPeerUuidLoggingString() + "Connection reset on " + chan);
+    Status statusNetworkError =
+        Status.NetworkError(getPeerUuidLoggingString() + "Connection reset on " + chan);
+    RecoverableException exception = new RecoverableException(statusNetworkError);
 
     failOrRetryRpcs(rpcs, exception);
   }
@@ -700,7 +718,7 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
    * @param exception an exception to propagate with the RPCs
    */
   private void failOrRetryRpcs(final Collection<KuduRpc<?>> rpcs,
-                               final ConnectionResetException exception) {
+                               final RecoverableException exception) {
     for (final KuduRpc<?> rpc : rpcs) {
       failOrRetryRpc(rpc, exception);
     }
@@ -712,7 +730,7 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
    * @param exception an exception to propagate with the RPC
    */
   private void failOrRetryRpc(final KuduRpc<?> rpc,
-                              final ConnectionResetException exception) {
+                              final RecoverableException exception) {
     AsyncKuduClient.RemoteTablet tablet = rpc.getTablet();
     // Note As of the time of writing (03/11/16), a null tablet doesn't make sense, if we see a null
     // tablet it's because we didn't set it properly before calling sendRpc().

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/main/java/org/kududb/client/TabletServerErrorException.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/TabletServerErrorException.java b/java/kudu-client/src/main/java/org/kududb/client/TabletServerErrorException.java
deleted file mode 100644
index 68fc54b..0000000
--- a/java/kudu-client/src/main/java/org/kududb/client/TabletServerErrorException.java
+++ /dev/null
@@ -1,40 +0,0 @@
-// 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.kududb.client;
-
-import org.kududb.WireProtocol;
-import org.kududb.annotations.InterfaceAudience;
-import org.kududb.annotations.InterfaceStability;
-import org.kududb.rpc.RpcHeader;
-
-/**
- * This exception is thrown by Tablet Servers when something goes wrong processing a request.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-@SuppressWarnings("serial")
-public class TabletServerErrorException extends KuduServerException {
-
-  TabletServerErrorException(String serverUuid, WireProtocol.AppStatusPB appStatus) {
-    super(serverUuid, appStatus);
-  }
-
-  TabletServerErrorException(String serverUuid, RpcHeader.ErrorStatusPB errorStatus) {
-    super(serverUuid, errorStatus);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/test/java/org/kududb/client/TestAsyncKuduSession.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/kududb/client/TestAsyncKuduSession.java b/java/kudu-client/src/test/java/org/kududb/client/TestAsyncKuduSession.java
index c4c2b57..ba69305 100644
--- a/java/kudu-client/src/test/java/org/kududb/client/TestAsyncKuduSession.java
+++ b/java/kudu-client/src/test/java/org/kududb/client/TestAsyncKuduSession.java
@@ -93,7 +93,7 @@ public class TestAsyncKuduSession extends BaseKuduTest {
         resp1.join(2000);
       } catch (TimeoutException e) {
         fail("First batch should not timeout in case of tablet server error");
-      } catch (TabletServerErrorException e) {
+      } catch (KuduException e) {
         // Expected.
         assertTrue(e.getMessage().contains("injected error for test"));
       }
@@ -101,7 +101,7 @@ public class TestAsyncKuduSession extends BaseKuduTest {
         resp2.join(2000);
       } catch (TimeoutException e) {
         fail("Second batch should not timeout in case of tablet server error");
-      } catch (TabletServerErrorException e) {
+      } catch (KuduException e) {
         // expected
         assertTrue(e.getMessage().contains("injected error for test"));
       }
@@ -139,8 +139,8 @@ public class TestAsyncKuduSession extends BaseKuduTest {
       try {
         session.apply(createInsert(1)).join(DEFAULT_SLEEP);
         fail("Insert should not succeed");
-      } catch (MasterErrorException e) {
-        // Expect NOT_FOUND, because the table was deleted.
+      } catch (KuduException e) {
+        assertTrue(e.getStatus().isNotFound());
       } catch (Throwable e) {
         fail("Should not throw other error: " + e);
       }
@@ -217,7 +217,7 @@ public class TestAsyncKuduSession extends BaseKuduTest {
     assertEquals(0, countInRange(10, 20));
     try {
       session.apply(createInsert(20));
-    } catch (NonRecoverableException ex) {
+    } catch (KuduException ex) {
       /* expected, buffer would be too big */
     }
     assertEquals(0, countInRange(10, 20)); // the buffer should still be full

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/test/java/org/kududb/client/TestKuduSession.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/kududb/client/TestKuduSession.java b/java/kudu-client/src/test/java/org/kududb/client/TestKuduSession.java
index 0a9311d..df2367f 100644
--- a/java/kudu-client/src/test/java/org/kududb/client/TestKuduSession.java
+++ b/java/kudu-client/src/test/java/org/kududb/client/TestKuduSession.java
@@ -273,8 +273,8 @@ public class TestKuduSession extends BaseKuduTest {
       try {
         session.apply(createBasicSchemaInsert(table, key));
         fail("apply should have thrown");
-      } catch (NonCoveredRangeException e) {
-        // Expected
+      } catch (KuduException e) {
+        assertTrue(e.getStatus().isNotFound());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/0a792366/java/kudu-client/src/test/java/org/kududb/client/TestKuduTable.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/kududb/client/TestKuduTable.java b/java/kudu-client/src/test/java/org/kududb/client/TestKuduTable.java
index 3a1160f..4e41a29 100644
--- a/java/kudu-client/src/test/java/org/kududb/client/TestKuduTable.java
+++ b/java/kudu-client/src/test/java/org/kududb/client/TestKuduTable.java
@@ -77,6 +77,23 @@ public class TestKuduTable extends BaseKuduTest {
           .addColumn("testaddmulticolnotnull", Type.INT32, 4)
           .addNullableColumn("testaddmulticolnull", Type.STRING);
       submitAlterAndCheck(ato, tableName);
+
+
+      // Try altering a table that doesn't exist.
+      String nonExistingTableName = "table_does_not_exist";
+      try {
+        syncClient.alterTable(nonExistingTableName, ato);
+        fail("Shouldn't be able to alter a table that doesn't exist");
+      } catch (KuduException ex) {
+        assertTrue(ex.getStatus().isNotFound());
+      }
+
+      try {
+        syncClient.isAlterTableDone(nonExistingTableName);
+        fail("Shouldn't be able to query if an alter table is done here");
+      } catch (KuduException ex) {
+        assertTrue(ex.getStatus().isNotFound());
+      }
     } finally {
       // Normally Java tests accumulate tables without issue, deleting them all
       // when shutting down the mini cluster at the end of every test class.