You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2012/08/31 20:29:54 UTC

svn commit: r1379541 - in /hbase/branches/0.89-fb: ./ src/main/java/org/apache/hadoop/hbase/ src/main/java/org/apache/hadoop/hbase/client/ src/main/java/org/apache/hadoop/hbase/regionserver/ src/main/java/org/apache/hadoop/hbase/thrift/ src/main/java/o...

Author: mbautin
Date: Fri Aug 31 18:29:54 2012
New Revision: 1379541

URL: http://svn.apache.org/viewvc?rev=1379541&view=rev
Log:
[jira] [HBASE-6681] [89-fb] Propagate RegionOverloadedException to the Thrift client

Author: mbautin

Summary: To correctly propagate RegionOverloadedException (added in HBASE-6423) to the Thrift client we need to make it a Thrift exception. One way to do that is to inherit from IOError and add the desired client-side delay in milliseconds to IOError.

Test Plan: Run a Thrift-based load test with small memstore that would result in blocking writes and verify that IOErrors indicating that the regionserver has been overloaded are being propagated to the client.

Reviewers: aaiyer, kranganathan, nzhang

Reviewed By: aaiyer

Differential Revision: https://reviews.facebook.net/D4983

Added:
    hbase/branches/0.89-fb/gen_thrift_code.sh
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java.new
      - copied, changed from r1379540, hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/RegionException.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionThriftServer.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOverloadedException.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
    hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift

