You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2010/04/09 04:10:03 UTC

svn commit: r932214 [1/4] - in /hadoop/hbase/branches/0.20_pre_durability: ./ src/contrib/stargate/ src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/auth/ src/contrib/stargat...

Author: apurtell
Date: Fri Apr  9 02:10:02 2010
New Revision: 932214

URL: http://svn.apache.org/viewvc?rev=932214&view=rev
Log:
HBASE-2402 [stargate] set maxVersions on gets
HBASE-2412 [stargate] PerformanceEvaluation
HBASE-2319 [stargate] multiuser mode: request shaping
HBASE-2403 [stargate] client HTable interface to REST connector

Added:
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ExistsResource.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/User.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/client/RemoteAdmin.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/client/RemoteHTable.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/util/
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/util/HTableTokenBucket.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/util/SoftUserData.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/util/TokenBucket.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/util/UserData.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/PerformanceEvaluation.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/client/
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/client/TestRemoteAdmin.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/client/TestRemoteTable.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/util/
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/util/TestHTableTokenBucket.java
Removed:
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/auth/User.java
Modified:
    hadoop/hbase/branches/0.20_pre_durability/CHANGES.txt
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/build.xml
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/Constants.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/Main.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RESTServlet.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RegionsResource.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ResourceConfig.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ResultGenerator.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RootResource.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResource.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResultGenerator.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowSpec.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ScannerInstanceResource.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ScannerResource.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ScannerResultGenerator.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/SchemaResource.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/StorageClusterStatusResource.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/StorageClusterVersionResource.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/TableResource.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/VersionResource.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/auth/Authenticator.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/auth/HBCAuthenticator.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/auth/HTableAuthenticator.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/auth/JDBCAuthenticator.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/auth/ZooKeeperAuthenticator.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/client/Client.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/client/Cluster.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/model/CellModel.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/model/ModelSchema.xsd
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/model/ScannerModel.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/model/TableRegionModel.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/model/TableSchemaModel.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/protobuf/CellMessage.proto
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/protobuf/CellSetMessage.proto
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/protobuf/ColumnSchemaMessage.proto
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/protobuf/ScannerMessage.proto
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/protobuf/StorageClusterStatusMessage.proto
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/protobuf/TableInfoMessage.proto
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/protobuf/TableSchemaMessage.proto
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/protobuf/VersionMessage.proto
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/MiniClusterTestCase.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/auth/TestHBCAuthenticator.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/auth/TestHTableAuthenticator.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/auth/TestJDBCAuthenticator.java
    hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/auth/TestZooKeeperAuthenticator.java

Modified: hadoop/hbase/branches/0.20_pre_durability/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/CHANGES.txt?rev=932214&r1=932213&r2=932214&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.20_pre_durability/CHANGES.txt Fri Apr  9 02:10:02 2010
@@ -39,6 +39,7 @@ Release 0.20.4 - Unreleased
    HBASE-2335  mapred package docs don't say zookeeper jar is a dependent
    HBASE-2417  HCM.locateRootRegion fails hard on "Connection refused"
    HBASE-2346  Usage of FilterList slows down scans
+   HBASE-2402  [stargate] set maxVersions on gets
 
   IMPROVEMENTS
    HBASE-2180  Bad read performance from synchronizing hfile.fddatainputstream
@@ -65,12 +66,15 @@ Release 0.20.4 - Unreleased
                (Todd Lipcon via Stack)
    HBASE-2423  Update 'Getting Started' for 0.20.4 including making 
                "important configurations more visiable"
+   HBASE-2412  [stargate] PerformanceEvaluation
 
   NEW FEATURES
    HBASE-2257  [stargate] multiuser mode
    HBASE-2263  [stargate] multiuser mode: authenticator for zookeeper
    HBASE-2273  [stargate] export metrics via Hadoop metrics, JMX, and zookeeper
    HBASE-2274  [stargate] filter support: JSON descriptors
+   HBASE-2319  [stargate] multiuser mode: request shaping
+   HBASE-2403  [stargate] client HTable interface to REST connector
 
 Release 0.20.3 - January 25th, 2010
   INCOMPATIBLE CHANGES

Modified: hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/build.xml
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/build.xml?rev=932214&r1=932213&r2=932214&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/build.xml (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/build.xml Fri Apr  9 02:10:02 2010
@@ -1,3 +1,20 @@
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
 <project name="stargate" default="package" basedir=".">
   <import file="../build-contrib.xml"/>
 
@@ -46,6 +63,15 @@
     <javac srcdir="${src.test}" includes="**/*.java" destdir="${build.test}" debug="${javac.debug}" source="1.6">
       <classpath refid="test.classpath"/>
     </javac>
+    <jar jarfile="${build.dir}/${test.jar.file}" >
+      <fileset dir="${build.test}" includes="org/**" />
+      <fileset dir="${build.classes}" />
+      <fileset dir="${src.test}" includes="**/*.properties" />
+      <manifest>
+        <attribute name="Main-Class"
+          value="org/apache/hadoop/hbase/stargate/PerformanceEvaluation"/>
+      </manifest>
+    </jar>
   </target>
   
   <target name="test" depends="compile-test" description="Build test jar and run tests">

Modified: hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/Constants.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/Constants.java?rev=932214&r1=932213&r2=932214&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/Constants.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/Constants.java Fri Apr  9 02:10:02 2010
@@ -31,10 +31,12 @@ public interface Constants {
   public static final String STATUS_REPORT_PERIOD_KEY =
     "stargate.status.period";
 
+  public static final String USERS_TABLE = "users";
+
   public static final String INSTANCE_ZNODE_ROOT = "/stargate/instance";
   public static final String USERS_ZNODE_ROOT = "/stargate/users";
 
-  public static final int DEFAULT_MAX_AGE = 60 * 60 * 4;       // 4 hours
+  public static final int DEFAULT_MAX_AGE = 60 * 60 * 4;  // 4 hours
 
   public static final String MIMETYPE_TEXT = "text/plain";
   public static final String MIMETYPE_HTML = "text/html";

Added: hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ExistsResource.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ExistsResource.java?rev=932214&view=auto
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ExistsResource.java (added)
+++ hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ExistsResource.java Fri Apr  9 02:10:02 2010
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.stargate;
+
+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;
+import javax.ws.rs.core.UriInfo;
+import javax.ws.rs.core.Response.ResponseBuilder;
+
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+
+public class ExistsResource implements Constants {
+
+  User user;
+  String tableName;
+  String actualTableName;
+  CacheControl cacheControl;
+  RESTServlet servlet;
+
+  public ExistsResource(User user, String table) throws IOException {
+    if (user != null) {
+      this.user = user;
+      this.actualTableName = 
+        !user.isAdmin() ? (user.getName() + "." + table) : table;
+    } else {
+      this.actualTableName = table;
+    }
+    this.tableName = table;
+    servlet = RESTServlet.getInstance();
+    cacheControl = new CacheControl();
+    cacheControl.setNoCache(true);
+    cacheControl.setNoTransform(false);
+  }
+
+  @GET
+  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_BINARY})
+  public Response get(final @Context UriInfo uriInfo) throws IOException {
+    if (!servlet.userRequestLimit(user, 1)) {
+      Response.status(509).build();
+    }
+    try {
+      HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
+      if (!admin.tableExists(actualTableName)) {
+        throw new WebApplicationException(Response.Status.NOT_FOUND);
+      }
+    } catch (IOException e) {
+      throw new WebApplicationException(Response.Status.SERVICE_UNAVAILABLE);
+    }
+    ResponseBuilder response = Response.ok();
+    response.cacheControl(cacheControl);
+    return response.build();
+  }
+
+}

