You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/02/14 13:58:21 UTC

svn commit: r1446147 [18/35] - in /hbase/branches/hbase-7290v2: ./ bin/ conf/ dev-support/ hbase-client/ hbase-common/ hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/ hbase-common/src/...

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java Thu Feb 14 12:58:12 2013
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.replicat
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.net.ConnectException;
 import java.net.SocketTimeoutException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -52,7 +53,6 @@ import org.apache.hadoop.hbase.client.HC
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
-import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
@@ -107,8 +107,6 @@ public class ReplicationSource extends T
   private int replicationQueueNbCapacity;
   // Our reader for the current log
   private HLog.Reader reader;
-  // Current position in the log
-  private long position = 0;
   // Last position in the log that we sent to ZooKeeper
   private long lastLoggedPosition = -1;
   // Path of the current log
@@ -134,10 +132,14 @@ public class ReplicationSource extends T
   private int currentNbEntries = 0;
   // Current number of operations (Put/Delete) that we need to replicate
   private int currentNbOperations = 0;
+  // Current size of data we need to replicate
+  private int currentSize = 0;
   // Indicates if this particular source is running
   private volatile boolean running = true;
   // Metrics for this source
   private MetricsSource metrics;
+  // Handle on the log reader helper
+  private ReplicationHLogReaderManager repLogReader;
 
   /**
    * Instantiation method used by region servers
@@ -185,7 +187,7 @@ public class ReplicationSource extends T
         this.conf.getLong("replication.source.sleepforretries", 1000);
     this.fs = fs;
     this.metrics = new MetricsSource(peerClusterZnode);
-
+    this.repLogReader = new ReplicationHLogReaderManager(this.fs, this.conf);
     try {
       this.clusterId = zkHelper.getUUIDForCluster(zkHelper.getZookeeperWatcher());
     } catch (KeeperException ke) {
@@ -266,8 +268,8 @@ public class ReplicationSource extends T
     // normally has a position (unless the RS failed between 2 logs)
     if (this.queueRecovered) {
       try {
-        this.position = this.zkHelper.getHLogRepPosition(
-            this.peerClusterZnode, this.queue.peek().getName());
+        this.repLogReader.setPosition(this.zkHelper.getHLogRepPosition(
+            this.peerClusterZnode, this.queue.peek().getName()));
       } catch (KeeperException e) {
         this.terminate("Couldn't get the position of this recovered queue " +
             peerClusterZnode, e);
@@ -325,6 +327,7 @@ public class ReplicationSource extends T
 
       boolean gotIOE = false;
       currentNbEntries = 0;
+      currentSize = 0;
       try {
         if (readAllEntriesToReplicateOrNextFile(currentWALisBeingWrittenTo)) {
           continue;
@@ -360,9 +363,7 @@ public class ReplicationSource extends T
         }
       } finally {
         try {
-          if (this.reader != null) {
-            this.reader.close();
-          }
+          this.repLogReader.closeReader();
         } catch (IOException e) {
           gotIOE = true;
           LOG.warn("Unable to finalize the tailing of a file", e);
@@ -373,10 +374,11 @@ public class ReplicationSource extends T
       // wait a bit and retry.
       // But if we need to stop, don't bother sleeping
       if (this.isActive() && (gotIOE || currentNbEntries == 0)) {
-        if (this.lastLoggedPosition != this.position) {
+        if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
           this.manager.logPositionAndCleanOldLogs(this.currentPath,
-              this.peerClusterZnode, this.position, queueRecovered, currentWALisBeingWrittenTo);
-          this.lastLoggedPosition = this.position;
+              this.peerClusterZnode, this.repLogReader.getPosition(),
+              queueRecovered, currentWALisBeingWrittenTo);
+          this.lastLoggedPosition = this.repLogReader.getPosition();
         }
         if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
           sleepMultiplier++;
@@ -409,11 +411,9 @@ public class ReplicationSource extends T
   protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo)
       throws IOException{
     long seenEntries = 0;
-    if (this.position != 0) {
-      this.reader.seek(this.position);
-    }
-    long startPosition = this.position;
-    HLog.Entry entry = readNextAndSetPosition(); 
+    this.repLogReader.seek();
+    HLog.Entry entry =
+        this.repLogReader.readNextAndSetPosition(this.entriesArray, this.currentNbEntries);
     while (entry != null) {
       WALEdit edit = entry.getEdit();
       this.metrics.incrLogEditsRead();
@@ -437,18 +437,18 @@ public class ReplicationSource extends T
           }
           currentNbOperations += countDistinctRowKeys(edit);
           currentNbEntries++;
+          currentSize += entry.getEdit().size();
         } else {
           this.metrics.incrLogEditsFiltered();
         }
       }
       // Stop if too many entries or too big
-      if ((this.reader.getPosition() - startPosition)
-          >= this.replicationQueueSizeCapacity ||
+      if (currentSize >= this.replicationQueueSizeCapacity ||
           currentNbEntries >= this.replicationQueueNbCapacity) {
         break;
       }
       try {
-        entry = readNextAndSetPosition();
+        entry = this.repLogReader.readNextAndSetPosition(this.entriesArray, this.currentNbEntries);
       } catch (IOException ie) {
         LOG.debug("Break on IOE: " + ie.getMessage());
         break;
@@ -462,16 +462,6 @@ public class ReplicationSource extends T
     return seenEntries == 0 && processEndOfFile();
   }
 
-  private HLog.Entry readNextAndSetPosition() throws IOException {
-    HLog.Entry entry = this.reader.next(entriesArray[currentNbEntries]);
-    // Store the position so that in the future the reader can start
-    // reading from here. If the above call to next() throws an
-    // exception, the position won't be changed and retry will happen
-    // from the last known good position
-    this.position = this.reader.getPosition();
-    return entry;
-  } 
-
   private void connectToPeers() {
     // Connect to peer cluster first, unless we have to stop
     while (this.isActive() && this.currentPeers.size() == 0) {
@@ -510,9 +500,7 @@ public class ReplicationSource extends T
   protected boolean openReader(int sleepMultiplier) {
     try {
       try {
-       this.reader = null;
-       this.reader = HLogFactory.createReader(this.fs, 
-           this.currentPath, this.conf);
+        this.reader = repLogReader.openReader(this.currentPath);
       } catch (FileNotFoundException fnfe) {
         if (this.queueRecovered) {
           // We didn't find the log in the archive directory, look if it still
@@ -565,6 +553,7 @@ public class ReplicationSource extends T
       }
     } catch (IOException ioe) {
       LOG.warn(peerClusterZnode + " Got: ", ioe);
+      this.reader = null;
       // TODO Need a better way to determinate if a file is really gone but
       // TODO without scanning all logs dir
       if (sleepMultiplier == this.maxRetriesMultiplier) {
@@ -648,10 +637,11 @@ public class ReplicationSource extends T
         AdminProtocol rrs = getRS();
         ProtobufUtil.replicateWALEntry(rrs,
           Arrays.copyOf(this.entriesArray, currentNbEntries));
-        if (this.lastLoggedPosition != this.position) {
+        if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
           this.manager.logPositionAndCleanOldLogs(this.currentPath,
-              this.peerClusterZnode, this.position, queueRecovered, currentWALisBeingWrittenTo);
-          this.lastLoggedPosition = this.position;
+              this.peerClusterZnode, this.repLogReader.getPosition(),
+              queueRecovered, currentWALisBeingWrittenTo);
+          this.lastLoggedPosition = this.repLogReader.getPosition();
         }
         this.totalReplicatedEdits += currentNbEntries;
         this.metrics.shipBatch(this.currentNbOperations);
@@ -674,6 +664,9 @@ public class ReplicationSource extends T
               "call to the remote cluster timed out, which is usually " +
               "caused by a machine failure or a massive slowdown",
               this.socketTimeoutMultiplier);
+          } else if (ioe instanceof ConnectException) {
+            LOG.warn("Peer is unavailable, rechecking all sinks: ", ioe);
+            chooseSinks();
           } else {
             LOG.warn("Can't replicate because of a local or network error: ", ioe);
           }
@@ -718,7 +711,8 @@ public class ReplicationSource extends T
   protected boolean processEndOfFile() {
     if (this.queue.size() != 0) {
       this.currentPath = null;
-      this.position = 0;
+      this.repLogReader.finishCurrentFile();
+      this.reader = null;
       return true;
     } else if (this.queueRecovered) {
       this.manager.closeRecoveredQueue(this);
@@ -842,8 +836,15 @@ public class ReplicationSource extends T
 
   @Override
   public String getStats() {
+    String position = "N/A";
+    try {
+      if (this.reader != null) {
+        position = this.reader.getPosition()+"";
+      }
+    } catch (IOException ioe) {
+    }
     return "Total replicated edits: " + totalReplicatedEdits +
       ", currently replicating from: " + this.currentPath +
-      " at position: " + this.position;
+      " at position: " + position;
   }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java Thu Feb 14 12:58:12 2013
@@ -40,6 +40,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
@@ -73,7 +74,7 @@ public class ReplicationSourceManager {
   private final ReplicationZookeeper zkHelper;
   // All about stopping
   private final Stoppable stopper;
-  // All logs we are currently trackign
+  // All logs we are currently tracking
   private final Map<String, SortedSet<String>> hlogsById;
   private final Configuration conf;
   private final FileSystem fs;
@@ -157,7 +158,7 @@ public class ReplicationSourceManager {
     }
     synchronized (this.hlogsById) {
       SortedSet<String> hlogs = this.hlogsById.get(id);
-      if (!queueRecovered && hlogs.first() != key) {
+      if (!queueRecovered && !hlogs.first().equals(key)) {
         SortedSet<String> hlogSet = hlogs.headSet(key);
         for (String hlog : hlogSet) {
           this.zkHelper.removeLogFromList(hlog, id);
@@ -504,8 +505,10 @@ public class ReplicationSourceManager {
       if (peers == null) {
         return;
       }
-      String id = ReplicationZookeeper.getZNodeName(path);
-      removePeer(id);
+      if (zkHelper.isPeerPath(path)) {
+        String id = ReplicationZookeeper.getZNodeName(path);
+        removePeer(id);
+      }
     }
 
     /**
@@ -579,14 +582,22 @@ public class ReplicationSourceManager {
         LOG.info("Not transferring queue since we are shutting down");
         return;
       }
-      if (!zkHelper.lockOtherRS(rsZnode)) {
-        return;
+      SortedMap<String, SortedSet<String>> newQueues = null;
+
+      // check whether there is multi support. If yes, use it.
+      if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
+        LOG.info("Atomically moving " + rsZnode + "'s hlogs to my queue");
+        newQueues = zkHelper.copyQueuesFromRSUsingMulti(rsZnode);
+      } else {
+        LOG.info("Moving " + rsZnode + "'s hlogs to my queue");
+        if (!zkHelper.lockOtherRS(rsZnode)) {
+          return;
+        }
+        newQueues = zkHelper.copyQueuesFromRS(rsZnode);
+        zkHelper.deleteRsQueues(rsZnode);
       }
-      LOG.info("Moving " + rsZnode + "'s hlogs to my queue");
-      SortedMap<String, SortedSet<String>> newQueues =
-          zkHelper.copyQueuesFromRS(rsZnode);
-      zkHelper.deleteRsQueues(rsZnode);
-      if (newQueues == null || newQueues.size() == 0) {
+      // process of copying over the failed queue is completed.
+      if (newQueues.size() == 0) {
         return;
       }
 

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/Constants.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/Constants.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/Constants.java Thu Feb 14 12:58:12 2013
@@ -39,5 +39,8 @@ public interface Constants {
   public static final String MIMETYPE_XML = "text/xml";
   public static final String MIMETYPE_BINARY = "application/octet-stream";
   public static final String MIMETYPE_PROTOBUF = "application/x-protobuf";
+  public static final String MIMETYPE_PROTOBUF_IETF = "application/protobuf";
   public static final String MIMETYPE_JSON = "application/json";
+
+  public static final String CRLF = "\r\n";
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ExistsResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ExistsResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ExistsResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ExistsResource.java Thu Feb 14 12:58:12 2013
@@ -23,7 +23,6 @@ import java.io.IOException;
 
 import javax.ws.rs.GET;
 import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.CacheControl;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
@@ -56,14 +55,18 @@ public class ExistsResource extends Reso
 
   @GET
   @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
-    MIMETYPE_BINARY})
+    MIMETYPE_PROTOBUF_IETF, MIMETYPE_BINARY})
   public Response get(final @Context UriInfo uriInfo) {
     try {
       if (!tableResource.exists()) {
-        throw new WebApplicationException(Response.Status.NOT_FOUND);
+        return Response.status(Response.Status.NOT_FOUND)
+          .type(MIMETYPE_TEXT).entity("Not found" + CRLF)
+          .build();
       }
     } catch (IOException e) {
-      throw new WebApplicationException(Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
     ResponseBuilder response = Response.ok();
     response.cacheControl(cacheControl);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java Thu Feb 14 12:58:12 2013
@@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.rest.mode
 
 import javax.ws.rs.GET;
 import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MultivaluedMap;
 import javax.ws.rs.core.Response;
@@ -61,7 +60,8 @@ public class MultiRowResource extends Re
   }
 
   @GET
-  @Produces({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Produces({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response get(final @Context UriInfo uriInfo) {
     MultivaluedMap<String, String> params = uriInfo.getQueryParameters();
 
@@ -75,9 +75,12 @@ public class MultiRowResource extends Re
           rowSpec.setMaxVersions(this.versions);
         }
 
-        ResultGenerator generator = ResultGenerator.fromRowSpec(this.tableResource.getName(), rowSpec, null);
+        ResultGenerator generator =
+          ResultGenerator.fromRowSpec(this.tableResource.getName(), rowSpec, null);
         if (!generator.hasNext()) {
-          throw new WebApplicationException(Response.Status.NOT_FOUND);
+          return Response.status(Response.Status.NOT_FOUND)
+            .type(MIMETYPE_TEXT).entity("Not found" + CRLF)
+            .build();
         }
 
         KeyValue value = null;
@@ -94,8 +97,9 @@ public class MultiRowResource extends Re
       return Response.ok(model).build();
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(e,
-              Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
 
   }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java Thu Feb 14 12:58:12 2013
@@ -44,6 +44,7 @@ import org.mortbay.jetty.Server;
 import org.mortbay.jetty.nio.SelectChannelConnector;
 import org.mortbay.jetty.servlet.Context;
 import org.mortbay.jetty.servlet.ServletHolder;
+import org.mortbay.thread.QueuedThreadPool;
 
 import com.sun.jersey.spi.container.servlet.ServletContainer;
 
@@ -140,6 +141,17 @@ public class RESTServer implements Const
 
     server.addConnector(connector);
 
+    // Set the default max thread number to 100 to limit
+    // the number of concurrent requests so that REST server doesn't OOM easily.
+    // Jetty set the default max thread number to 250, if we don't set it.
+    //
+    // Our default min thread number 2 is the same as that used by Jetty.
+    int maxThreads = servlet.getConfiguration().getInt("hbase.rest.threads.max", 100);
+    int minThreads = servlet.getConfiguration().getInt("hbase.rest.threads.min", 2);
+    QueuedThreadPool threadPool = new QueuedThreadPool(maxThreads);
+    threadPool.setMinThreads(minThreads);
+    server.setThreadPool(threadPool);
+
     server.setSendServerVersion(false);
     server.setSendDateHeader(false);
     server.setStopAtShutdown(true);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java Thu Feb 14 12:58:12 2013
@@ -23,6 +23,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTablePool;
 
 /**
@@ -34,6 +35,7 @@ public class RESTServlet implements Cons
   private final Configuration conf;
   private final HTablePool pool;
   private final MetricsREST metrics = new MetricsREST();
+  private final HBaseAdmin admin;
 
   /**
    * @return the RESTServlet singleton instance
@@ -69,6 +71,11 @@ public class RESTServlet implements Cons
   RESTServlet(Configuration conf) throws IOException {
     this.conf = conf;
     this.pool = new HTablePool(conf, 10);
+    this.admin = new HBaseAdmin(conf);
+  }
+
+  HBaseAdmin getAdmin() {
+    return admin;
   }
 
   HTablePool getTablePool() {

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java Thu Feb 14 12:58:12 2013
@@ -24,7 +24,6 @@ import java.util.Map;
 
 import javax.ws.rs.GET;
 import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.CacheControl;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
@@ -67,7 +66,8 @@ public class RegionsResource extends Res
   }
 
   @GET
-  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response get(final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("GET " + uriInfo.getAbsolutePath());
@@ -91,11 +91,14 @@ public class RegionsResource extends Res
       return response.build();
     } catch (TableNotFoundException e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(Response.Status.NOT_FOUND);
+      return Response.status(Response.Status.NOT_FOUND)
+        .type(MIMETYPE_TEXT).entity("Not found" + CRLF)
+        .build();
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(e,
-                  Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java Thu Feb 14 12:58:12 2013
@@ -25,7 +25,6 @@ import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.CacheControl;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
@@ -37,7 +36,6 @@ import org.apache.commons.logging.LogFac
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.rest.model.TableListModel;
 import org.apache.hadoop.hbase.rest.model.TableModel;
 
@@ -63,8 +61,7 @@ public class RootResource extends Resour
 
   private final TableListModel getTableList() throws IOException {
     TableListModel tableList = new TableListModel();
-    HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
-    HTableDescriptor[] list = admin.listTables();
+    HTableDescriptor[] list = servlet.getAdmin().listTables();
     for (HTableDescriptor htd: list) {
       tableList.add(new TableModel(htd.getNameAsString()));
     }
@@ -72,7 +69,8 @@ public class RootResource extends Resour
   }
 
   @GET
-  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response get(final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("GET " + uriInfo.getAbsolutePath());
@@ -85,8 +83,9 @@ public class RootResource extends Resour
       return response.build();
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(e, 
-                  Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java Thu Feb 14 12:58:12 2013
@@ -29,7 +29,6 @@ import javax.ws.rs.GET;
 import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Response;
@@ -41,6 +40,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.HTablePool;
@@ -80,7 +80,8 @@ public class RowResource extends Resourc
   }
 
   @GET
-  @Produces({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Produces({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response get(final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("GET " + uriInfo.getAbsolutePath());
@@ -90,7 +91,9 @@ public class RowResource extends Resourc
       ResultGenerator generator =
         ResultGenerator.fromRowSpec(tableResource.getName(), rowspec, null);
       if (!generator.hasNext()) {
-        throw new WebApplicationException(Response.Status.NOT_FOUND);
+        return Response.status(Response.Status.NOT_FOUND)
+          .type(MIMETYPE_TEXT).entity("Not found" + CRLF)
+          .build();
       }
       int count = 0;
       CellSetModel model = new CellSetModel();
@@ -113,10 +116,21 @@ public class RowResource extends Resourc
       model.addRow(rowModel);
       servlet.getMetrics().incrementSucessfulGetRequests(1);
       return Response.ok(model).build();
-    } catch (IOException e) {
-      servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(e,
-                  Response.Status.SERVICE_UNAVAILABLE);
+    } catch (RuntimeException e) {
+      servlet.getMetrics().incrementFailedPutRequests(1);
+      if (e.getCause() instanceof TableNotFoundException) {
+        return Response.status(Response.Status.NOT_FOUND)
+          .type(MIMETYPE_TEXT).entity("Not found" + CRLF)
+          .build();
+      }
+      return Response.status(Response.Status.BAD_REQUEST)
+        .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+        .build();
+    } catch (Exception e) {
+      servlet.getMetrics().incrementFailedPutRequests(1);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 
@@ -130,13 +144,17 @@ public class RowResource extends Resourc
     // doesn't make sense to use a non specific coordinate as this can only
     // return a single cell
     if (!rowspec.hasColumns() || rowspec.getColumns().length > 1) {
-      throw new WebApplicationException(Response.Status.BAD_REQUEST);
+      return Response.status(Response.Status.BAD_REQUEST)
+        .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+        .build();
     }
     try {
       ResultGenerator generator =
         ResultGenerator.fromRowSpec(tableResource.getName(), rowspec, null);
       if (!generator.hasNext()) {
-        throw new WebApplicationException(Response.Status.NOT_FOUND);
+        return Response.status(Response.Status.NOT_FOUND)
+          .type(MIMETYPE_TEXT).entity("Not found" + CRLF)
+          .build();
       }
       KeyValue value = generator.next();
       ResponseBuilder response = Response.ok(value.getValue());
@@ -145,15 +163,18 @@ public class RowResource extends Resourc
       return response.build();
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(e,
-                  Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 
   Response update(final CellSetModel model, final boolean replace) {
     servlet.getMetrics().incrementRequests(1);
     if (servlet.isReadOnly()) {
-      throw new WebApplicationException(Response.Status.FORBIDDEN);
+      return Response.status(Response.Status.FORBIDDEN)
+        .type(MIMETYPE_TEXT).entity("Forbidden" + CRLF)
+        .build();
     }
 
     if (CHECK_PUT.equalsIgnoreCase(check)) {
@@ -175,7 +196,9 @@ public class RowResource extends Resourc
           key = rowspec.getRow();
         }
         if (key == null) {
-          throw new WebApplicationException(Response.Status.BAD_REQUEST);
+          return Response.status(Response.Status.BAD_REQUEST)
+            .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+            .build();
         }
         Put put = new Put(key);
         int i = 0;
@@ -187,7 +210,9 @@ public class RowResource extends Resourc
             col = null;
           }
           if (col == null) {
-            throw new WebApplicationException(Response.Status.BAD_REQUEST);
+            return Response.status(Response.Status.BAD_REQUEST)
+              .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+              .build();
           }
           byte [][] parts = KeyValue.parseColumn(col);
           if (parts.length == 2 && parts[1].length > 0) {
@@ -209,12 +234,15 @@ public class RowResource extends Resourc
       return response.build();
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedPutRequests(1);
-      throw new WebApplicationException(e,
-                  Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     } finally {
       if (table != null) try {
         table.close();
-      } catch (IOException ioe) { }
+      } catch (IOException ioe) {
+        LOG.debug("Exception received while closing the table", ioe);
+      }
     }
   }
 
@@ -223,7 +251,9 @@ public class RowResource extends Resourc
       final boolean replace) {
     servlet.getMetrics().incrementRequests(1);
     if (servlet.isReadOnly()) {
-      throw new WebApplicationException(Response.Status.FORBIDDEN);
+      return Response.status(Response.Status.FORBIDDEN)
+        .type(MIMETYPE_TEXT).entity("Forbidden" + CRLF)
+        .build();
     }
     HTablePool pool = servlet.getTablePool();
     HTableInterface table = null;
@@ -248,7 +278,9 @@ public class RowResource extends Resourc
         timestamp = Long.valueOf(vals.get(0));
       }
       if (column == null) {
-        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+        return Response.status(Response.Status.BAD_REQUEST)
+          .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+          .build();
       }
       Put put = new Put(row);
       byte parts[][] = KeyValue.parseColumn(column);
@@ -266,8 +298,9 @@ public class RowResource extends Resourc
       return Response.ok().build();
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedPutRequests(1);
-      throw new WebApplicationException(e,
-                  Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     } finally {
       if (table != null) try {
         table.close();
@@ -276,7 +309,8 @@ public class RowResource extends Resourc
   }
 
   @PUT
-  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response put(final CellSetModel model,
       final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
@@ -297,7 +331,8 @@ public class RowResource extends Resourc
   }
 
   @POST
-  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response post(final CellSetModel model,
       final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
@@ -324,11 +359,13 @@ public class RowResource extends Resourc
     }
     servlet.getMetrics().incrementRequests(1);
     if (servlet.isReadOnly()) {
-      throw new WebApplicationException(Response.Status.FORBIDDEN);
+      return Response.status(Response.Status.FORBIDDEN)
+        .type(MIMETYPE_TEXT).entity("Forbidden" + CRLF)
+        .build();
     }
     Delete delete = null;
     if (rowspec.hasTimestamp())
-      delete = new Delete(rowspec.getRow(), rowspec.getTimestamp(), null);
+      delete = new Delete(rowspec.getRow(), rowspec.getTimestamp());
     else
       delete = new Delete(rowspec.getRow());
 
@@ -359,8 +396,9 @@ public class RowResource extends Resourc
       }
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedDeleteRequests(1);
-      throw new WebApplicationException(e, 
-                  Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     } finally {
       if (table != null) try {
         table.close();
@@ -381,7 +419,9 @@ public class RowResource extends Resourc
     HTableInterface table = null;
     try {
       if (model.getRows().size() != 1) {
-        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+        return Response.status(Response.Status.BAD_REQUEST)
+          .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+          .build();
       }
 
       RowModel rowModel = model.getRows().get(0);
@@ -393,7 +433,9 @@ public class RowResource extends Resourc
       List<CellModel> cellModels = rowModel.getCells();
       int cellModelCount = cellModels.size();
       if (key == null || cellModelCount <= 1) {
-        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+        return Response.status(Response.Status.BAD_REQUEST)
+          .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+          .build();
       }
 
       Put put = new Put(key);
@@ -413,10 +455,14 @@ public class RowResource extends Resourc
           put.add(valueToPutParts[0], valueToPutParts[1], valueToPutCell
             .getTimestamp(), valueToPutCell.getValue());
         } else {
-          throw new WebApplicationException(Response.Status.BAD_REQUEST);
+          return Response.status(Response.Status.BAD_REQUEST)
+            .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+            .build();
         }
       } else {
-        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+        return Response.status(Response.Status.BAD_REQUEST)
+          .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+          .build();
       }
 
       table = pool.getTable(this.tableResource.getName());
@@ -432,7 +478,9 @@ public class RowResource extends Resourc
       }
       return response.build();
     } catch (IOException e) {
-      throw new WebApplicationException(e, Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     } finally {
       if (table != null) try {
         table.close();
@@ -453,7 +501,9 @@ public class RowResource extends Resourc
     Delete delete = null;
     try {
       if (model.getRows().size() != 1) {
-        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+        return Response.status(Response.Status.BAD_REQUEST)
+          .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+          .build();
       }
       RowModel rowModel = model.getRows().get(0);
       byte[] key = rowModel.getKey();
@@ -461,7 +511,9 @@ public class RowResource extends Resourc
         key = rowspec.getRow();
       }
       if (key == null) {
-        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+        return Response.status(Response.Status.BAD_REQUEST)
+          .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+          .build();
       }
 
       delete = new Delete(key);
@@ -471,14 +523,18 @@ public class RowResource extends Resourc
         try {
           valueToDeleteColumn = rowspec.getColumns()[0];
         } catch (final ArrayIndexOutOfBoundsException e) {
-          throw new WebApplicationException(Response.Status.BAD_REQUEST);
+          return Response.status(Response.Status.BAD_REQUEST)
+            .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+            .build();
         }
       }
       byte[][] parts = KeyValue.parseColumn(valueToDeleteColumn);
       if (parts.length == 2 && parts[1].length > 0) {
         delete.deleteColumns(parts[0], parts[1]);
       } else {
-        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+        return Response.status(Response.Status.BAD_REQUEST)
+          .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+          .build();
       }
 
       table = pool.getTable(tableResource.getName());
@@ -495,11 +551,15 @@ public class RowResource extends Resourc
       }
       return response.build();
     } catch (IOException e) {
-      throw new WebApplicationException(e, Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     } finally {
       if (table != null) try {
         table.close();
-      } catch (IOException ioe) { }
+      } catch (IOException ioe) {
+        LOG.debug("Exception received while closing the table", ioe);
+      }
     }
   }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java Thu Feb 14 12:58:12 2013
@@ -25,7 +25,6 @@ import javax.ws.rs.DELETE;
 import javax.ws.rs.GET;
 import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.CacheControl;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
@@ -55,10 +54,12 @@ public class ScannerInstanceResource ext
     cacheControl.setNoTransform(false);
   }
 
-  ResultGenerator generator;
-  String id;
+  ResultGenerator generator = null;
+  String id = null;
   int batch = 1;
 
+  public ScannerInstanceResource() throws IOException { }
+
   public ScannerInstanceResource(String table, String id, 
       ResultGenerator generator, int batch) throws IOException {
     this.id = id;
@@ -67,13 +68,20 @@ public class ScannerInstanceResource ext
   }
 
   @GET
-  @Produces({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Produces({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response get(final @Context UriInfo uriInfo, 
       @QueryParam("n") int maxRows, final @QueryParam("c") int maxValues) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("GET " + uriInfo.getAbsolutePath());
     }
     servlet.getMetrics().incrementRequests(1);
+    if (generator == null) {
+      servlet.getMetrics().incrementFailedGetRequests(1);
+      return Response.status(Response.Status.NOT_FOUND)
+        .type(MIMETYPE_TEXT).entity("Not found" + CRLF)
+        .build();
+    }
     CellSetModel model = new CellSetModel();
     RowModel rowModel = null;
     byte[] rowKey = null;
@@ -93,7 +101,9 @@ public class ScannerInstanceResource ext
           servlet.getMetrics().incrementFailedDeleteRequests(1);
         }
         servlet.getMetrics().incrementFailedGetRequests(1);
-        throw new WebApplicationException(Response.Status.GONE);
+        return Response.status(Response.Status.GONE)
+          .type(MIMETYPE_TEXT).entity("Gone" + CRLF)
+          .build();
       }
       if (value == null) {
         LOG.info("generator exhausted");
@@ -162,7 +172,9 @@ public class ScannerInstanceResource ext
         servlet.getMetrics().incrementFailedDeleteRequests(1);
       }
       servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(Response.Status.GONE);
+      return Response.status(Response.Status.GONE)
+        .type(MIMETYPE_TEXT).entity("Gone" + CRLF)
+        .build();
     }
   }
 
@@ -173,7 +185,9 @@ public class ScannerInstanceResource ext
     }
     servlet.getMetrics().incrementRequests(1);
     if (servlet.isReadOnly()) {
-      throw new WebApplicationException(Response.Status.FORBIDDEN);
+      return Response.status(Response.Status.FORBIDDEN)
+        .type(MIMETYPE_TEXT).entity("Forbidden" + CRLF)
+        .build();
     }
     if (ScannerResource.delete(id)) {
       servlet.getMetrics().incrementSucessfulDeleteRequests(1);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResource.java Thu Feb 14 12:58:12 2013
@@ -30,7 +30,6 @@ import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.UriBuilder;
@@ -78,7 +77,9 @@ public class ScannerResource extends Res
       final UriInfo uriInfo) {
     servlet.getMetrics().incrementRequests(1);
     if (servlet.isReadOnly()) {
-      throw new WebApplicationException(Response.Status.FORBIDDEN);
+      return Response.status(Response.Status.FORBIDDEN)
+        .type(MIMETYPE_TEXT).entity("Forbidden" + CRLF)
+        .build();
     }
     byte[] endRow = model.hasEndRow() ? model.getEndRow() : null;
     RowSpec spec = new RowSpec(model.getStartRow(), endRow,
@@ -100,24 +101,27 @@ public class ScannerResource extends Res
       URI uri = builder.path(id).build();
       servlet.getMetrics().incrementSucessfulPutRequests(1);
       return Response.created(uri).build();
-    } catch (IOException e) {
-      servlet.getMetrics().incrementFailedPutRequests(1);
-      throw new WebApplicationException(e,
-              Response.Status.SERVICE_UNAVAILABLE);
     } catch (RuntimeException e) {
       servlet.getMetrics().incrementFailedPutRequests(1);
       if (e.getCause() instanceof TableNotFoundException) {
-        throw new WebApplicationException(e, Response.Status.NOT_FOUND);
+        return Response.status(Response.Status.NOT_FOUND)
+          .type(MIMETYPE_TEXT).entity("Not found" + CRLF)
+          .build();
       }
-      throw new WebApplicationException(e, Response.Status.BAD_REQUEST);
+      return Response.status(Response.Status.BAD_REQUEST)
+        .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+        .build();
     } catch (Exception e) {
       servlet.getMetrics().incrementFailedPutRequests(1);
-      throw new WebApplicationException(e, Response.Status.BAD_REQUEST);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 
   @PUT
-  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response put(final ScannerModel model, 
       final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
@@ -127,7 +131,8 @@ public class ScannerResource extends Res
   }
 
   @POST
-  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response post(final ScannerModel model,
       final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
@@ -138,11 +143,11 @@ public class ScannerResource extends Res
 
   @Path("{scanner: .+}")
   public ScannerInstanceResource getScannerInstanceResource(
-      final @PathParam("scanner") String id) {
+      final @PathParam("scanner") String id) throws IOException {
     ScannerInstanceResource instance = scanners.get(id);
     if (instance == null) {
       servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(Response.Status.NOT_FOUND);
+      return new ScannerInstanceResource();
     } else {
       servlet.getMetrics().incrementSucessfulGetRequests(1);
     }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java Thu Feb 14 12:58:12 2013
@@ -28,7 +28,6 @@ import javax.ws.rs.GET;
 import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.CacheControl;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
@@ -87,7 +86,8 @@ public class SchemaResource extends Reso
   }
 
   @GET
-  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response get(final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("GET " + uriInfo.getAbsolutePath());
@@ -101,18 +101,23 @@ public class SchemaResource extends Reso
       return response.build();
     } catch (TableNotFoundException e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(Response.Status.NOT_FOUND);
+      return Response.status(Response.Status.NOT_FOUND)
+        .type(MIMETYPE_TEXT).entity("Not found" + CRLF)
+        .build();
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(e,
-                  Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 
   private Response replace(final byte[] name, final TableSchemaModel model,
       final UriInfo uriInfo, final HBaseAdmin admin) {
     if (servlet.isReadOnly()) {
-      throw new WebApplicationException(Response.Status.FORBIDDEN);
+      return Response.status(Response.Status.FORBIDDEN)
+        .type(MIMETYPE_TEXT).entity("Forbidden" + CRLF)
+        .build();
     }
     try {
       HTableDescriptor htd = new HTableDescriptor(name);
@@ -136,19 +141,24 @@ public class SchemaResource extends Reso
         servlet.getMetrics().incrementSucessfulPutRequests(1);
       } catch (TableExistsException e) {
         // race, someone else created a table with the same name
-        throw new WebApplicationException(e, Response.Status.NOT_MODIFIED);
+        return Response.status(Response.Status.NOT_MODIFIED)
+          .type(MIMETYPE_TEXT).entity("Not modified" + CRLF)
+          .build();
       }
       return Response.created(uriInfo.getAbsolutePath()).build();
     } catch (IOException e) {
-      throw new WebApplicationException(e,
-            Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 
   private Response update(final byte[] name, final TableSchemaModel model,
       final UriInfo uriInfo, final HBaseAdmin admin) {
     if (servlet.isReadOnly()) {
-      throw new WebApplicationException(Response.Status.FORBIDDEN);
+      return Response.status(Response.Status.FORBIDDEN)
+        .type(MIMETYPE_TEXT).entity("Forbidden" + CRLF)
+        .build();
     }
     try {
       HTableDescriptor htd = admin.getTableDescriptor(name);
@@ -166,16 +176,18 @@ public class SchemaResource extends Reso
           }
         }
       } catch (IOException e) {
-        throw new WebApplicationException(e,
-            Response.Status.INTERNAL_SERVER_ERROR);
+        return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+          .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+          .build();
       } finally {
         admin.enableTable(tableResource.getName());
       }
       servlet.getMetrics().incrementSucessfulPutRequests(1);
       return Response.ok().build();
     } catch (IOException e) {
-      throw new WebApplicationException(e,
-          Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 
@@ -183,7 +195,7 @@ public class SchemaResource extends Reso
       final UriInfo uriInfo) {
     try {
       byte[] name = Bytes.toBytes(tableResource.getName());
-      HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
+      HBaseAdmin admin = servlet.getAdmin();
       if (replace || !admin.tableExists(name)) {
         return replace(name, model, uriInfo, admin);
       } else {
@@ -191,13 +203,15 @@ public class SchemaResource extends Reso
       }
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedPutRequests(1);
-      throw new WebApplicationException(e, 
-            Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 
   @PUT
-  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response put(final TableSchemaModel model, 
       final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
@@ -208,7 +222,8 @@ public class SchemaResource extends Reso
   }
 
   @POST
-  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response post(final TableSchemaModel model, 
       final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
@@ -225,7 +240,7 @@ public class SchemaResource extends Reso
     }
     servlet.getMetrics().incrementRequests(1);
     try {
-      HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
+      HBaseAdmin admin = servlet.getAdmin();
       boolean success = false;
       for (int i = 0; i < 10; i++) try {
         admin.disableTable(tableResource.getName());
@@ -241,11 +256,14 @@ public class SchemaResource extends Reso
       return Response.ok().build();
     } catch (TableNotFoundException e) {
       servlet.getMetrics().incrementFailedDeleteRequests(1);
-      throw new WebApplicationException(Response.Status.NOT_FOUND);
+      return Response.status(Response.Status.NOT_FOUND)
+        .type(MIMETYPE_TEXT).entity("Not found" + CRLF)
+        .build();
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedDeleteRequests(1);
-      throw new WebApplicationException(e, 
-            Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java Thu Feb 14 12:58:12 2013
@@ -23,7 +23,6 @@ import java.io.IOException;
 
 import javax.ws.rs.GET;
 import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.CacheControl;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.ClusterSt
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
 
 @InterfaceAudience.Private
@@ -62,15 +60,15 @@ public class StorageClusterStatusResourc
   }
 
   @GET
-  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response get(final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("GET " + uriInfo.getAbsolutePath());
     }
     servlet.getMetrics().incrementRequests(1);
     try {
-      HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
-      ClusterStatus status = admin.getClusterStatus();
+      ClusterStatus status = servlet.getAdmin().getClusterStatus();
       StorageClusterStatusModel model = new StorageClusterStatusModel();
       model.setRegions(status.getRegionsCount());
       model.setRequests(status.getRequestsCount());
@@ -103,8 +101,9 @@ public class StorageClusterStatusResourc
       return response.build();
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(e, 
-                  Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java Thu Feb 14 12:58:12 2013
@@ -23,7 +23,6 @@ import java.io.IOException;
 
 import javax.ws.rs.GET;
 import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.CacheControl;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
@@ -34,7 +33,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.rest.model.StorageClusterVersionModel;
 
 @InterfaceAudience.Private
@@ -65,17 +63,17 @@ public class StorageClusterVersionResour
     }
     servlet.getMetrics().incrementRequests(1);
     try {
-      HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
       StorageClusterVersionModel model = new StorageClusterVersionModel();
-      model.setVersion(admin.getClusterStatus().getHBaseVersion());
+      model.setVersion(servlet.getAdmin().getClusterStatus().getHBaseVersion());
       ResponseBuilder response = Response.ok(model);
       response.cacheControl(cacheControl);
       servlet.getMetrics().incrementSucessfulGetRequests(1);
       return response.build();
     } catch (IOException e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
-      throw new WebApplicationException(e, 
-                  Response.Status.SERVICE_UNAVAILABLE);
+      return Response.status(Response.Status.SERVICE_UNAVAILABLE)
+        .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF)
+        .build();
     }
   }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java Thu Feb 14 12:58:12 2013
@@ -27,7 +27,6 @@ import javax.ws.rs.PathParam;
 import javax.ws.rs.QueryParam;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 
 @InterfaceAudience.Private
 public class TableResource extends ResourceBase {
@@ -54,12 +53,7 @@ public class TableResource extends Resou
    * @throws IOException
    */
   boolean exists() throws IOException {
-    HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
-    try {
-      return admin.tableExists(table);
-    } finally {
-      admin.close();
-    }
+    return servlet.getAdmin().tableExists(table);
   }
 
   @Path("exists")

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/VersionResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/VersionResource.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/VersionResource.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/VersionResource.java Thu Feb 14 12:58:12 2013
@@ -71,7 +71,8 @@ public class VersionResource extends Res
    * @return a response for a version request 
    */
   @GET