Added: hbase/branches/0.89-fb/gen_thrift_code.sh
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/gen_thrift_code.sh?rev=1379541&view=auto
==============================================================================
--- hbase/branches/0.89-fb/gen_thrift_code.sh (added)
+++ hbase/branches/0.89-fb/gen_thrift_code.sh Fri Aug 31 18:29:54 2012
@@ -0,0 +1,37 @@
+#!/bin/bash
+
+set -e -u -o pipefail
+
+if [ ! -f pom.xml ]; then
+  echo "Have to run from HBase directory" >&2
+  exit 1
+fi
+
+THRIFT=thrift
+
+IF=src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift
+
+THRIFT_VERSION=`$THRIFT -version`
+EXPECTED_THRIFT_VERSION="Thrift version 0.8.0"
+if [ "$THRIFT_VERSION" != "$EXPECTED_THRIFT_VERSION" ]; then
+  echo "Expected $EXPECTED_THRIFT_VERSION, got $THRIFT_VERSION" >&2
+  exit 1
+fi
+
+THRIFT_VERSION=`echo $THRIFT_VERSION`
+
+$THRIFT --gen java -out src/main/java $IF
+
+echo "Automatically changing IOError's superclass"
+F=src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java 
+awk '{ sub(/extends Exception/, "extends java.io.IOException"); print }' <$F \
+  >$F.new
+set +e
+diff $F $F.new
+if [ $# -eq 0 ]; then
+  echo "Failed to replace 'extends Exception'" >&2
+  exit 1
+fi
+set -e
+cp $F.new $F
+

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/RegionException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/RegionException.java?rev=1379541&r1=1379540&r2=1379541&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/RegionException.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/RegionException.java Fri Aug 31 18:29:54 2012
@@ -19,12 +19,13 @@
  */
 package org.apache.hadoop.hbase;
 
-import java.io.IOException;
+import org.apache.hadoop.hbase.thrift.generated.IOError;
+
 /**
  * Thrown when something happens related to region handling.
  * Subclasses have to be more specific.
  */
-public class RegionException extends IOException {
+public class RegionException extends IOError {
   private static final long serialVersionUID = 1473510258071111371L;
 
   /** default constructor */
@@ -37,7 +38,11 @@ public class RegionException extends IOE
    * @param s message
    */
   public RegionException(String s) {
-    super(s);
+    super(s, 0);
+  }
+
+  public RegionException(String s, long waitMillis) {
+    super(s, waitMillis);
   }
 
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1379541&r1=1379540&r2=1379541&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Fri Aug 31 18:29:54 2012
@@ -29,7 +29,6 @@ import java.net.SocketTimeoutException;
 import java.nio.channels.ClosedChannelException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
@@ -1389,6 +1388,7 @@ public class HConnectionManager {
     public <T> T getRegionServerWithRetries(ServerCallable<T> callable)
         throws IOException {
       List<Throwable> exceptions = new ArrayList<Throwable>();
+      RegionOverloadedException roe = null;
 
       long callStartTime;
       int serverRequestedRetries = 0;
@@ -1398,15 +1398,17 @@ public class HConnectionManager {
       // do not retry if region cannot be located. There are enough retries
       // within instantiateRegionLocation.
       callable.instantiateRegionLocation(false /* reload cache? */);
+
       for(int tries = 0; ; tries++) {
         // If server requested wait. We will wait for that time, and start
         // again. Do not count this time/tries against the client retries.
         if (serverRequestedWaitTime > 0) {
           serverRequestedRetries++;
 
-          if (serverRequestedRetries > this.maxServerRequestedRetries)
-            throw new RetriesExhaustedException(callable.getServerName(),
-            callable.getRegionName(), callable.getRow(), serverRequestedRetries, exceptions);
+          if (serverRequestedRetries > this.maxServerRequestedRetries) {
+            throw RegionOverloadedException.create(roe, exceptions,
+                serverRequestedRetries);
+          }
 
           long pauseTime = serverRequestedWaitTime + callStartTime
               - System.currentTimeMillis();
@@ -1439,8 +1441,8 @@ public class HConnectionManager {
 
           // If we are not supposed to retry; Let it pass through.
           throw ioe;
-        } catch (RegionOverloadedException roe) {
-          exceptions.add(roe);
+        } catch (RegionOverloadedException ex) {
+          roe = ex;
           serverRequestedWaitTime = roe.getBackoffTimeMillis();
           continue;
         } catch (PreemptiveFastFailException pfe) {
@@ -2338,10 +2340,11 @@ public class HConnectionManager {
             put, options);
         Future<MultiPutResponse> task;
         if (singleServer) {
-          task = new FutureTask<MultiPutResponse>(callable);
-          ((FutureTask<MultiPutResponse>)task).run();
+          FutureTask<MultiPutResponse> futureTask = new FutureTask<MultiPutResponse>(callable);
+          task = futureTask;
+          futureTask.run();
         } else {
-            task = HTable.multiActionThreadPool.submit(callable);
+          task = HTable.multiActionThreadPool.submit(callable);
         }
         futures.add(task);
       }
@@ -2401,14 +2404,18 @@ public class HConnectionManager {
             failed.addAll(e.getValue());
           } else if (result != HConstants.MULTIPUT_SUCCESS) {
             // some failures
-            if (failed == null) failed = new ArrayList<Put>();
+            if (failed == null) {
+              failed = new ArrayList<Put>();
+            }
             failed.addAll(lst.subList(result, lst.size()));
             LOG.debug("Failed past " + result + " for region: " +
                 Bytes.toStringBinary(region) + ", removing from cache");
           }
         }
       }
-      if (toThrow != null) throw toThrow;
+      if (toThrow != null) {
+        throw toThrow;
+      }
 
       return failed;
     }
@@ -2433,35 +2440,36 @@ public class HConnectionManager {
      */
     public void processBatchOfPuts(List<Put> list, final byte[] tableName, HBaseRPCOptions options)
     throws IOException {
+      RegionOverloadedException roe = null;
       long callStartTime;
       callStartTime = System.currentTimeMillis();
 
       int tries;
       long serverRequestedWaitTime = 0;
       int serverRequestedRetries = 0;
+
       for ( tries = 0 ; tries < numRetries && !list.isEmpty(); ++tries) {
         // If server requested wait. We will wait for that time, and start
         // again. Do not count this time/tries against the client retries.
         if (serverRequestedWaitTime > 0) {
           serverRequestedRetries++;
 
-          // Only do this for a configurable number of times?
-          if (serverRequestedRetries > this.maxServerRequestedRetries)
-            throw new RetriesExhaustedException("Server Overloaded: Still had "
-                + list.size() + " puts left after server requested " +
-                serverRequestedRetries + " retries.");
+          // Only do this for a configurable number of times.
+          if (serverRequestedRetries > this.maxServerRequestedRetries) {
+            throw roe;
+          }
 
           long sleepTimePending = callStartTime + serverRequestedWaitTime
               - System.currentTimeMillis();
-            try {
-              Thread.sleep(sleepTimePending);
-            } catch (InterruptedException e) {
-              Thread.currentThread().interrupt();
-              throw new InterruptedIOException();
-            }
-            tries = 0;
-            callStartTime = System.currentTimeMillis();
-            serverRequestedWaitTime = 0;
+          try {
+            Thread.sleep(sleepTimePending);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new InterruptedIOException();
+          }
+          tries = 0;
+          callStartTime = System.currentTimeMillis();
+          serverRequestedWaitTime = 0;
         }
 
         List<Put> failed = null;
@@ -2469,9 +2477,9 @@ public class HConnectionManager {
         try {
           failed = this.processListOfMultiPut(multiPuts, tableName, options);
         } catch (RegionOverloadedException ex) {
-            serverRequestedWaitTime = ex.getBackoffTimeMillis();
-            // do not clear the list
-            continue;
+          roe = ex;
+          serverRequestedWaitTime = roe.getBackoffTimeMillis();
+          continue;
         } catch (PreemptiveFastFailException pfe) {
           throw pfe;
         }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionThriftServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionThriftServer.java?rev=1379541&r1=1379540&r2=1379541&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionThriftServer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionThriftServer.java Fri Aug 31 18:29:54 2012
@@ -128,13 +128,13 @@ public class HRegionThriftServer extends
                                        qualifier, amount, true);
       } catch (NotServingRegionException e) {
         if (!redirect) {
-          throw new IOError(e.getMessage());
+          throw new IOError(e.getMessage(), 0);
         }
         LOG.info("ThriftServer redirecting atomicIncrement");
         return super.atomicIncrement(tableName, row, family,
                                      qualifier, amount);
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw new IOError(e.getMessage(), 0);
       }
     }
 
@@ -173,12 +173,12 @@ public class HRegionThriftServer extends
         return ThriftUtilities.rowResultFromHBase(result);
       } catch (NotServingRegionException e) {
         if (!redirect) {
-          throw new IOError(e.getMessage());
+          throw new IOError(e.getMessage(), 0);
         }
         LOG.info("ThriftServer redirecting getRowWithColumnsTs");
         return super.getRowWithColumnsTs(tableName, row, columns, timestamp);
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw new IOError(e.getMessage(), 0);
       }
     }
 
@@ -213,7 +213,7 @@ public class HRegionThriftServer extends
         Result result = table.get(get);
         return ThriftUtilities.rowResultFromHBase(result);
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw new IOError(e.getMessage(), 0);
       }
     }
   }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOverloadedException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOverloadedException.java?rev=1379541&r1=1379540&r2=1379541&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOverloadedException.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOverloadedException.java Fri Aug 31 18:29:54 2012
@@ -1,13 +1,12 @@
 package org.apache.hadoop.hbase.regionserver;
 
+import java.util.List;
+
 import org.apache.hadoop.hbase.RegionException;
 
 public class RegionOverloadedException extends RegionException {
   private static final long serialVersionUID = -8436877560512061623L;
 
-  long backOffTime;
-
-
   /** default constructor */
   public RegionOverloadedException() {
     super();
@@ -17,12 +16,32 @@ public class RegionOverloadedException e
    *  @param waitMillis -- request client to backoff for waitMillis
    */
   public RegionOverloadedException(String s, long waitMillis) {
-    super(s);
-    backOffTime = waitMillis;
+    super(s, waitMillis);
   }
 
-  public long getBackoffTimeMillis() {
-    return backOffTime;
+  /**
+   * Create a RegionOverloadedException from another one, attaching a set of related exceptions
+   * from a batch operation. The new exception reuses the original exception's stack trace.
+   *  
+   * @param roe the original exception
+   * @param exceptions other exceptions that happened in the same batch operation
+   * @param waitMillis remaining time for the client to wait in milliseconds
+   * @return the new exception with complete information
+   */
+  public static RegionOverloadedException create(RegionOverloadedException roe,
+      List<Throwable> exceptions, int waitMillis) {
+    StringBuilder sb = new StringBuilder(roe.getMessage());
+    for (Throwable t : exceptions) {
+      if (t != roe) {
+        sb.append(t.toString());
+        sb.append("\n");
+      }
+    }
+    RegionOverloadedException e = new RegionOverloadedException(sb.toString(), waitMillis);
+    if (roe != null) {  // Safety check
+      e.setStackTrace(roe.getStackTrace());
+    }
+    return e;
   }
 
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java?rev=1379541&r1=1379540&r2=1379541&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java Fri Aug 31 18:29:54 2012
@@ -532,7 +532,7 @@ public class ThriftServerRunner implemen
       try{
         getHBaseAdmin().enableTable(getBytes(tableName));
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -541,7 +541,7 @@ public class ThriftServerRunner implemen
       try{
         getHBaseAdmin().disableTable(getBytes(tableName));
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -550,7 +550,7 @@ public class ThriftServerRunner implemen
       try {
         return HTable.isTableEnabled(conf, getBytes(tableName));
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -559,7 +559,7 @@ public class ThriftServerRunner implemen
       try{
         getHBaseAdmin().compact(getBytes(tableNameOrRegionName));
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -568,7 +568,7 @@ public class ThriftServerRunner implemen
       try{
         getHBaseAdmin().majorCompact(getBytes(tableNameOrRegionName));
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -582,7 +582,7 @@ public class ThriftServerRunner implemen
         }
         return list;
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -619,7 +619,7 @@ public class ThriftServerRunner implemen
         }
         return regions;
       } catch (IOException e){
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -649,7 +649,7 @@ public class ThriftServerRunner implemen
         Result result = table.get(get);
         return ThriftUtilities.cellFromHBase(result.sorted());
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -672,7 +672,7 @@ public class ThriftServerRunner implemen
         Result result = table.get(get);
         return ThriftUtilities.cellFromHBase(result.sorted());
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -697,7 +697,7 @@ public class ThriftServerRunner implemen
         Result result = table.get(get);
         return ThriftUtilities.cellFromHBase(result.sorted());
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -747,7 +747,7 @@ public class ThriftServerRunner implemen
         Result result = table.get(get);
         return ThriftUtilities.rowResultFromHBase(result);
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -781,7 +781,7 @@ public class ThriftServerRunner implemen
         Result result = table.get(get);
         return ThriftUtilities.rowResultFromHBase(result);
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -841,7 +841,7 @@ public class ThriftServerRunner implemen
         Result[] result = table.get(gets);
         return ThriftUtilities.rowResultFromHBase(result);
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -860,7 +860,7 @@ public class ThriftServerRunner implemen
         table.delete(delete);
 
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -876,7 +876,7 @@ public class ThriftServerRunner implemen
         Delete delete  = new Delete(getBytes(row), timestamp, null);
         table.delete(delete);
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -896,7 +896,7 @@ public class ThriftServerRunner implemen
         }
         getHBaseAdmin().createTable(desc);
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       } catch (IllegalArgumentException e) {
         throw new IllegalArgument(e.getMessage());
       }
@@ -910,11 +910,11 @@ public class ThriftServerRunner implemen
       }
       try {
         if (!getHBaseAdmin().tableExists(tableNameBytes)) {
-          throw new IOError("table does not exist");
+          throw new IOError("table does not exist", 0);
         }
         getHBaseAdmin().deleteTable(tableNameBytes);
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -980,7 +980,7 @@ public class ThriftServerRunner implemen
           table.put(put);
         }
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       } catch (IllegalArgumentException e) {
         throw new IllegalArgument(e.getMessage());
       }
@@ -1077,7 +1077,7 @@ public class ThriftServerRunner implemen
           }
         }
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       } catch (IllegalArgumentException e) {
         throw new IllegalArgument(e.getMessage());
       }