Modified: hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/Main.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/Main.java?rev=932214&r1=932213&r2=932214&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/Main.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/Main.java Fri Apr  9 02:10:02 2010
@@ -1,88 +1,93 @@
-/*
- * Copyright 2009 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.stargate;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.PosixParser;
-import org.mortbay.jetty.Connector;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.servlet.Context;
-import org.mortbay.jetty.servlet.ServletHolder;
-
-import com.sun.jersey.spi.container.servlet.ServletContainer;
-
-/**
- * Main class for launching Stargate as a servlet hosted by an embedded Jetty
- * servlet container.
- * <p> 
- * The following options are supported:
- * <ul>
- * <li>-p: service port</li>
- * </ul>
- */
-public class Main implements Constants {
-
-  public static void main(String[] args) throws Exception {
-    // process command line
-
-    Options options = new Options();
-    options.addOption("p", "port", true, "service port");
-    options.addOption("m", "multiuser", false, "enable multiuser mode");
-    CommandLineParser parser = new PosixParser();
-    CommandLine cmd = parser.parse(options, args);
-    int port = 8080;
-    if (cmd.hasOption("p")) {
-      port = Integer.valueOf(cmd.getOptionValue("p"));
-    }
-
-    // set up the Jersey servlet container for Jetty
-
-    ServletHolder sh = new ServletHolder(ServletContainer.class);
-    sh.setInitParameter(
-      "com.sun.jersey.config.property.resourceConfigClass",
-      ResourceConfig.class.getCanonicalName());
-    sh.setInitParameter("com.sun.jersey.config.property.packages",
-      "jetty");
-
-    // set up Jetty and run the embedded server
-
-    Server server = new Server(port);
-    server.setSendServerVersion(false);
-    server.setSendDateHeader(false);
-    server.setStopAtShutdown(true);
-      // set up context
-    Context context = new Context(server, "/", Context.SESSIONS);
-    context.addServlet(sh, "/*");
-
-    // configure the Stargate singleton
-
-    RESTServlet servlet = RESTServlet.getInstance();
-    servlet.setMultiUser(cmd.hasOption("m"));
-    for (Connector conn: server.getConnectors()) {
-      servlet.addConnectorAddress(conn.getHost(), conn.getLocalPort());
-    }
-
-    server.start();
-    server.join();
-  }
-}
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.stargate;
+
+import java.net.InetAddress;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.PosixParser;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.servlet.Context;
+import org.mortbay.jetty.servlet.ServletHolder;
+
+import com.sun.jersey.spi.container.servlet.ServletContainer;
+
+/**
+ * Main class for launching Stargate as a servlet hosted by an embedded Jetty
+ * servlet container.
+ * <p> 
+ * The following options are supported:
+ * <ul>
+ * <li>-p: service port</li>
+ * </ul>
+ */
+public class Main implements Constants {
+
+  public static void main(String[] args) throws Exception {
+    // process command line
+
+    Options options = new Options();
+    options.addOption("p", "port", true, "service port");
+    options.addOption("m", "multiuser", false, "enable multiuser mode");
+    CommandLineParser parser = new PosixParser();
+    CommandLine cmd = parser.parse(options, args);
+    int port = 8080;
+    if (cmd.hasOption("p")) {
+      port = Integer.valueOf(cmd.getOptionValue("p"));
+    }
+
+    // set up the Jersey servlet container for Jetty
+
+    ServletHolder sh = new ServletHolder(ServletContainer.class);
+    sh.setInitParameter(
+      "com.sun.jersey.config.property.resourceConfigClass",
+      ResourceConfig.class.getCanonicalName());
+    sh.setInitParameter("com.sun.jersey.config.property.packages",
+      "jetty");
+
+    // configure the Stargate singleton
+
+    RESTServlet servlet = RESTServlet.getInstance();
+    port = servlet.getConfiguration().getInt("stargate.port", port);
+    if (!servlet.isMultiUser()) {
+      servlet.setMultiUser(cmd.hasOption("m"));
+    }
+    servlet.addConnectorAddress(
+      servlet.getConfiguration().get("stargate.hostname",
+        InetAddress.getLocalHost().getCanonicalHostName()),
+      port);
+
+    // set up Jetty and run the embedded server
+
+    Server server = new Server(port);
+    server.setSendServerVersion(false);
+    server.setSendDateHeader(false);
+    server.setStopAtShutdown(true);
+      // set up context
+    Context context = new Context(server, "/", Context.SESSIONS);
+    context.addServlet(sh, "/*");
+
+    server.start();
+    server.join();
+  }
+}