-  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
   public Response get(final @Context ServletContext context, 
       final @Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java Thu Feb 14 12:58:12 2013
@@ -35,7 +35,6 @@ import org.apache.commons.logging.LogFac
 
 import org.apache.hadoop.hbase.client.*;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
-import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.util.StringUtils;
 
@@ -55,7 +54,6 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.RowLock;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.rest.Constants;
@@ -82,7 +80,7 @@ public class RemoteHTable implements HTa
   final long sleepTime;
 
   @SuppressWarnings("rawtypes")
-  protected String buildRowSpec(final byte[] row, final Map familyMap, 
+  protected String buildRowSpec(final byte[] row, final Map familyMap,
       final long startTime, final long endTime, final int maxVersions) {
     StringBuffer sb = new StringBuffer();
     sb.append('/');
@@ -174,7 +172,7 @@ public class RemoteHTable implements HTa
         byte[][] split = KeyValue.parseColumn(cell.getColumn());
         byte[] column = split[0];
         byte[] qualifier = split.length > 1 ? split[1] : null;
-        kvs.add(new KeyValue(row.getKey(), column, qualifier, 
+        kvs.add(new KeyValue(row.getKey(), column, qualifier,
           cell.getTimestamp(), cell.getValue()));
       }
       results.add(new Result(kvs));
@@ -252,7 +250,7 @@ public class RemoteHTable implements HTa
         TableSchemaModel schema = new TableSchemaModel();
         schema.getObjectFromMessage(response.getBody());
         return schema.getTableDescriptor();
-      case 509: 
+      case 509:
         try {
           Thread.sleep(sleepTime);
         } catch (InterruptedException e) { }
@@ -346,6 +344,19 @@ public class RemoteHTable implements HTa
     return (result != null && !(result.isEmpty()));
   }
 
+  /**
+   * exists(List) is really a list of get() calls. Just use get().
+   * @param gets list of Get to test for the existence
+   */
+  public Boolean[] exists(List<Get> gets) throws IOException {
+    LOG.warn("exists(List<Get>) is really list of get() calls, just use get()");
+    Boolean[] results = new Boolean[gets.size()];
+    for (int i = 0; i < results.length; i++) {
+      results[i] = exists(gets.get(i));
+    }
+    return results;
+  }
+
   public void put(Put put) throws IOException {
     CellSetModel model = buildModelFromPut(put);
     StringBuilder sb = new StringBuilder();
@@ -529,7 +540,7 @@ public class RemoteHTable implements HTa
       }
       return results[0];
     }
-    
+
     class Iter implements Iterator<Result> {
 
       Result cache;
@@ -563,7 +574,7 @@ public class RemoteHTable implements HTa
       public void remove() {
         throw new RuntimeException("remove() not supported");
       }
-      
+
     }
 
     @Override
@@ -607,14 +618,6 @@ public class RemoteHTable implements HTa
     throw new IOException("getRowOrBefore not supported");
   }
 
-  public RowLock lockRow(byte[] row) throws IOException {
-    throw new IOException("lockRow not implemented");
-  }
-
-  public void unlockRow(RowLock rl) throws IOException {
-    throw new IOException("unlockRow not implemented");
-  }
-
   public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
       byte[] value, Put put) throws IOException {
     // column to check-the-value
@@ -726,29 +729,6 @@ public class RemoteHTable implements HTa
   }
 
   @Override