@@ -1155,7 +1155,7 @@ public class ThriftServerRunner implemen
         throw new IllegalArgument(
             "Thrift CheckAndMutate call must do either put or delete.");
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       } catch (IllegalArgumentException e) {
         throw new IllegalArgument(e.getMessage());
       }
@@ -1178,7 +1178,7 @@ public class ThriftServerRunner implemen
         table = getTable(tableName);
         return table.incrementColumnValue(row, family, qualifier, amount);
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -1208,7 +1208,7 @@ public class ThriftServerRunner implemen
         } catch (IOException e) {
           scanner.close();
           removeScanner(id);
-          throw new IOError(e.getMessage());
+          throw convertIOException(e);
         }
         return ThriftUtilities.rowResultFromHBase(results);
     }
@@ -1235,7 +1235,7 @@ public class ThriftServerRunner implemen
           }
           return addScanner(table.getScanner(scan));
         } catch (IOException e) {
-          throw new IOError(e.getMessage());
+          throw convertIOException(e);
         }
     }
 
@@ -1257,7 +1257,7 @@ public class ThriftServerRunner implemen
         }
         return addScanner(table.getScanner(scan));
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -1283,7 +1283,7 @@ public class ThriftServerRunner implemen
         }
         return addScanner(table.getScanner(scan));
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -1326,7 +1326,7 @@ public class ThriftServerRunner implemen
         }
         return addScanner(table.getScanner(scan));
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -1349,7 +1349,7 @@ public class ThriftServerRunner implemen
         }
         return addScanner(table.getScanner(scan));
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -1374,7 +1374,7 @@ public class ThriftServerRunner implemen
         scan.setTimeRange(Long.MIN_VALUE, timestamp);
         return addScanner(table.getScanner(scan));
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -1414,7 +1414,7 @@ public class ThriftServerRunner implemen
         }
         return addScanner(table.getScanner(scan));
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -1434,7 +1434,7 @@ public class ThriftServerRunner implemen
         }
         return columns;
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -1479,7 +1479,7 @@ public class ThriftServerRunner implemen
         }
         return region;
       } catch (IOException e) {
-        throw new IOError(e.getMessage());
+        throw convertIOException(e);
       }
     }
 