Modified: hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RESTServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RESTServlet.java?rev=932214&r1=932213&r2=932214&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RESTServlet.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RESTServlet.java Fri Apr  9 02:10:02 2010
@@ -43,8 +43,12 @@ import org.apache.hadoop.hbase.stargate.
 import org.apache.hadoop.hbase.stargate.auth.JDBCAuthenticator;
 import org.apache.hadoop.hbase.stargate.auth.ZooKeeperAuthenticator;
 import org.apache.hadoop.hbase.stargate.metrics.StargateMetrics;
+import org.apache.hadoop.hbase.stargate.util.HTableTokenBucket;
+import org.apache.hadoop.hbase.stargate.util.SoftUserData;
+import org.apache.hadoop.hbase.stargate.util.UserData;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
 
 import org.apache.hadoop.util.StringUtils;
@@ -95,13 +99,12 @@ public class RESTServlet extends Servlet
         }
         status.endArray();
         status.endObject();
-        updateNode(wrapper, znode, CreateMode.EPHEMERAL, 
+        ensureExists(znode, CreateMode.EPHEMERAL, 
           Bytes.toBytes(status.toString()));
       } catch (Exception e) {
         LOG.error(StringUtils.stringifyException(e));
       }
     }
-
   }
 
   final String znode = INSTANCE_ZNODE_ROOT + "/" + System.currentTimeMillis();
@@ -129,49 +132,38 @@ public class RESTServlet extends Servlet
     return instance;
   }
 
-  static boolean ensureExists(final ZooKeeperWrapper zkw, final String znode,
-      final CreateMode mode) throws IOException {
-    ZooKeeper zk = zkw.getZooKeeper();
+  private boolean ensureExists(final String znode, final CreateMode mode,
+      final byte[] data) {
     try {
+      ZooKeeper zk = wrapper.getZooKeeper();
       Stat stat = zk.exists(znode, false);
       if (stat != null) {
+        zk.setData(znode, data, -1);
         return true;
       }
-      zk.create(znode, new byte[0], Ids.OPEN_ACL_UNSAFE, mode);
-      LOG.debug("Created ZNode " + znode);
+      zk.create(znode, data, Ids.OPEN_ACL_UNSAFE, mode);
+      LOG.info("Created ZNode " + znode);
       return true;
     } catch (KeeperException.NodeExistsException e) {
       return true;      // ok, move on.
     } catch (KeeperException.NoNodeException e) {
-      return ensureParentExists(zkw, znode, mode) && 
-        ensureExists(zkw, znode, mode);
+      return ensureParentExists(znode, CreateMode.PERSISTENT, new byte[]{}) &&
+        ensureExists(znode, mode, data);
     } catch (KeeperException e) {
-      throw new IOException(e);
+      LOG.warn(StringUtils.stringifyException(e));
     } catch (InterruptedException e) {
-      throw new IOException(e);
+      LOG.warn(StringUtils.stringifyException(e));
     }
+    return false;
   }
 
-  static boolean ensureParentExists(final ZooKeeperWrapper zkw,
-      final String znode, final CreateMode mode) throws IOException {
-    int index = znode.lastIndexOf("/");
+  private boolean ensureParentExists(final String znode, final CreateMode mode,
+      final byte[] data) {
+    int index = znode.lastIndexOf('/');
     if (index <= 0) {   // Parent is root, which always exists.
       return true;
     }
-    return ensureExists(zkw, znode.substring(0, index), mode);
-  }
-
-  static void updateNode(final ZooKeeperWrapper zkw, final String znode, 
-        final CreateMode mode, final byte[] data) throws IOException  {
-    ensureExists(zkw, znode, mode);
-    ZooKeeper zk = zkw.getZooKeeper();
-    try {
-      zk.setData(znode, data, -1);
-    } catch (KeeperException e) {
-      throw new IOException(e);
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
+    return ensureExists(znode.substring(0, index), mode, data);
   }
 
   ZooKeeperWrapper initZooKeeperWrapper() throws IOException {
@@ -187,7 +179,13 @@ public class RESTServlet extends Servlet
     this.pool = new HTablePool(conf, 10);
     this.wrapper = initZooKeeperWrapper();
     this.statusReporter = new StatusReporter(
-      conf.getInt(STATUS_REPORT_PERIOD_KEY, 1000 * 60), stopping);
+      conf.getInt(STATUS_REPORT_PERIOD_KEY, 1000 * 30), stopping);
+    Threads.setDaemonThreadRunning(statusReporter, "Stargate.statusReporter");
+    this.multiuser = conf.getBoolean("stargate.multiuser", false);
+    if (this.multiuser) {
+      LOG.info("multiuser mode enabled");
+      getAuthenticator();
+    }
   }
 
   @Override
@@ -316,6 +314,7 @@ public class RESTServlet extends Servlet
       if (authenticator == null) {
         authenticator = new HBCAuthenticator(conf);
       }
+      LOG.info("using authenticator " + authenticator);
     }
     return authenticator;
   }
@@ -327,4 +326,28 @@ public class RESTServlet extends Servlet
     this.authenticator = authenticator;
   }
 
+  /**
+   * Check if the user has exceeded their request token limit within the
+   * current interval
+   * @param user the user
+   * @param want the number of tokens desired
+   * @throws IOException
+   */
+  public boolean userRequestLimit(final User user, int want) 
+      throws IOException {
+    if (multiuser) {
+      UserData ud = SoftUserData.get(user);
+      HTableTokenBucket tb = (HTableTokenBucket) ud.get(UserData.TOKENBUCKET);
+      if (tb == null) {
+        tb = new HTableTokenBucket(conf, Bytes.toBytes(user.getToken()));
+        ud.put(UserData.TOKENBUCKET, tb);
+      }
+      if (tb.available() < want) {
+        return false;
+      }
+      tb.remove(want);
+    }
+    return true;
+  }
+
 }