-  public <T extends CoprocessorProtocol> T coprocessorProxy(Class<T> protocol,
-      byte[] row) {
-    throw new
-    UnsupportedOperationException("coprocessorProxy not implemented");
-  }
-
-  @Override
-  public <T extends CoprocessorProtocol, R> Map<byte[], R> coprocessorExec(
-      Class<T> protocol, byte[] startKey, byte[] endKey,
-      Batch.Call<T, R> callable)
-      throws IOException, Throwable {
-    throw new UnsupportedOperationException("coprocessorExec not implemented");
-  }
-
-  @Override
-  public <T extends CoprocessorProtocol, R> void coprocessorExec(
-      Class<T> protocol, byte[] startKey, byte[] endKey,
-      Batch.Call<T, R> callable, Batch.Callback<R> callback)
-      throws IOException, Throwable {
-    throw new UnsupportedOperationException("coprocessorExec not implemented");
-  }
-
-  @Override
   public CoprocessorRpcChannel coprocessorService(byte[] row) {
     throw new UnsupportedOperationException("coprocessorService not implemented");
   }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterStatusModel.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterStatusModel.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterStatusModel.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterStatusModel.java Thu Feb 14 12:58:12 2013
@@ -638,7 +638,7 @@ public class StorageClusterStatusModel 
 	public String toString() {
 	  StringBuilder sb = new StringBuilder();
 	  sb.append(String.format("%d live servers, %d dead servers, " + 
-      "%.4f average load\n\n", liveNodes.size(), deadNodes.size(),
+      "%.4f average load%n%n", liveNodes.size(), deadNodes.size(),
       averageLoad));
     if (!liveNodes.isEmpty()) {
       sb.append(liveNodes.size());

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/provider/consumer/ProtobufMessageBodyConsumer.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/provider/consumer/ProtobufMessageBodyConsumer.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/provider/consumer/ProtobufMessageBodyConsumer.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/provider/consumer/ProtobufMessageBodyConsumer.java Thu Feb 14 12:58:12 2013
@@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.rest.Prot
  * ProtobufMessageHandler interface capable handlers for decoding protobuf input.
  */
 @Provider
-@Consumes(Constants.MIMETYPE_PROTOBUF)
+@Consumes({Constants.MIMETYPE_PROTOBUF, Constants.MIMETYPE_PROTOBUF_IETF})
 @InterfaceAudience.Private
 public class ProtobufMessageBodyConsumer 
     implements MessageBodyReader<ProtobufMessageHandler> {

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java Thu Feb 14 12:58:12 2013
@@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.rest.Prot
  * sent, then writeTo to perform the actual I/O.
  */
 @Provider
-@Produces(Constants.MIMETYPE_PROTOBUF)
+@Produces({Constants.MIMETYPE_PROTOBUF, Constants.MIMETYPE_PROTOBUF_IETF})
 @InterfaceAudience.Private
 public class ProtobufMessageBodyProducer
   implements MessageBodyWriter<ProtobufMessageHandler> {

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java Thu Feb 14 12:58:12 2013
@@ -32,7 +32,7 @@ import org.apache.hadoop.security.author
  * protocol interfaces to hbase-policy.xml entries.
  */
 public class HBasePolicyProvider extends PolicyProvider {
-  protected static Service[] services = {
+  protected final static Service[] services = {
       new Service("security.client.protocol.acl", ClientProtocol.class),
       new Service("security.client.protocol.acl", AdminProtocol.class),
       new Service("security.admin.protocol.acl", MasterMonitorProtocol.class),
@@ -48,7 +48,7 @@ public class HBasePolicyProvider extends
   public static void init(Configuration conf,
       ServiceAuthorizationManager authManager) {
     // set service-level authorization security policy
-    conf.set("hadoop.policy.file", "hbase-policy.xml");
+    System.setProperty("hadoop.policy.file", "hbase-policy.xml");
     if (conf.getBoolean(
           ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, false)) {
       authManager.refresh(conf, new HBasePolicyProvider());

Modified: hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java?rev=1446147&r1=1446146&r2=1446147&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java Thu Feb 14 12:58:12 2013
@@ -18,9 +18,17 @@
 
 package org.apache.hadoop.hbase.security.access;
 
-import com.google.protobuf.InvalidProtocolBufferException;
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.ListMultimap;
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -37,26 +45,23 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.io.HbaseObjectWritable;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.filter.RegexStringComparator;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.hbase.filter.RegexStringComparator;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.Text;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.*;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * Maintains lists of permission grants to users and groups to allow for
@@ -71,11 +76,11 @@ import java.util.*;
  * Key                      Desc
  * --------                 --------
  * user                     table level permissions for a user [R=read, W=write]
- * @group                   table level permissions for a group
+ * group                    table level permissions for a group
  * user,family              column family level permissions for a user
- * @group,family            column family level permissions for a group
+ * group,family             column family level permissions for a group
  * user,family,qualifier    column qualifier level permissions for a user
- * @group,family,qualifier  column qualifier level permissions for a group
+ * group,family,qualifier   column qualifier level permissions for a group
  * </pre>
  * All values are encoded as byte arrays containing the codes from the
  * {@link org.apache.hadoop.hbase.security.access.TablePermission.Action} enum.
@@ -98,7 +103,7 @@ public class AccessControlLists {
         new HColumnDescriptor(ACL_LIST_FAMILY,
             10, // Ten is arbitrary number.  Keep versions to help debugging.
             Compression.Algorithm.NONE.getName(), true, true, 8 * 1024,
-            HConstants.FOREVER, StoreFile.BloomType.NONE.toString(),
+            HConstants.FOREVER, BloomType.NONE.toString(),
             HConstants.REPLICATION_SCOPE_LOCAL));
   }
 
@@ -302,7 +307,7 @@ public class AccessControlLists {
    * table.
    *
    * @param aclRegion
-   * @return
+   * @return a map of the permissions for this table.
    * @throws IOException
    */
   static Map<byte[],ListMultimap<String,TablePermission>> loadAll(
@@ -510,7 +515,8 @@ public class AccessControlLists {
    *
    * Writes a set of permission [user: table permission]
    */
-  public static byte[] writePermissionsAsBytes(ListMultimap<String, TablePermission> perms, Configuration conf) {
+  public static byte[] writePermissionsAsBytes(ListMultimap<String, TablePermission> perms,
+      Configuration conf) {
     return ProtobufUtil.prependPBMagic(ProtobufUtil.toUserTablePermissions(perms).toByteArray());
   }
 
@@ -519,7 +525,8 @@ public class AccessControlLists {
    * from the input stream.
    */
   public static ListMultimap<String, TablePermission> readPermissions(byte[] data,
-      Configuration conf) throws DeserializationException {
+      Configuration conf)
+  throws DeserializationException {
     if (ProtobufUtil.isPBMagicPrefix(data)) {
       int pblen = ProtobufUtil.lengthOfPBMagic();
       try {
@@ -537,7 +544,8 @@ public class AccessControlLists {
         int length = in.readInt();
         for (int i=0; i<length; i++) {
           String user = Text.readString(in);
-          List<TablePermission> userPerms = (List)HbaseObjectWritable.readObject(in, conf);
+          List<TablePermission> userPerms =
+            (List)HbaseObjectWritableFor96Migration.readObject(in, conf);
           perms.putAll(user, userPerms);
         }
       } catch (IOException e) {