@@ -1503,4 +1503,11 @@ public class ThriftServerRunner implemen
     }
   }
 
+  private static IOError convertIOException(IOException e) {
+    if (e instanceof IOError) {
+      return (IOError) e;
+    }
+    return new IOError(e.getMessage(), 0);
+  }
+  
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java?rev=1379541&r1=1379540&r2=1379541&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java Fri Aug 31 18:29:54 2012
@@ -7179,6 +7179,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -10252,7 +10254,7 @@ public class Hbase {
                   for (int _i35 = 0; _i35 < _map34.size; ++_i35)
                   {
                     ByteBuffer _key36; // required
-                    ColumnDescriptor _val37; // required
+                    ColumnDescriptor _val37; // optional
                     _key36 = iprot.readBinary();
                     _val37 = new ColumnDescriptor();
                     _val37.read(iprot);
@@ -10358,7 +10360,7 @@ public class Hbase {
             for (int _i41 = 0; _i41 < _map40.size; ++_i41)
             {
               ByteBuffer _key42; // required
-              ColumnDescriptor _val43; // required
+              ColumnDescriptor _val43; // optional
               _key42 = iprot.readBinary();
               _val43 = new ColumnDescriptor();
               _val43.read(iprot);
@@ -16112,8 +16114,6 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -34093,6 +34093,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -34986,8 +34988,6 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -35641,6 +35641,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -36925,6 +36927,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -38608,8 +38612,6 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -40446,8 +40448,6 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -41949,6 +41949,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -42890,6 +42892,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -44141,6 +44145,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -45254,6 +45260,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -46488,6 +46496,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -47297,6 +47307,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -47848,6 +47860,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -48797,6 +48811,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -49382,6 +49398,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -49855,6 +49873,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -51060,6 +51080,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -51855,8 +51877,6 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -52376,6 +52396,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
@@ -52763,6 +52785,8 @@ public class Hbase {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bit_vector = new BitSet(1);
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java?rev=1379541&r1=1379540&r2=1379541&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java Fri Aug 31 18:29:54 2012
@@ -32,10 +32,11 @@ import org.slf4j.LoggerFactory;
  * to the Hbase master or an Hbase region server.  Also used to return
  * more general Hbase error conditions.
  */
-public class IOError extends Exception implements org.apache.thrift.TBase<IOError, IOError._Fields>, java.io.Serializable, Cloneable {
+public class IOError extends java.io.IOException implements org.apache.thrift.TBase<IOError, IOError._Fields>, java.io.Serializable, Cloneable {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IOError");
 
   private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField BACKOFF_TIME_MILLIS_FIELD_DESC = new org.apache.thrift.protocol.TField("backoffTimeMillis", org.apache.thrift.protocol.TType.I64, (short)2);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -44,10 +45,12 @@ public class IOError extends Exception i
   }
 
   public String message; // required
+  public long backoffTimeMillis; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    MESSAGE((short)1, "message");
+    MESSAGE((short)1, "message"),
+    BACKOFF_TIME_MILLIS((short)2, "backoffTimeMillis");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -64,6 +67,8 @@ public class IOError extends Exception i
       switch(fieldId) {
         case 1: // MESSAGE
           return MESSAGE;
+        case 2: // BACKOFF_TIME_MILLIS
+          return BACKOFF_TIME_MILLIS;
         default:
           return null;
       }
@@ -104,11 +109,15 @@ public class IOError extends Exception i
   }
 
   // isset id assignments
+  private static final int __BACKOFFTIMEMILLIS_ISSET_ID = 0;
+  private BitSet __isset_bit_vector = new BitSet(1);
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
     tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT,
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.BACKOFF_TIME_MILLIS, new org.apache.thrift.meta_data.FieldMetaData("backoffTimeMillis", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(IOError.class, metaDataMap);
   }
@@ -117,19 +126,25 @@ public class IOError extends Exception i
   }
 
   public IOError(
-    String message)
+    String message,
+    long backoffTimeMillis)
   {
     this();
     this.message = message;
+    this.backoffTimeMillis = backoffTimeMillis;
+    setBackoffTimeMillisIsSet(true);
   }
 
   /**
    * Performs a deep copy on <i>other</i>.
    */
   public IOError(IOError other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
     if (other.isSetMessage()) {
       this.message = other.message;
     }
+    this.backoffTimeMillis = other.backoffTimeMillis;
   }
 
   public IOError deepCopy() {
@@ -139,6 +154,8 @@ public class IOError extends Exception i
   @Override
   public void clear() {
     this.message = null;
+    setBackoffTimeMillisIsSet(false);
+    this.backoffTimeMillis = 0;
   }
 
   public String getMessage() {
@@ -165,6 +182,29 @@ public class IOError extends Exception i
     }
   }
 
+  public long getBackoffTimeMillis() {
+    return this.backoffTimeMillis;
+  }
+
+  public IOError setBackoffTimeMillis(long backoffTimeMillis) {
+    this.backoffTimeMillis = backoffTimeMillis;
+    setBackoffTimeMillisIsSet(true);
+    return this;
+  }
+
+  public void unsetBackoffTimeMillis() {
+    __isset_bit_vector.clear(__BACKOFFTIMEMILLIS_ISSET_ID);
+  }
+
+  /** Returns true if field backoffTimeMillis is set (has been assigned a value) and false otherwise */
+  public boolean isSetBackoffTimeMillis() {
+    return __isset_bit_vector.get(__BACKOFFTIMEMILLIS_ISSET_ID);
+  }
+
+  public void setBackoffTimeMillisIsSet(boolean value) {
+    __isset_bit_vector.set(__BACKOFFTIMEMILLIS_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case MESSAGE:
@@ -175,6 +215,14 @@ public class IOError extends Exception i
       }
       break;
 
+    case BACKOFF_TIME_MILLIS:
+      if (value == null) {
+        unsetBackoffTimeMillis();
+      } else {
+        setBackoffTimeMillis((Long)value);
+      }
+      break;
+
     }
   }
 
@@ -183,6 +231,9 @@ public class IOError extends Exception i
     case MESSAGE:
       return getMessage();
 
+    case BACKOFF_TIME_MILLIS:
+      return Long.valueOf(getBackoffTimeMillis());
+
     }
     throw new IllegalStateException();
   }