Modified: hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RegionsResource.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RegionsResource.java?rev=932214&r1=932213&r2=932214&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RegionsResource.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RegionsResource.java Fri Apr  9 02:10:02 2010
@@ -1,109 +1,123 @@
-/*
- * Copyright 2010 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.stargate;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-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;
-import javax.ws.rs.core.UriInfo;
-import javax.ws.rs.core.Response.ResponseBuilder;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HServerAddress;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HTablePool;
-import org.apache.hadoop.hbase.stargate.auth.User;
-import org.apache.hadoop.hbase.stargate.model.TableInfoModel;
-import org.apache.hadoop.hbase.stargate.model.TableRegionModel;
-
-public class RegionsResource implements Constants {
-  private static final Log LOG = LogFactory.getLog(RegionsResource.class);
-
-  User user;
-  String table;
-  CacheControl cacheControl;
-  RESTServlet servlet;
-
-  public RegionsResource(User user, String table) throws IOException {
-    if (user != null) {
-      if (!user.isAdmin()) {
-        throw new WebApplicationException(Response.Status.FORBIDDEN);
-      }
-      this.user = user;
-    }
-    this.table = table;
-    cacheControl = new CacheControl();
-    cacheControl.setNoCache(true);
-    cacheControl.setNoTransform(false);
-    servlet = RESTServlet.getInstance();
-  }
-
-  private Map<HRegionInfo,HServerAddress> getTableRegions()
-      throws IOException {
-    HTablePool pool = servlet.getTablePool();
-    HTable table = pool.getTable(this.table);
-    try {
-      return table.getRegionsInfo();
-    } finally {
-      pool.putTable(table);
-    }
-  }
-
-  @GET
-  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
-  public Response get(@Context UriInfo uriInfo) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("GET " + uriInfo.getAbsolutePath());
-    }
-    servlet.getMetrics().incrementRequests(1);
-    try {
-      TableInfoModel model = new TableInfoModel(table);
-      Map<HRegionInfo,HServerAddress> regions = getTableRegions();
-      for (Map.Entry<HRegionInfo,HServerAddress> e: regions.entrySet()) {
-        HRegionInfo hri = e.getKey();
-        HServerAddress addr = e.getValue();
-        InetSocketAddress sa = addr.getInetSocketAddress();
-        model.add(
-          new TableRegionModel(table, hri.getRegionId(), hri.getStartKey(),
-                hri.getEndKey(),
-                sa.getHostName() + ":" + Integer.valueOf(sa.getPort())));
-      }
-      ResponseBuilder response = Response.ok(model);
-      response.cacheControl(cacheControl);
-      return response.build();
-    } catch (TableNotFoundException e) {
-      throw new WebApplicationException(Response.Status.NOT_FOUND);
-    } catch (IOException e) {
-      throw new WebApplicationException(e,
-                  Response.Status.SERVICE_UNAVAILABLE);
-    }
-  }
-}
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.stargate;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+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;
+import javax.ws.rs.core.UriInfo;
+import javax.ws.rs.core.Response.ResponseBuilder;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTablePool;
+import org.apache.hadoop.hbase.stargate.User;
+import org.apache.hadoop.hbase.stargate.model.TableInfoModel;
+import org.apache.hadoop.hbase.stargate.model.TableRegionModel;
+
+public class RegionsResource implements Constants {
+  private static final Log LOG = LogFactory.getLog(RegionsResource.class);
+
+  User user;
+  String tableName;
+  String actualTableName;
+  CacheControl cacheControl;
+  RESTServlet servlet;
+
+  public RegionsResource(User user, String table) throws IOException {
+    if (user != null) {
+      this.user = user;
+      this.actualTableName = 
+        !user.isAdmin() ? (user.getName() + "." + table) : table;
+    } else {
+      this.actualTableName = table;
+    }
+    this.tableName = table;
+    cacheControl = new CacheControl();
+    cacheControl.setNoCache(true);
+    cacheControl.setNoTransform(false);
+    servlet = RESTServlet.getInstance();
+  }
+
+  private Map<HRegionInfo,HServerAddress> getTableRegions()
+      throws IOException {
+    HTablePool pool = servlet.getTablePool();
+    HTable table = pool.getTable(actualTableName);
+    try {
+      return table.getRegionsInfo();
+    } finally {
+      pool.putTable(table);
+    }
+  }
+
+  @GET
+  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  public Response get(final @Context UriInfo uriInfo) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("GET " + uriInfo.getAbsolutePath());
+    }
+    if (!servlet.userRequestLimit(user, 1)) {
+      Response.status(509).build();
+    }
+    servlet.getMetrics().incrementRequests(1);
+    try {
+      String name = user.isAdmin() ? actualTableName : tableName;
+      TableInfoModel model = new TableInfoModel(name);
+      Map<HRegionInfo,HServerAddress> regions = getTableRegions();
+      for (Map.Entry<HRegionInfo,HServerAddress> e: regions.entrySet()) {
+        HRegionInfo hri = e.getKey();
+        if (user.isAdmin()) {
+          HServerAddress addr = e.getValue();
+          InetSocketAddress sa = addr.getInetSocketAddress();
+          model.add(
+            new TableRegionModel(name, hri.getRegionId(), hri.getStartKey(),
+              hri.getEndKey(),
+              sa.getHostName() + ":" + Integer.valueOf(sa.getPort())));
+        } else {
+          model.add(
+            new TableRegionModel(name, hri.getRegionId(), hri.getStartKey(),
+              hri.getEndKey()));
+        }
+      }
+      ResponseBuilder response = Response.ok(model);
+      response.cacheControl(cacheControl);
+      return response.build();
+    } catch (TableNotFoundException e) {
+      throw new WebApplicationException(Response.Status.NOT_FOUND);
+    } catch (IOException e) {
+      throw new WebApplicationException(e,
+                  Response.Status.SERVICE_UNAVAILABLE);
+    }
+  }
+
+}

Modified: hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ResourceConfig.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ResourceConfig.java?rev=932214&r1=932213&r2=932214&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ResourceConfig.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ResourceConfig.java Fri Apr  9 02:10:02 2010
@@ -1,29 +1,29 @@
-/*
- * Copyright 2010 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.stargate;
-
-import com.sun.jersey.api.core.PackagesResourceConfig;
-
-public class ResourceConfig extends PackagesResourceConfig {
-  public ResourceConfig() {
-    super("org.apache.hadoop.hbase.stargate");
-  }
-}
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.stargate;
+
+import com.sun.jersey.api.core.PackagesResourceConfig;
+
+public class ResourceConfig extends PackagesResourceConfig {
+  public ResourceConfig() {
+    super("org.apache.hadoop.hbase.stargate");
+  }
+}

Modified: hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ResultGenerator.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ResultGenerator.java?rev=932214&r1=932213&r2=932214&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ResultGenerator.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ResultGenerator.java Fri Apr  9 02:10:02 2010
@@ -1,49 +1,51 @@
-/*
- * Copyright 2010 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.stargate;
-
-import java.io.IOException;
-import java.util.Iterator;
- 
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.stargate.model.ScannerModel;
-
-import org.json.JSONObject;
-
-public abstract class ResultGenerator implements Iterator<KeyValue> {
-
-  public static ResultGenerator fromRowSpec(final String table, 
-      final RowSpec rowspec, final Filter filter) throws IOException {
-    if (rowspec.isSingleRow()) {
-      return new RowResultGenerator(table, rowspec, filter);
-    } else {
-      return new ScannerResultGenerator(table, rowspec, filter);
-    }
-  }
-
-  public static Filter buildFilter(String filter) throws Exception {
-    return ScannerModel.buildFilter(new JSONObject(filter));
-  }
-
-  public abstract void close();
-
-}
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.stargate;
+
+import java.io.IOException;
+import java.util.Iterator;
+ 
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.stargate.model.ScannerModel;
+
+import org.json.JSONObject;
+
+public abstract class ResultGenerator implements Iterator<KeyValue> {
+
+  public static ResultGenerator fromRowSpec(final String table, 
+      final RowSpec rowspec, final Filter filter) throws IOException {
+    if (rowspec.isSingleRow()) {
+      return new RowResultGenerator(table, rowspec, filter);
+    } else {
+      return new ScannerResultGenerator(table, rowspec, filter);
+    }
+  }
+
+  public static Filter buildFilter(final String filter) throws Exception {
+    return ScannerModel.buildFilter(new JSONObject(filter));
+  }
+
+  public abstract void putBack(KeyValue kv);
+
+  public abstract void close();
+
+}

Modified: hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RootResource.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RootResource.java?rev=932214&r1=932213&r2=932214&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RootResource.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RootResource.java Fri Apr  9 02:10:02 2010
@@ -1,177 +1,185 @@
-/*
- * Copyright 2010 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.stargate;
-
-import java.io.IOException;
-
-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;
-import javax.ws.rs.core.UriInfo;
-import javax.ws.rs.core.Response.ResponseBuilder;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.stargate.auth.User;
-import org.apache.hadoop.hbase.stargate.model.TableListModel;
-import org.apache.hadoop.hbase.stargate.model.TableModel;
-
-@Path("/")
-public class RootResource implements Constants {
-  private static final Log LOG = LogFactory.getLog(RootResource.class);
-
-  RESTServlet servlet;
-  CacheControl cacheControl;
-
-  public RootResource() throws IOException {
-    servlet = RESTServlet.getInstance();
-    cacheControl = new CacheControl();
-    cacheControl.setNoCache(true);
-    cacheControl.setNoTransform(false);
-  }
-
-  TableListModel getTableList() throws IOException {
-    TableListModel tableList = new TableListModel();
-    HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
-    HTableDescriptor[] list = admin.listTables();
-    for (HTableDescriptor htd: list) {
-      tableList.add(new TableModel(htd.getNameAsString()));
-    }
-    return tableList;
-  }
-
-  TableListModel getTableListForUser(User user) throws IOException {
-    TableListModel tableList;
-    if (user.isAdmin()) {
-      tableList = getTableList();
-    } else {
-      tableList = new TableListModel();
-      HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
-      HTableDescriptor[] list = admin.listTables();
-      String prefix = user.getName() + ".";
-      for (HTableDescriptor htd: list) {
-        String name = htd.getNameAsString();
-        if (!name.startsWith(prefix)) {
-          continue;
-        }
-        tableList.add(new TableModel(name.substring(prefix.length())));
-      }
-    }
-    return tableList;
-  }
-
-  @GET
-  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
-  public Response get(@Context UriInfo uriInfo) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("GET " + uriInfo.getAbsolutePath());
-    }
-    servlet.getMetrics().incrementRequests(1);
-    if (servlet.isMultiUser()) {
-      throw new WebApplicationException(Response.Status.BAD_REQUEST);
-    }
-    try {
-      ResponseBuilder response = Response.ok(getTableList());
-      response.cacheControl(cacheControl);
-      return response.build();
-    } catch (IOException e) {
-      throw new WebApplicationException(e, 
-                  Response.Status.SERVICE_UNAVAILABLE);
-    }
-  }
-
-  @Path("status/cluster")
-  public StorageClusterStatusResource getClusterStatusResource() 
-      throws IOException {
-    if (servlet.isMultiUser()) {
-      throw new WebApplicationException(Response.Status.BAD_REQUEST);
-    }
-    return new StorageClusterStatusResource();
-  }
-
-  @Path("version")
-  public VersionResource getVersionResource() throws IOException {
-    return new VersionResource();
-  }
-
-  @Path("{token: [0-9a-fA-F]{32} }") // 128 bit md5 sums
-  public Response getTableRootResource(
-      @PathParam("token") String token) throws IOException {
-    if (servlet.isMultiUser()) {
-      User user = servlet.getAuthenticator().getUserForToken(token);
-      if (user == null || user.isDisabled()) {
-        throw new WebApplicationException(Response.Status.FORBIDDEN);
-      }
-      try {
-        ResponseBuilder response = Response.ok(getTableListForUser(user));
-        response.cacheControl(cacheControl);
-        return response.build();
-      } catch (IOException e) {
-        throw new WebApplicationException(e, 
-                    Response.Status.SERVICE_UNAVAILABLE);
-      }
-    }
-    throw new WebApplicationException(Response.Status.BAD_REQUEST);
-  }
-
-  @Path("{token: [0-9a-fA-F]{32} }/status/cluster") // 128 bit md5 sums
-  public StorageClusterStatusResource getClusterStatusResourceAuthorized(
-      @PathParam("token") String token) throws IOException {
-    if (servlet.isMultiUser()) {
-      User user = servlet.getAuthenticator().getUserForToken(token);
-      if (user != null && user.isAdmin() && !user.isDisabled()) {
-        return new StorageClusterStatusResource();
-      }
-      throw new WebApplicationException(Response.Status.FORBIDDEN);
-    }
-    throw new WebApplicationException(Response.Status.BAD_REQUEST);
-  }
-
-  @Path("{token: [0-9a-fA-F]{32} }/{table}")
-  public TableResource getTableResource(@PathParam("token") String token, 
-      @PathParam("table") String table) throws IOException {
-    if (servlet.isMultiUser()) {
-      User user = servlet.getAuthenticator().getUserForToken(token);
-      if (user == null || user.isDisabled()) {
-        throw new WebApplicationException(Response.Status.FORBIDDEN);
-      }
-      return new TableResource(user, table);
-    }
-    throw new WebApplicationException(Response.Status.BAD_REQUEST);
-  }
-
-  @Path("{table}")
-  public TableResource getTableResource(@PathParam("table") String table)
-    throws IOException {
-    if (servlet.isMultiUser()) {
-      throw new WebApplicationException(Response.Status.BAD_REQUEST);
-    }
-    return new TableResource(null, table);
-  }
-
-}
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.stargate;
+
+import java.io.IOException;
+
+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;
+import javax.ws.rs.core.UriInfo;
+import javax.ws.rs.core.Response.ResponseBuilder;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.stargate.User;
+import org.apache.hadoop.hbase.stargate.model.TableListModel;
+import org.apache.hadoop.hbase.stargate.model.TableModel;
+
+@Path("/")
+public class RootResource implements Constants {
+  private static final Log LOG = LogFactory.getLog(RootResource.class);
+
+  RESTServlet servlet;
+  CacheControl cacheControl;
+
+  public RootResource() throws IOException {
+    servlet = RESTServlet.getInstance();
+    cacheControl = new CacheControl();
+    cacheControl.setNoCache(true);
+    cacheControl.setNoTransform(false);
+  }
+
+  private final User auth(final String token) throws IOException {
+    User user = servlet.getAuthenticator().getUserForToken(token);
+    if (user == null || user.isDisabled()) {
+      throw new WebApplicationException(Response.Status.FORBIDDEN);
+    }
+    return user;
+  }
+
+  private final TableListModel getTableList() throws IOException {
+    TableListModel tableList = new TableListModel();
+    HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
+    HTableDescriptor[] list = admin.listTables();
+    for (HTableDescriptor htd: list) {
+      tableList.add(new TableModel(htd.getNameAsString()));
+    }
+    return tableList;
+  }
+
+  private final TableListModel getTableListForUser(final User user) 
+      throws IOException {
+    TableListModel tableList;
+    if (user.isAdmin()) {
+      tableList = getTableList();
+    } else {
+      tableList = new TableListModel();
+      HBaseAdmin admin = new HBaseAdmin(servlet.getConfiguration());
+      HTableDescriptor[] list = admin.listTables();
+      String prefix = user.getName() + ".";
+      for (HTableDescriptor htd: list) {
+        String name = htd.getNameAsString();
+        if (!name.startsWith(prefix)) {
+          continue;
+        }
+        tableList.add(new TableModel(name.substring(prefix.length())));
+      }
+    }
+    return tableList;
+  }
+
+  @GET
+  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
+  public Response get(final @Context UriInfo uriInfo) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("GET " + uriInfo.getAbsolutePath());
+    }
+    servlet.getMetrics().incrementRequests(1);
+    if (servlet.isMultiUser()) {
+      throw new WebApplicationException(Response.Status.BAD_REQUEST);
+    }
+    try {
+      ResponseBuilder response = Response.ok(getTableList());
+      response.cacheControl(cacheControl);
+      return response.build();
+    } catch (IOException e) {
+      throw new WebApplicationException(e, 
+                  Response.Status.SERVICE_UNAVAILABLE);
+    }
+  }
+
+  @Path("status/cluster")
+  public StorageClusterStatusResource getClusterStatusResource() 
+      throws IOException {
+    if (servlet.isMultiUser()) {
+      throw new WebApplicationException(Response.Status.BAD_REQUEST);
+    }
+    return new StorageClusterStatusResource(User.DEFAULT_USER);
+  }
+
+  @Path("version")
+  public VersionResource getVersionResource() throws IOException {
+    return new VersionResource();
+  }
+
+  @Path("{token: [0-9a-fA-F]{32} }") // 128 bit md5 sums
+  public Response getTableRootResource(
+      final @PathParam("token") String token) throws IOException {
+    if (servlet.isMultiUser()) {
+      User user = auth(token);
+      if (!servlet.userRequestLimit(user, 1)) {
+        return Response.status(509).build();
+      }
+      try {
+        ResponseBuilder response = Response.ok(getTableListForUser(user));
+        response.cacheControl(cacheControl);
+        return response.build();
+      } catch (IOException e) {
+        throw new WebApplicationException(e, 
+                    Response.Status.SERVICE_UNAVAILABLE);
+      }
+    }
+    throw new WebApplicationException(Response.Status.BAD_REQUEST);
+  }
+
+  @Path("{token: [0-9a-fA-F]{32} }/status/cluster") // 128 bit md5 sums
+  public StorageClusterStatusResource getClusterStatusResourceAuthorized(
+      final @PathParam("token") String token) throws IOException {
+    if (servlet.isMultiUser()) {
+      User user = auth(token);
+      if (user != null && user.isAdmin()) {
+        return new StorageClusterStatusResource(user);
+      }
+      throw new WebApplicationException(Response.Status.FORBIDDEN);
+    }
+    throw new WebApplicationException(Response.Status.BAD_REQUEST);
+  }
+
+  @Path("{token: [0-9a-fA-F]{32} }/{table}")
+  public TableResource getTableResource(
+      final @PathParam("token") String token, 
+      final @PathParam("table") String table) throws IOException {
+    if (servlet.isMultiUser()) {
+      User user = auth(token);
+      return new TableResource(user, table);
+    }
+    throw new WebApplicationException(Response.Status.BAD_REQUEST);
+  }
+
+  @Path("{table}")
+  public TableResource getTableResource(
+      final @PathParam("table") String table) throws IOException {
+    if (servlet.isMultiUser()) {
+      throw new WebApplicationException(Response.Status.BAD_REQUEST);
+    }
+    return new TableResource(User.DEFAULT_USER, table);
+  }
+
+}

Modified: hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResource.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResource.java?rev=932214&r1=932213&r2=932214&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResource.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResource.java Fri Apr  9 02:10:02 2010
@@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.client.De
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.HTablePool;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.stargate.auth.User;
+import org.apache.hadoop.hbase.stargate.User;
 import org.apache.hadoop.hbase.stargate.model.CellModel;
 import org.apache.hadoop.hbase.stargate.model.CellSetModel;
 import org.apache.hadoop.hbase.stargate.model.RowModel;
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.util.Byte
 public class RowResource implements Constants {
   private static final Log LOG = LogFactory.getLog(RowResource.class);
 
+  User user;
   String tableName;
   String actualTableName;
   RowSpec rowspec;
@@ -63,6 +64,7 @@ public class RowResource implements Cons
 
   public RowResource(User user, String table, String rowspec, String versions)
       throws IOException {
+    this.user = user;
     if (user != null) {
       this.actualTableName =
         !user.isAdmin() ? user.getName() + "." + table : table;
@@ -77,16 +79,19 @@ public class RowResource implements Cons
     }
     this.servlet = RESTServlet.getInstance();
     cacheControl = new CacheControl();
-    cacheControl.setMaxAge(servlet.getMaxAge(table));
+    cacheControl.setMaxAge(servlet.getMaxAge(actualTableName));
     cacheControl.setNoTransform(false);
   }
 
   @GET
   @Produces({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
-  public Response get(@Context UriInfo uriInfo) {
+  public Response get(final @Context UriInfo uriInfo) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("GET " + uriInfo.getAbsolutePath());
     }
+    if (!servlet.userRequestLimit(user, 1)) {
+      return Response.status(509).build();
+    }
     servlet.getMetrics().incrementRequests(1);
     try {
       ResultGenerator generator =
@@ -94,6 +99,7 @@ public class RowResource implements Cons
       if (!generator.hasNext()) {
         throw new WebApplicationException(Response.Status.NOT_FOUND);
       }
+      int count = 0;
       CellSetModel model = new CellSetModel();
       KeyValue value = generator.next();
       byte[] rowKey = value.getRow();
@@ -107,6 +113,9 @@ public class RowResource implements Cons
         rowModel.addCell(
           new CellModel(value.getColumn(), value.getTimestamp(),
               value.getValue()));
+        if (++count > rowspec.getMaxValues()) {
+          break;
+        }
         value = generator.next();
       } while (value != null);
       model.addRow(rowModel);
@@ -121,10 +130,14 @@ public class RowResource implements Cons
 
   @GET
   @Produces(MIMETYPE_BINARY)
-  public Response getBinary(@Context UriInfo uriInfo) {
+  public Response getBinary(final @Context UriInfo uriInfo) 
+      throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("GET " + uriInfo.getAbsolutePath() + " as "+ MIMETYPE_BINARY);
     }
+    if (!servlet.userRequestLimit(user, 1)) {
+      return Response.status(509).build();
+    }
     servlet.getMetrics().incrementRequests(1);
     // doesn't make sense to use a non specific coordinate as this can only
     // return a single cell
@@ -148,13 +161,19 @@ public class RowResource implements Cons
     }
   }
 
-  Response update(CellSetModel model, boolean replace) {
-    servlet.getMetrics().incrementRequests(1);
+  Response update(final CellSetModel model, final boolean replace) {
     HTablePool pool = servlet.getTablePool();
     HTable table = null;
     try {
+      List<RowModel> rows = model.getRows();
+      // the user request limit is a transaction limit, so we need to
+      // account for updates by row
+      if (user != null && !servlet.userRequestLimit(user, rows.size())) {
+        throw new WebApplicationException(Response.status(509).build());
+      }
       table = pool.getTable(actualTableName);
-      for (RowModel row: model.getRows()) {
+      table.setAutoFlush(false);
+      for (RowModel row: rows) {
         byte[] key = row.getKey();
         Put put = new Put(key);
         for (CellModel cell: row.getCells()) {
@@ -166,6 +185,7 @@ public class RowResource implements Cons
           LOG.debug("PUT " + put.toString());
         }
       }
+      table.setAutoFlush(true);
       table.flushCommits();
       ResponseBuilder response = Response.ok();
       return response.build();
@@ -179,9 +199,9 @@ public class RowResource implements Cons
     }
   }
 
-  Response updateBinary(byte[] message, HttpHeaders headers, 
-      boolean replace) {
-    servlet.getMetrics().incrementRequests(1);
+  // This currently supports only update of one row at a time.
+  Response updateBinary(final byte[] message, final HttpHeaders headers,
+      final boolean replace) {
     HTablePool pool = servlet.getTablePool();
     HTable table = null;    
     try {
@@ -215,7 +235,6 @@ public class RowResource implements Cons
       if (LOG.isDebugEnabled()) {
         LOG.debug("PUT " + put.toString());
       }
-      table.flushCommits();
       return Response.ok().build();
     } catch (IOException e) {
       throw new WebApplicationException(e,
@@ -229,49 +248,71 @@ public class RowResource implements Cons
 
   @PUT
   @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
-  public Response put(CellSetModel model, @Context UriInfo uriInfo) {
+  public Response put(final CellSetModel model,
+      final @Context UriInfo uriInfo) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("PUT " + uriInfo.getAbsolutePath());
     }
+    if (!servlet.userRequestLimit(user, 1)) {
+      return Response.status(509).build();
+    }
+    servlet.getMetrics().incrementRequests(1);
     return update(model, true);
   }
 
   @PUT
   @Consumes(MIMETYPE_BINARY)
-  public Response putBinary(byte[] message, @Context UriInfo uriInfo, 
-      @Context HttpHeaders headers)
+  public Response putBinary(final byte[] message,
+      final @Context UriInfo uriInfo, final @Context HttpHeaders headers)
+      throws IOException
   {
     if (LOG.isDebugEnabled()) {
       LOG.debug("PUT " + uriInfo.getAbsolutePath() + " as "+ MIMETYPE_BINARY);
     }
+    if (!servlet.userRequestLimit(user, 1)) {
+      return Response.status(509).build();
+    }
+    servlet.getMetrics().incrementRequests(1);
     return updateBinary(message, headers, true);
   }
 
   @POST
   @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF})
-  public Response post(CellSetModel model, @Context UriInfo uriInfo) {
+  public Response post(final CellSetModel model,
+      final @Context UriInfo uriInfo) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("POST " + uriInfo.getAbsolutePath());
     }
+    if (!servlet.userRequestLimit(user, 1)) {
+      return Response.status(509).build();
+    }
+    servlet.getMetrics().incrementRequests(1);
     return update(model, false);
   }
 
   @POST
   @Consumes(MIMETYPE_BINARY)
-  public Response postBinary(byte[] message, @Context UriInfo uriInfo, 
-      @Context HttpHeaders headers)
-  {
+  public Response postBinary(final byte[] message,
+      final @Context UriInfo uriInfo, final @Context HttpHeaders headers) 
+      throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("POST " + uriInfo.getAbsolutePath() + " as "+MIMETYPE_BINARY);
     }
+    if (!servlet.userRequestLimit(user, 1)) {
+      return Response.status(509).build();
+    }
+    servlet.getMetrics().incrementRequests(1);
     return updateBinary(message, headers, false);
   }
 
   @DELETE
-  public Response delete(@Context UriInfo uriInfo) {
+  public Response delete(final @Context UriInfo uriInfo) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("DELETE " + uriInfo.getAbsolutePath());
     }
+    if (!servlet.userRequestLimit(user, 1)) {
+      return Response.status(509).build();
+    }
     servlet.getMetrics().incrementRequests(1);
     Delete delete = null;
     if (rowspec.hasTimestamp())
@@ -303,7 +344,6 @@ public class RowResource implements Cons
       if (LOG.isDebugEnabled()) {
         LOG.debug("DELETE " + delete.toString());
       }
-      table.flushCommits();
     } catch (IOException e) {
       throw new WebApplicationException(e, 
                   Response.Status.SERVICE_UNAVAILABLE);
@@ -314,4 +354,5 @@ public class RowResource implements Cons
     }
     return Response.ok().build();
   }
+
 }

Modified: hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResultGenerator.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResultGenerator.java?rev=932214&r1=932213&r2=932214&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResultGenerator.java (original)
+++ hadoop/hbase/branches/0.20_pre_durability/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResultGenerator.java Fri Apr  9 02:10:02 2010
@@ -1,91 +1,105 @@
-/*
- * Copyright 2010 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.stargate;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HTablePool;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.filter.Filter;
-
-public class RowResultGenerator extends ResultGenerator {
-  private Iterator<KeyValue> valuesI;
-
-  public RowResultGenerator(final String tableName, final RowSpec rowspec,
-      final Filter filter) throws IllegalArgumentException, IOException {
-    HTablePool pool = RESTServlet.getInstance().getTablePool(); 
-    HTable table = pool.getTable(tableName);
-    try {
-      Get get = new Get(rowspec.getRow());
-      if (rowspec.hasColumns()) {
-        get.addColumns(rowspec.getColumns());
-      } else {
-        // rowspec does not explicitly specify columns, return them all
-        for (HColumnDescriptor family: 
-            table.getTableDescriptor().getFamilies()) {
-          get.addFamily(family.getName());
-        }
-      }
-      get.setTimeRange(rowspec.getStartTime(), rowspec.getEndTime());
-      get.setMaxVersions(rowspec.getMaxVersions());
-      if (filter != null) {
-        get.setFilter(filter);
-      }
-      Result result = table.get(get);
-      if (result != null && !result.isEmpty()) {
-        valuesI = result.list().iterator();
-      }
-    } finally {
-      pool.putTable(table);
-    }
-  }
-
-  public void close() {
-  }
-
-  public boolean hasNext() {
-    if (valuesI == null) {
-      return false;
-    }
-    return valuesI.hasNext();
-  }
-
-  public KeyValue next() {
-    if (valuesI == null) {
-      return null;
-    }
-    try {
-      return valuesI.next();
-    } catch (NoSuchElementException e) {
-      return null;
-    }
-  }
-
-  public void remove() {
-    throw new UnsupportedOperationException("remove not supported");
-  }
-}
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.stargate;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTablePool;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.filter.Filter;
+
+public class RowResultGenerator extends ResultGenerator {
+  private Iterator<KeyValue> valuesI;
+  private KeyValue cache;
+
+  public RowResultGenerator(final String tableName, final RowSpec rowspec,
+      final Filter filter) throws IllegalArgumentException, IOException {
+    HTablePool pool = RESTServlet.getInstance().getTablePool(); 
+    HTable table = pool.getTable(tableName);
+    try {
+      Get get = new Get(rowspec.getRow());
+      if (rowspec.hasColumns()) {
+        get.addColumns(rowspec.getColumns());
+      } else {
+        // rowspec does not explicitly specify columns, return them all
+        for (HColumnDescriptor family: 
+            table.getTableDescriptor().getFamilies()) {
+          get.addFamily(family.getName());
+        }
+      }
+      get.setTimeRange(rowspec.getStartTime(), rowspec.getEndTime());
+      get.setMaxVersions(rowspec.getMaxVersions());
+      if (filter != null) {
+        get.setFilter(filter);
+      }
+      Result result = table.get(get);
+      if (result != null && !result.isEmpty()) {
+        valuesI = result.list().iterator();
+      }
+    } finally {
+      pool.putTable(table);
+    }
+  }
+
+  public void close() {
+  }
+
+  public boolean hasNext() {
+    if (cache != null) {
+      return true;
+    }
+    if (valuesI == null) {
+      return false;
+    }
+    return valuesI.hasNext();
+  }
+
+  public KeyValue next() {
+    if (cache != null) {
+      KeyValue kv = cache;
+      cache = null;
+      return kv;
+    }
+    if (valuesI == null) {
+      return null;
+    }
+    try {
+      return valuesI.next();
+    } catch (NoSuchElementException e) {
+      return null;
+    }
+  }
+
+  public void putBack(KeyValue kv) {
+    this.cache = kv;
+  }
+
+  public void remove() {
+    throw new UnsupportedOperationException("remove not supported");
+  }
+
+}