@@ -196,6 +247,8 @@ public class IOError extends Exception i
     switch (field) {
     case MESSAGE:
       return isSetMessage();
+    case BACKOFF_TIME_MILLIS:
+      return isSetBackoffTimeMillis();
     }
     throw new IllegalStateException();
   }
@@ -222,6 +275,15 @@ public class IOError extends Exception i
         return false;
     }
 
+    boolean this_present_backoffTimeMillis = true;
+    boolean that_present_backoffTimeMillis = true;
+    if (this_present_backoffTimeMillis || that_present_backoffTimeMillis) {
+      if (!(this_present_backoffTimeMillis && that_present_backoffTimeMillis))
+        return false;
+      if (this.backoffTimeMillis != that.backoffTimeMillis)
+        return false;
+    }
+
     return true;
   }
 
@@ -248,6 +310,16 @@ public class IOError extends Exception i
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetBackoffTimeMillis()).compareTo(typedOther.isSetBackoffTimeMillis());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetBackoffTimeMillis()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.backoffTimeMillis, typedOther.backoffTimeMillis);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -275,6 +347,10 @@ public class IOError extends Exception i
       sb.append(this.message);
     }
     first = false;
+    if (!first) sb.append(", ");
+    sb.append("backoffTimeMillis:");
+    sb.append(this.backoffTimeMillis);
+    first = false;
     sb.append(")");
     return sb.toString();
   }
@@ -293,6 +369,8 @@ public class IOError extends Exception i
 
   private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
     try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bit_vector = new BitSet(1);
       read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
     } catch (org.apache.thrift.TException te) {
       throw new java.io.IOException(te);
@@ -325,6 +403,14 @@ public class IOError extends Exception i
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 2: // BACKOFF_TIME_MILLIS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.backoffTimeMillis = iprot.readI64();
+              struct.setBackoffTimeMillisIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -345,6 +431,9 @@ public class IOError extends Exception i
         oprot.writeString(struct.message);
         oprot.writeFieldEnd();
       }
+      oprot.writeFieldBegin(BACKOFF_TIME_MILLIS_FIELD_DESC);
+      oprot.writeI64(struct.backoffTimeMillis);
+      oprot.writeFieldEnd();
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -366,20 +455,30 @@ public class IOError extends Exception i
       if (struct.isSetMessage()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetBackoffTimeMillis()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.isSetMessage()) {
         oprot.writeString(struct.message);
       }
+      if (struct.isSetBackoffTimeMillis()) {
+        oprot.writeI64(struct.backoffTimeMillis);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, IOError struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(1);
+      BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.message = iprot.readString();
         struct.setMessageIsSet(true);
       }
+      if (incoming.get(1)) {
+        struct.backoffTimeMillis = iprot.readI64();
+        struct.setBackoffTimeMillisIsSet(true);
+      }
     }
   }
 

Copied: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java.new (from r1379540, hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java)
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java.new?p2=hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java.new&p1=hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java&r1=1379540&r2=1379541&rev=1379541&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java.new Fri Aug 31 18:29:54 2012
@@ -32,10 +32,11 @@ import org.slf4j.LoggerFactory;
  * to the Hbase master or an Hbase region server.  Also used to return
  * more general Hbase error conditions.
  */
-public class IOError extends Exception implements org.apache.thrift.TBase<IOError, IOError._Fields>, java.io.Serializable, Cloneable {
+public class IOError extends java.io.IOException implements org.apache.thrift.TBase<IOError, IOError._Fields>, java.io.Serializable, Cloneable {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IOError");
 
   private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField BACKOFF_TIME_MILLIS_FIELD_DESC = new org.apache.thrift.protocol.TField("backoffTimeMillis", org.apache.thrift.protocol.TType.I64, (short)2);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -44,10 +45,12 @@ public class IOError extends Exception i
   }
 
   public String message; // required
+  public long backoffTimeMillis; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    MESSAGE((short)1, "message");
+    MESSAGE((short)1, "message"),
+    BACKOFF_TIME_MILLIS((short)2, "backoffTimeMillis");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -64,6 +67,8 @@ public class IOError extends Exception i
       switch(fieldId) {
         case 1: // MESSAGE
           return MESSAGE;
+        case 2: // BACKOFF_TIME_MILLIS
+          return BACKOFF_TIME_MILLIS;
         default:
           return null;
       }
@@ -104,11 +109,15 @@ public class IOError extends Exception i
   }
 
   // isset id assignments
+  private static final int __BACKOFFTIMEMILLIS_ISSET_ID = 0;
+  private BitSet __isset_bit_vector = new BitSet(1);
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT,
+    tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.BACKOFF_TIME_MILLIS, new org.apache.thrift.meta_data.FieldMetaData("backoffTimeMillis", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(IOError.class, metaDataMap);
   }
@@ -117,19 +126,25 @@ public class IOError extends Exception i
   }
 
   public IOError(
-    String message)
+    String message,
+    long backoffTimeMillis)
   {
     this();
     this.message = message;
+    this.backoffTimeMillis = backoffTimeMillis;
+    setBackoffTimeMillisIsSet(true);
   }
 
   /**
    * Performs a deep copy on <i>other</i>.
    */
   public IOError(IOError other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
     if (other.isSetMessage()) {
       this.message = other.message;
     }
+    this.backoffTimeMillis = other.backoffTimeMillis;
   }
 
   public IOError deepCopy() {
@@ -139,6 +154,8 @@ public class IOError extends Exception i
   @Override
   public void clear() {
     this.message = null;
+    setBackoffTimeMillisIsSet(false);
+    this.backoffTimeMillis = 0;
   }
 
   public String getMessage() {
@@ -165,6 +182,29 @@ public class IOError extends Exception i
     }
   }
 
+  public long getBackoffTimeMillis() {
+    return this.backoffTimeMillis;
+  }
+
+  public IOError setBackoffTimeMillis(long backoffTimeMillis) {
+    this.backoffTimeMillis = backoffTimeMillis;
+    setBackoffTimeMillisIsSet(true);
+    return this;
+  }
+
+  public void unsetBackoffTimeMillis() {
+    __isset_bit_vector.clear(__BACKOFFTIMEMILLIS_ISSET_ID);
+  }
+
+  /** Returns true if field backoffTimeMillis is set (has been assigned a value) and false otherwise */
+  public boolean isSetBackoffTimeMillis() {
+    return __isset_bit_vector.get(__BACKOFFTIMEMILLIS_ISSET_ID);
+  }
+
+  public void setBackoffTimeMillisIsSet(boolean value) {
+    __isset_bit_vector.set(__BACKOFFTIMEMILLIS_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case MESSAGE:
@@ -175,6 +215,14 @@ public class IOError extends Exception i
       }
       break;
 
+    case BACKOFF_TIME_MILLIS:
+      if (value == null) {
+        unsetBackoffTimeMillis();
+      } else {
+        setBackoffTimeMillis((Long)value);
+      }
+      break;
+
     }
   }
 
@@ -183,6 +231,9 @@ public class IOError extends Exception i
     case MESSAGE:
       return getMessage();
 
+    case BACKOFF_TIME_MILLIS:
+      return Long.valueOf(getBackoffTimeMillis());
+
     }
     throw new IllegalStateException();
   }
@@ -196,6 +247,8 @@ public class IOError extends Exception i
     switch (field) {
     case MESSAGE:
       return isSetMessage();
+    case BACKOFF_TIME_MILLIS:
+      return isSetBackoffTimeMillis();
     }
     throw new IllegalStateException();
   }
@@ -222,6 +275,15 @@ public class IOError extends Exception i
         return false;
     }
 
+    boolean this_present_backoffTimeMillis = true;
+    boolean that_present_backoffTimeMillis = true;
+    if (this_present_backoffTimeMillis || that_present_backoffTimeMillis) {
+      if (!(this_present_backoffTimeMillis && that_present_backoffTimeMillis))
+        return false;
+      if (this.backoffTimeMillis != that.backoffTimeMillis)
+        return false;
+    }
+
     return true;
   }
 
@@ -248,6 +310,16 @@ public class IOError extends Exception i
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetBackoffTimeMillis()).compareTo(typedOther.isSetBackoffTimeMillis());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetBackoffTimeMillis()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.backoffTimeMillis, typedOther.backoffTimeMillis);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -275,6 +347,10 @@ public class IOError extends Exception i
       sb.append(this.message);
     }
     first = false;
+    if (!first) sb.append(", ");
+    sb.append("backoffTimeMillis:");
+    sb.append(this.backoffTimeMillis);
+    first = false;
     sb.append(")");
     return sb.toString();
   }
@@ -293,6 +369,8 @@ public class IOError extends Exception i
 
   private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
     try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bit_vector = new BitSet(1);
       read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
     } catch (org.apache.thrift.TException te) {
       throw new java.io.IOException(te);
@@ -313,7 +391,7 @@ public class IOError extends Exception i
       while (true)
       {
         schemeField = iprot.readFieldBegin();
-        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (schemeField.id) {
@@ -321,7 +399,15 @@ public class IOError extends Exception i
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.message = iprot.readString();
               struct.setMessageIsSet(true);
-            } else {
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // BACKOFF_TIME_MILLIS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.backoffTimeMillis = iprot.readI64();
+              struct.setBackoffTimeMillisIsSet(true);
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
@@ -345,6 +431,9 @@ public class IOError extends Exception i
         oprot.writeString(struct.message);
         oprot.writeFieldEnd();
       }
+      oprot.writeFieldBegin(BACKOFF_TIME_MILLIS_FIELD_DESC);
+      oprot.writeI64(struct.backoffTimeMillis);
+      oprot.writeFieldEnd();
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -366,20 +455,30 @@ public class IOError extends Exception i
       if (struct.isSetMessage()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetBackoffTimeMillis()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.isSetMessage()) {
         oprot.writeString(struct.message);
       }
+      if (struct.isSetBackoffTimeMillis()) {
+        oprot.writeI64(struct.backoffTimeMillis);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, IOError struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(1);
+      BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.message = iprot.readString();
         struct.setMessageIsSet(true);
       }
+      if (incoming.get(1)) {
+        struct.backoffTimeMillis = iprot.readI64();
+        struct.setBackoffTimeMillisIsSet(true);
+      }
     }
   }
 

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java?rev=1379541&r1=1379540&r2=1379541&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java Fri Aug 31 18:29:54 2012
@@ -443,7 +443,7 @@ public class TRowResult implements org.a
                 for (int _i9 = 0; _i9 < _map8.size; ++_i9)
                 {
                   ByteBuffer _key10; // required
-                  TCell _val11; // required
+                  TCell _val11; // optional
                   _key10 = iprot.readBinary();
                   _val11 = new TCell();
                   _val11.read(iprot);
@@ -544,7 +544,7 @@ public class TRowResult implements org.a
           for (int _i15 = 0; _i15 < _map14.size; ++_i15)
           {
             ByteBuffer _key16; // required
-            TCell _val17; // required
+            TCell _val17; // optional
             _key16 = iprot.readBinary();
             _val17 = new TCell();
             _val17.read(iprot);

Modified: hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift?rev=1379541&r1=1379540&r2=1379541&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift (original)
+++ hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift Fri Aug 31 18:29:54 2012
@@ -143,7 +143,8 @@ struct TScan {
  * more general Hbase error conditions.
  */
 exception IOError {
-  1:string message
+  1: string message,
+  2: i64 backoffTimeMillis
 }
 
 /**