You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2009/01/22 01:21:18 UTC

svn commit: r736503 [1/4] - in /hadoop/hbase/trunk: ./ lib/ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/io/ src/java/org/apache/hadoop/hbase/regionserver/ src/java/org/apache/hadoop/hbase/rest/ src/java/org/apache/hadoop/hbase/re...

Author: stack
Date: Wed Jan 21 16:21:16 2009
New Revision: 736503

URL: http://svn.apache.org/viewvc?rev=736503&view=rev
Log:
HBASE-1064 HBase REST xml/json improvements

Added:
    hadoop/hbase/trunk/lib/AgileJSON-2.0.jar   (with props)
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/AbstractController.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/AbstractModel.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/DatabaseController.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/DatabaseModel.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RESTConstants.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RowController.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RowModel.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/ScannerController.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/ScannerModel.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Status.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableController.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableModel.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TimestampController.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TimestampModel.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/RestCell.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/RowUpdateDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/ScannerDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/ScannerIdentifier.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/TimestampsDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/exception/
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/exception/HBaseRestException.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/ColumnValueFilterFactory.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/FilterFactory.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/FilterFactoryConstants.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/InclusiveStopRowFilterFactory.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/PageRowFilterFactory.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/RegExpRowFilterFactory.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/RowFilterSetFactory.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/StopRowFilterFactory.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/WhileMatchRowFilterFactory.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/HBaseRestParserFactory.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/IHBaseRestParser.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/JsonRestParser.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/XMLRestParser.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/serializer/
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/serializer/AbstractRestSerializer.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/serializer/IRestSerializer.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/serializer/ISerializable.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/serializer/JSONSerializer.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/serializer/RestSerializerFactory.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/serializer/SimpleXMLSerializer.java
Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/NOTICE.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Cell.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/RowResult.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RowHandler.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/ScannerHandler.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableHandler.java
    hadoop/hbase/trunk/src/webapps/rest/WEB-INF/web.xml

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=736503&r1=736502&r2=736503&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Wed Jan 21 16:21:16 2009
@@ -23,8 +23,10 @@
    HBASE-1031  Add the Zookeeper jar
    HBASE-1142  Cleanup thrift server; remove Text and profuse DEBUG messaging
                (Tim Sell via Stack)
+   HBASE-1064  HBase REST xml/json improvements (Brian Beggs working of
+               initial Michael Gottesman work via Stack)
 
-Release 0.19.0 - Unreleased
+Release 0.19.0 - 01/21/2009
   INCOMPATIBLE CHANGES
    HBASE-885   TableMap and TableReduce should be interfaces
                (Doğacan Güney via Stack)

Modified: hadoop/hbase/trunk/NOTICE.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/NOTICE.txt?rev=736503&r1=736502&r2=736503&view=diff
==============================================================================
--- hadoop/hbase/trunk/NOTICE.txt (original)
+++ hadoop/hbase/trunk/NOTICE.txt Wed Jan 21 16:21:16 2009
@@ -3,8 +3,42 @@
 
 In addition, this product includes software developed by:
 
+
 European Commission project OneLab (http://www.one-lab.org)
 
+
 Facebook, Inc. (http://developers.facebook.com/thrift/ -- Page includes the Thrift Software License)
 
+
 JUnit (http://www.junit.org/)
+
+
+Michael Gottesman developed AgileJSON.  Its source code is here:
+
+ http://github.com/gottesmm/agile-json-2.0/tree/master
+
+It has this license at the head of the each source file:
+
+ * Permission is hereby granted, free of charge, to any person obtaining a
+ * copy
+ * of this software and associated documentation files (the "Software"), to
+ * deal
+ * in the Software without restriction, including without limitation the
+ * rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in
+ * all
+ * copies or substantial portions of the Software.
+ *
+ * The Software shall be used for Good, not Evil.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+ * FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+ * THE SOFTWARE.

Added: hadoop/hbase/trunk/lib/AgileJSON-2.0.jar
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/lib/AgileJSON-2.0.jar?rev=736503&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hadoop/hbase/trunk/lib/AgileJSON-2.0.jar
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java?rev=736503&r1=736502&r2=736503&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java Wed Jan 21 16:21:16 2009
@@ -27,10 +27,15 @@
 import java.util.Map;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.rest.serializer.IRestSerializer;
+import org.apache.hadoop.hbase.rest.serializer.ISerializable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
 
+import agilejson.TOJSON;
+
 /**
  * An HColumnDescriptor contains information about a column family such as the
  * number of versions, compression settings, etc.
@@ -40,7 +45,7 @@
  * column and recreating it. If there is data stored in the column, it will be
  * deleted when the column is deleted.
  */
-public class HColumnDescriptor implements WritableComparable<HColumnDescriptor> {
+public class HColumnDescriptor implements ISerializable, WritableComparable<HColumnDescriptor> {
   // For future backward compatibility
 
   // Version 3 was when column names become byte arrays and when we picked up
@@ -257,6 +262,7 @@
   /**
    * @return Name of this column family with colon as required by client API
    */
+  @TOJSON(fieldName = "name", base64=true)
   public byte [] getNameWithColon() {
     return HStoreKey.addDelimiter(this.name);
   }
@@ -315,6 +321,7 @@
   }
 
   /** @return compression type being used for the column family */
+  @TOJSON
   public CompressionType getCompression() {
     String value = getValue(COMPRESSION);
     if (value != null) {
@@ -327,6 +334,7 @@
   }
   
   /** @return maximum number of versions */
+  @TOJSON
   public int getMaxVersions() {
     String value = getValue(HConstants.VERSIONS);
     if (value != null)
@@ -344,6 +352,7 @@
   /**
    * @return Compression type setting.
    */
+  @TOJSON
   public CompressionType getCompressionType() {
     return getCompression();
   }
@@ -364,6 +373,7 @@
   /**
    * @return True if we are to keep all in use HRegionServer cache.
    */
+  @TOJSON(prefixLength = 2)
   public boolean isInMemory() {
     String value = getValue(HConstants.IN_MEMORY);
     if (value != null)
@@ -382,6 +392,7 @@
   /**
    * @return Maximum value length.
    */
+  @TOJSON
   public synchronized int getMaxValueLength() {
     if (this.maxValueLength == null) {
       String value = getValue(LENGTH);
@@ -402,6 +413,7 @@
   /**
    * @return Time-to-live of cell contents, in seconds.
    */
+  @TOJSON
   public int getTimeToLive() {
     String value = getValue(TTL);
     return (value != null)? Integer.valueOf(value).intValue(): DEFAULT_TTL;
@@ -417,6 +429,7 @@
   /**
    * @return True if MapFile blocks should be cached.
    */
+  @TOJSON(prefixLength = 2)
   public boolean isBlockCacheEnabled() {
     String value = getValue(BLOCKCACHE);
     if (value != null)
@@ -434,6 +447,7 @@
   /**
    * @return true if a bloom filter is enabled
    */
+  @TOJSON(prefixLength = 2)
   public boolean isBloomfilter() {
     String value = getValue(BLOOMFILTER);
     if (value != null)
@@ -577,4 +591,11 @@
     }
     return result;
   }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.hbase.rest.xml.IOutputXML#toXML()
+   */
+  public void restSerialize(IRestSerializer serializer) throws HBaseRestException {
+    serializer.serializeColumnDescriptor(this);    
+  }
 }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java?rev=736503&r1=736502&r2=736503&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java Wed Jan 21 16:21:16 2009
@@ -31,14 +31,19 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.rest.serializer.IRestSerializer;
+import org.apache.hadoop.hbase.rest.serializer.ISerializable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableComparable;
 
+import agilejson.TOJSON;
+
 /**
  * HTableDescriptor contains the name of an HTable, and its
  * column families.
  */
-public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
+public class HTableDescriptor implements WritableComparable<HTableDescriptor>, ISerializable {
 
   // Changes prior to version 3 were not recorded here.
   // Version 3 adds metadata as a map where keys and values are byte[].
@@ -383,6 +388,7 @@
   }
 
   /** @return name of table */
+  @TOJSON
   public byte [] getName() {
     return name;
   }
@@ -621,6 +627,11 @@
   public Collection<HColumnDescriptor> getFamilies() {
     return Collections.unmodifiableCollection(this.families.values());
   }
+  
+  @TOJSON(fieldName = "columns")
+  public HColumnDescriptor[] getColumnFamilies() {
+    return getFamilies().toArray(new HColumnDescriptor[0]);
+  }
 
   /**
    * @param column
@@ -667,4 +678,11 @@
           new HColumnDescriptor(HConstants.COLUMN_FAMILY_HISTORIAN,
             HConstants.ALL_VERSIONS, HColumnDescriptor.CompressionType.NONE,
             false, false, Integer.MAX_VALUE, HConstants.WEEK_IN_SECONDS, false)});
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.hbase.rest.xml.IOutputXML#toXML()
+   */
+  public void restSerialize(IRestSerializer serializer) throws HBaseRestException {
+    serializer.serializeTableDescriptor(this);
+  }
 }
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java?rev=736503&r1=736502&r2=736503&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java Wed Jan 21 16:21:16 2009
@@ -328,4 +328,4 @@
       LOG.debug("Offlined");
     }
   }
-}
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Cell.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Cell.java?rev=736503&r1=736502&r2=736503&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Cell.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Cell.java Wed Jan 21 16:21:16 2009
@@ -29,28 +29,36 @@
 import java.util.TreeMap;
 import java.util.Map.Entry;
 
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.rest.serializer.IRestSerializer;
+import org.apache.hadoop.hbase.rest.serializer.ISerializable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.Writable;
 
+import agilejson.TOJSON;
+
 /**
- * Cell - Used to transport a cell value (byte[]) and the timestamp it was 
+ * Cell - Used to transport a cell value (byte[]) and the timestamp it was
  * stored with together as a result for get and getRow methods. This promotes
- * the timestamp of a cell to a first-class value, making it easy to take 
- * note of temporal data. Cell is used all the way from HStore up to HTable.
+ * the timestamp of a cell to a first-class value, making it easy to take note
+ * of temporal data. Cell is used all the way from HStore up to HTable.
  */
-public class Cell implements Writable, Iterable<Map.Entry<Long, byte[]>> {
-  protected final SortedMap<Long, byte[]> valueMap =
-    new TreeMap<Long, byte[]>(new Comparator<Long>() {
-      public int compare(Long l1, Long l2) {
-        return l2.compareTo(l1);
-    }});
-  
+public class Cell implements Writable, Iterable<Map.Entry<Long, byte[]>>,
+    ISerializable {
+  protected final SortedMap<Long, byte[]> valueMap = new TreeMap<Long, byte[]>(
+      new Comparator<Long>() {
+        public int compare(Long l1, Long l2) {
+          return l2.compareTo(l1);
+        }
+      });
+
   /** For Writable compatibility */
   public Cell() {
   }
 
   /**
    * Create a new Cell with a given value and timestamp. Used by HStore.
+   * 
    * @param value
    * @param timestamp
    */
@@ -60,24 +68,29 @@
 
   /**
    * Create a new Cell with a given value and timestamp. Used by HStore.
+   * 
    * @param value
    * @param timestamp
    */
   public Cell(byte[] value, long timestamp) {
     valueMap.put(timestamp, value);
   }
-  
+
   /**
-   * @param vals array of values
-   * @param ts array of timestamps
+   * @param vals
+   *          array of values
+   * @param ts
+   *          array of timestamps
    */
   public Cell(String[] vals, long[] ts) {
     this(Bytes.toByteArrays(vals), ts);
   }
-  
+
   /**
-   * @param vals array of values
-   * @param ts array of timestamps
+   * @param vals
+   *          array of values
+   * @param ts
+   *          array of timestamps
    */
   public Cell(byte[][] vals, long[] ts) {
     if (vals.length != ts.length) {
@@ -88,42 +101,51 @@
       valueMap.put(ts[i], vals[i]);
     }
   }
-  
+
   /** @return the current cell's value */
+  @TOJSON(base64=true)
   public byte[] getValue() {
     return valueMap.get(valueMap.firstKey());
   }
-  
+
   /** @return the current cell's timestamp */
+  @TOJSON
   public long getTimestamp() {
     return valueMap.firstKey();
   }
-  
+
   /** @return the number of values this cell holds */
   public int getNumValues() {
     return valueMap.size();
   }
-  
-  /** Add values and timestamps of another cell into this cell 
-   * @param c Cell
+
+  /**
+   * Add values and timestamps of another cell into this cell
+   * 
+   * @param c
+   *          Cell
    */
   public void mergeCell(Cell c) {
     valueMap.putAll(c.valueMap);
   }
-  
-  /** Add a new timestamp and value to this cell
-   * @param val value
-   * @param ts timestamp
+
+  /**
+   * Add a new timestamp and value to this cell
+   * 
+   * @param val
+   *          value
+   * @param ts
+   *          timestamp
    */
   public void add(byte[] val, long ts) {
     valueMap.put(ts, val);
   }
-  
+
   @Override
   public String toString() {
     if (valueMap.size() == 1) {
-      return "timestamp=" + getTimestamp() + ", value=" +
-        Bytes.toString(getValue());
+      return "timestamp=" + getTimestamp() + ", value="
+          + Bytes.toString(getValue());
     }
     StringBuilder s = new StringBuilder("{ ");
     int i = 0;
@@ -141,7 +163,7 @@
     s.append(" }");
     return s.toString();
   }
-  
+
   //
   // Writable
   //
@@ -162,7 +184,7 @@
       Bytes.writeByteArray(out, entry.getValue());
     }
   }
-  
+
   //
   // Iterable
   //
@@ -170,23 +192,36 @@
   public Iterator<Entry<Long, byte[]>> iterator() {
     return new CellIterator();
   }
-  
+
   private class CellIterator implements Iterator<Entry<Long, byte[]>> {
     private Iterator<Entry<Long, byte[]>> it;
+
     CellIterator() {
       it = valueMap.entrySet().iterator();
     }
-    
+
     public boolean hasNext() {
       return it.hasNext();
     }
-    
+
     public Entry<Long, byte[]> next() {
       return it.next();
     }
-    
+
     public void remove() throws UnsupportedOperationException {
       throw new UnsupportedOperationException("remove is not supported");
     }
   }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.hbase.rest.serializer.ISerializable#restSerialize(org
+   * .apache.hadoop.hbase.rest.serializer.IRestSerializer)
+   */
+  public void restSerialize(IRestSerializer serializer)
+      throws HBaseRestException {
+    serializer.serializeCell(this);
+  }
 }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/RowResult.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/RowResult.java?rev=736503&r1=736502&r2=736503&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/RowResult.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/RowResult.java Wed Jan 21 16:21:16 2009
@@ -32,14 +32,20 @@
 import java.util.TreeSet;
 
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.rest.descriptors.RestCell;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.rest.serializer.IRestSerializer;
+import org.apache.hadoop.hbase.rest.serializer.ISerializable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Writable;
 
+import agilejson.TOJSON;
+
 /**
  * Holds row name and then a map of columns to cells.
  */
-public class RowResult implements Writable, SortedMap<byte [], Cell> {
+public class RowResult implements Writable, SortedMap<byte [], Cell>, ISerializable {
   private byte [] row = null;
   private final HbaseMapWritable<byte [], Cell> cells;
 
@@ -63,6 +69,7 @@
    * Get the row for this RowResult
    * @return the row
    */
+  @TOJSON(base64=true)
   public byte [] getRow() {
     return row;
   }
@@ -124,6 +131,22 @@
   public Set<Map.Entry<byte [], Cell>> entrySet() {
     return Collections.unmodifiableSet(this.cells.entrySet());
   }
+  
+  /**
+   * This method used solely for the REST serialization
+   * 
+   * @return
+   */
+  @TOJSON
+  public RestCell[] getCells() {
+    RestCell[] restCells = new RestCell[this.cells.size()];
+    int i = 0;
+    for (Map.Entry<byte[], Cell> entry : this.cells.entrySet()) {
+      restCells[i] = new RestCell(entry.getKey(), entry.getValue());
+      i++;
+    }
+    return restCells;
+  }
 
   public Collection<Cell> values() {
     ArrayList<Cell> result = new ArrayList<Cell>();
@@ -235,10 +258,17 @@
     return sb.toString();
   }
   
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.hbase.rest.xml.IOutputXML#toXML()
+   */
+  public void restSerialize(IRestSerializer serializer) throws HBaseRestException {
+    serializer.serializeRowResult(this);
+  }  
+  
   //
   // Writable
   //
-  
+
   public void readFields(final DataInput in) throws IOException {
     this.row = Bytes.readByteArray(in);
     this.cells.readFields(in);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java?rev=736503&r1=736502&r2=736503&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java Wed Jan 21 16:21:16 2009
@@ -248,4 +248,4 @@
       this.interrupt();
     }
   }
-}
+}
\ No newline at end of file

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/AbstractController.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/AbstractController.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/AbstractController.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/AbstractController.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,72 @@
+/**
+ * Copyright 2008 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.rest;
+
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.rest.parser.IHBaseRestParser;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public abstract class AbstractController implements RESTConstants {
+
+  private Log LOG = LogFactory.getLog(AbstractController.class);
+  protected Configuration conf;
+  protected AbstractModel model;
+
+  public void initialize(HBaseConfiguration conf, HBaseAdmin admin) {
+    this.conf = conf;
+    this.model = generateModel(conf, admin);
+  }
+
+  public abstract void get(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap) throws HBaseRestException;
+
+  public abstract void post(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap, byte[] input, IHBaseRestParser parser)
+      throws HBaseRestException;
+
+  public abstract void put(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap, byte[] input, IHBaseRestParser parser)
+      throws HBaseRestException;
+
+  public abstract void delete(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap) throws HBaseRestException;
+
+  protected abstract AbstractModel generateModel(HBaseConfiguration conf,
+      HBaseAdmin a);
+  
+  protected byte[][] getColumnsFromQueryMap(Map<String, String[]> queryMap) {
+    byte[][] columns = null;
+    String[] columnArray = queryMap.get(RESTConstants.COLUMN);
+    if (columnArray != null) {
+      columns = new byte[columnArray.length][];
+      for (int i = 0; i < columnArray.length; i++) {
+        columns[i] = Bytes.toBytes(columnArray[i]);
+      }
+    }
+    return columns;
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/AbstractModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/AbstractModel.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/AbstractModel.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/AbstractModel.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,99 @@
+/**
+ * Copyright 2008 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.rest;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public abstract class AbstractModel {
+
+  @SuppressWarnings("unused")
+  private Log LOG = LogFactory.getLog(AbstractModel.class);
+  protected HBaseConfiguration conf;
+  protected HBaseAdmin admin;
+
+  protected static class Encodings {
+
+    protected interface Encoding {
+
+      String encode(byte[] b) throws HBaseRestException;
+    }
+
+    public static Encoding EBase64 = new Encoding() {
+
+      public String encode(byte[] b) throws HBaseRestException {
+        return new String(Base64.encodeBytes(b));
+      }
+    };
+    public static Encoding EUTF8 = new Encoding() {
+
+      public String encode(byte[] b) throws HBaseRestException {
+        return new String(b);
+      }
+    };
+  }
+
+  protected static Encodings.Encoding encoding = Encodings.EUTF8;
+
+  public void initialize(HBaseConfiguration conf, HBaseAdmin admin) {
+    this.conf = conf;
+    this.admin = admin;
+  }
+
+  protected byte[][] getColumns(byte[] tableName) throws HBaseRestException {
+    try {
+      HTable h = new HTable(tableName);
+      Collection<HColumnDescriptor> columns = h.getTableDescriptor()
+          .getFamilies();
+      byte[][] resultant = new byte[columns.size()][];
+      int count = 0;
+
+      for (HColumnDescriptor c : columns) {
+        resultant[count++] = c.getNameWithColon();
+      }
+
+      return resultant;
+    } catch (IOException e) {
+      throw new HBaseRestException(e);
+    }
+  }
+
+  protected static byte COLON = Bytes.toBytes(":")[0];
+
+  protected boolean isColumnFamily(byte[] columnName) {
+    for (int i = 0; i < columnName.length; i++) {
+      if (columnName[i] == COLON) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/DatabaseController.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/DatabaseController.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/DatabaseController.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/DatabaseController.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,84 @@
+/**
+ * Copyright 2008 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.rest;
+
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.rest.parser.IHBaseRestParser;
+
+public class DatabaseController extends AbstractController {
+
+  @SuppressWarnings("unused")
+  private Log LOG = LogFactory.getLog(DatabaseController.class);
+
+  protected DatabaseModel getModel() {
+    return (DatabaseModel) model;
+  }
+
+  @Override
+  protected AbstractModel generateModel(
+      @SuppressWarnings("hiding") HBaseConfiguration conf,
+      HBaseAdmin admin) {
+    return new DatabaseModel(conf, admin);
+  }
+
+  @Override
+  public void get(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap) throws HBaseRestException {
+    s.setNoQueryResults();
+    DatabaseModel innerModel = getModel();
+
+    if (queryMap.size() == 0) {
+      s.setOK(innerModel.getDatabaseMetadata());
+    } else {
+      s.setBadRequest("Unknown query.");
+    }
+    s.respond();
+  }
+
+  @Override
+  public void post(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap, byte[] input, IHBaseRestParser parser)
+      throws HBaseRestException {
+    s.setMethodNotImplemented();
+    s.respond();
+
+  }
+
+  @Override
+  public void put(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap, byte[] input, IHBaseRestParser parser)
+      throws HBaseRestException {
+    s.setMethodNotImplemented();
+    s.respond();
+  }
+
+  @Override
+  public void delete(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap) throws HBaseRestException {
+    s.setMethodNotImplemented();
+    s.respond();
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/DatabaseModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/DatabaseModel.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/DatabaseModel.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/DatabaseModel.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,85 @@
+/**
+ * Copyright 2008 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.rest;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.rest.serializer.IRestSerializer;
+import org.apache.hadoop.hbase.rest.serializer.ISerializable;
+
+import agilejson.TOJSON;
+
+public class DatabaseModel extends AbstractModel {
+
+  @SuppressWarnings("unused")
+  private Log LOG = LogFactory.getLog(DatabaseModel.class);
+
+  public DatabaseModel(HBaseConfiguration conf, HBaseAdmin admin) {
+    super.initialize(conf, admin);
+  }
+
+  public static class DatabaseMetadata implements ISerializable {
+    protected boolean master_running;
+    protected HTableDescriptor[] tables;
+
+    public DatabaseMetadata(HBaseAdmin a) throws IOException {
+      master_running = a.isMasterRunning();
+      tables = a.listTables();
+    }
+
+    @TOJSON(prefixLength = 2)
+    public boolean isMasterRunning() {
+      return master_running;
+    }
+
+    @TOJSON
+    public HTableDescriptor[] getTables() {
+      return tables;
+    }
+
+    public void restSerialize(IRestSerializer serializer)
+        throws HBaseRestException {
+      serializer.serializeDatabaseMetadata(this);
+    }
+  }
+
+  // Serialize admin ourselves to json object
+  // rather than returning the admin object for obvious reasons
+  public DatabaseMetadata getMetadata() throws HBaseRestException {
+    return getDatabaseMetadata();
+  }
+
+  protected DatabaseMetadata getDatabaseMetadata() throws HBaseRestException {
+    DatabaseMetadata databaseMetadata = null;
+    try {
+      databaseMetadata = new DatabaseMetadata(this.admin);
+    } catch (IOException e) {
+      throw new HBaseRestException(e);
+    }
+
+    return databaseMetadata;
+  }
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java?rev=736503&r1=736502&r2=736503&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java Wed Jan 21 16:21:16 2009
@@ -19,7 +19,12 @@
  */
 package org.apache.hadoop.hbase.rest;
 
+import java.io.BufferedReader;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
 
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
@@ -27,49 +32,98 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.rest.parser.HBaseRestParserFactory;
+import org.apache.hadoop.hbase.rest.parser.IHBaseRestParser;
+import org.apache.hadoop.hbase.rest.serializer.RestSerializerFactory;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.InfoServer;
-import org.apache.hadoop.mapred.StatusHttpServer;
 import org.mortbay.http.NCSARequestLog;
 import org.mortbay.http.SocketListener;
 import org.mortbay.jetty.servlet.WebApplicationContext;
 
 /**
- * Servlet implementation class for hbase REST interface.
- * Presumes container ensures single thread through here at any one time
- * (Usually the default configuration).  In other words, code is not
- * written thread-safe.
- * <p>This servlet has explicit dependency on Jetty server; it uses the
- * jetty implementation of MultipartResponse.
+ * Servlet implementation class for hbase REST interface. Presumes container
+ * ensures single thread through here at any one time (Usually the default
+ * configuration). In other words, code is not written thread-safe.
+ * <p>
+ * This servlet has explicit dependency on Jetty server; it uses the jetty
+ * implementation of MultipartResponse.
  * 
- * <p>TODO:
+ * <p>
+ * TODO:
  * <ul>
- * <li>multipart/related response is not correct; the servlet setContentType
- * is broken.  I am unable to add parameters such as boundary or start to
- * multipart/related.  They get stripped.</li>
- * <li>Currently creating a scanner, need to specify a column.  Need to make
- * it so the HTable instance has current table's metadata to-hand so easy to
- * find the list of all column families so can make up list of columns if none
+ * <li>multipart/related response is not correct; the servlet setContentType is
+ * broken. I am unable to add parameters such as boundary or start to
+ * multipart/related. They get stripped.</li>
+ * <li>Currently creating a scanner, need to specify a column. Need to make it
+ * so the HTable instance has current table's metadata to-hand so easy to find
+ * the list of all column families so can make up list of columns if none
  * specified.</li>
  * <li>Minor items are we are decoding URLs in places where probably already
  * done and how to timeout scanners that are in the scanner list.</li>
  * </ul>
- * @see <a href="http://wiki.apache.org/lucene-hadoop/Hbase/HbaseRest">Hbase REST Specification</a>
+ * 
+ * @see <a href="http://wiki.apache.org/lucene-hadoop/Hbase/HbaseRest">Hbase
+ *      REST Specification</a>
  */
-@SuppressWarnings("serial")
-public class Dispatcher extends javax.servlet.http.HttpServlet
-implements javax.servlet.Servlet {
-  @SuppressWarnings("unused")
-  private static final Log LOG = LogFactory.getLog(Dispatcher.class.getName());
-  private MetaHandler metaHandler;
-  private TableHandler tableHandler;
-  private RowHandler rowHandler;
-  private ScannerHandler scannerHandler;
+public class Dispatcher extends javax.servlet.http.HttpServlet {
+
+  /**
+   * 
+   */
+  private static final long serialVersionUID = -8075335435797071569L;
+  private static final Log LOG = LogFactory.getLog(Dispatcher.class);
+  protected DatabaseController dbController;
+  protected TableController tableController;
+  protected RowController rowController;
+  protected ScannerController scannercontroller;
+  protected TimestampController tsController;
+
+  public enum ContentType {
+    XML("text/xml"), JSON("application/json"), PLAIN("text/plain"), MIME(
+        "multipart/related"), NOT_ACCEPTABLE("");
+
+    private final String type;
+
+    private ContentType(final String t) {
+      this.type = t;
+    }
+
+    @Override
+    public String toString() {
+      return this.type;
+    }
+
+    /**
+     * Utility method used looking at Accept header content.
+     * 
+     * @param t
+     *          The content type to examine.
+     * @return The enum that matches the prefix of <code>t</code> or the default
+     *         enum if <code>t</code> is empty. If unsupported type, we return
+     *         NOT_ACCEPTABLE.
+     */
+    public static ContentType getContentType(final String t) {
+      // Default to text/plain. Curl sends */*.
+      if (t == null || t.equals("*/*")) {
+        return ContentType.XML;
+      }
+      String lowerCased = t.toLowerCase();
+      ContentType[] values = ContentType.values();
+      ContentType result = null;
+      for (int i = 0; i < values.length; i++) {
+        if (lowerCased.startsWith(values[i].type)) {
+          result = values[i];
+          break;
+        }
+      }
+      return result == null ? NOT_ACCEPTABLE : result;
+    }
+  }
 
-  private static final String SCANNER = "scanner";
-  private static final String ROW = "row";
-      
   /**
    * Default constructor
    */
@@ -80,149 +134,293 @@
   @Override
   public void init() throws ServletException {
     super.init();
-    
+
     HBaseConfiguration conf = new HBaseConfiguration();
     HBaseAdmin admin = null;
-    
-    try{
+
+    try {
       admin = new HBaseAdmin(conf);
-      metaHandler = new MetaHandler(conf, admin);
-      tableHandler = new TableHandler(conf, admin);
-      rowHandler = new RowHandler(conf, admin);
-      scannerHandler = new ScannerHandler(conf, admin);
-    } catch(Exception e){
+      createControllers();
+
+      dbController.initialize(conf, admin);
+      tableController.initialize(conf, admin);
+      rowController.initialize(conf, admin);
+      tsController.initialize(conf, admin);
+      scannercontroller.initialize(conf, admin);
+
+      LOG.debug("no errors in init.");
+    } catch (Exception e) {
+      System.out.println(e.toString());
       throw new ServletException(e);
     }
   }
 
+  protected void createControllers() {
+    dbController = new DatabaseController();
+    tableController = new TableController();
+    rowController = new RowController();
+    tsController = new TimestampController();
+    scannercontroller = new ScannerController();
+  }
+
+  @SuppressWarnings("unchecked")
   @Override
   protected void doGet(HttpServletRequest request, HttpServletResponse response)
-  throws IOException, ServletException {
-    String [] pathSegments = getPathSegments(request);
-    
-    if (pathSegments.length == 0 || pathSegments[0].length() <= 0) {
-      // if it was a root request, then get some metadata about 
-      // the entire instance.
-      metaHandler.doGet(request, response, pathSegments);
-    } else {
-      if (pathSegments.length >= 2 && pathSegments[0].length() > 0 && pathSegments[1].toLowerCase().equals(ROW)) {
-        // if it has table name and row path segments
-        rowHandler.doGet(request, response, pathSegments);
+      throws IOException, ServletException {
+    try {
+      Status s = this.createStatus(request, response);
+      byte[][] pathSegments = getPathSegments(request);
+      Map<String, String[]> queryMap = request.getParameterMap();
+
+      if (pathSegments.length == 0 || pathSegments[0].length <= 0) {
+        // if it was a root request, then get some metadata about
+        // the entire instance.
+        dbController.get(s, pathSegments, queryMap);
       } else {
-        // otherwise, it must be a GET request suitable for the
-        // table handler.
-        tableHandler.doGet(request, response, pathSegments);
+        if (pathSegments.length >= 2
+            && pathSegments.length <= 3
+            && pathSegments[0].length > 0
+            && Bytes.toString(pathSegments[1]).toLowerCase().equals(
+                RESTConstants.ROW)) {
+          // if it has table name and row path segments
+          rowController.get(s, pathSegments, queryMap);
+        } else if (pathSegments.length == 4
+            && Bytes.toString(pathSegments[1]).toLowerCase().equals(
+                RESTConstants.ROW)) {
+          tsController.get(s, pathSegments, queryMap);
+        } else {
+          // otherwise, it must be a GET request suitable for the
+          // table handler.
+          tableController.get(s, pathSegments, queryMap);
+        }
+      }
+      LOG.debug("GET - No Error");
+    } catch (HBaseRestException e) {
+      LOG.debug("GET - Error: " + e.toString());
+      try {
+        Status sError = createStatus(request, response);
+        sError.setInternalError(e);
+        sError.respond();
+      } catch (HBaseRestException f) {
+        response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
       }
     }
   }
 
+  @SuppressWarnings("unchecked")
   @Override
   protected void doPost(HttpServletRequest request, HttpServletResponse response)
-  throws IOException, ServletException {
-    String [] pathSegments = getPathSegments(request);
-    
-    if (pathSegments.length == 0 || pathSegments[0].length() <= 0) {
-      // if it was a root request, it must be a create table request
-      tableHandler.doPost(request, response, pathSegments);
-      return;
-    } else {
-      // there should be at least two path segments (table name and row or
-      // scanner or disable/enable operation)
-      if (pathSegments.length >= 2 && pathSegments[0].length() > 0) {
-        if (pathSegments[1].toLowerCase().equals(SCANNER)
-            && pathSegments.length >= 2) {
-          scannerHandler.doPost(request, response, pathSegments);
-          return;
-        } else if (pathSegments[1].toLowerCase().equals(ROW)
-            && pathSegments.length >= 3) {
-          rowHandler.doPost(request, response, pathSegments);
-          return;
-        } else if ((pathSegments[1].toLowerCase().equals(TableHandler.DISABLE) || pathSegments[1].toLowerCase().equals(TableHandler.ENABLE))
-            && pathSegments.length == 2) {
-          tableHandler.doPost(request, response, pathSegments);
-          return;
+      throws IOException, ServletException {
+    try {
+
+      Status s = createStatus(request, response);
+      byte[][] pathSegments = getPathSegments(request);
+      Map<String, String[]> queryMap = request.getParameterMap();
+      byte[] input = readInputBuffer(request);
+      IHBaseRestParser parser = this.getParser(request);
+
+      if ((pathSegments.length >= 0 && pathSegments.length <= 1)
+          || Bytes.toString(pathSegments[1]).toLowerCase().equals(
+              RESTConstants.ENABLE)
+          || Bytes.toString(pathSegments[1]).toLowerCase().equals(
+              RESTConstants.DISABLE)) {
+        // this is a table request
+        tableController.post(s, pathSegments, queryMap, input, parser);
+      } else {
+        // there should be at least two path segments (table name and row or
+        // scanner)
+        if (pathSegments.length >= 2 && pathSegments[0].length > 0) {
+          if (Bytes.toString(pathSegments[1]).toLowerCase().equals(
+              RESTConstants.SCANNER)) {
+            scannercontroller.post(s, pathSegments, queryMap, input, parser);
+            return;
+          } else if (Bytes.toString(pathSegments[1]).toLowerCase().equals(
+              RESTConstants.ROW)
+              && pathSegments.length >= 3) {
+            rowController.post(s, pathSegments, queryMap, input, parser);
+            return;
+          }
         }
       }
+    } catch (HBaseRestException e) {
+      LOG.debug("POST - Error: " + e.toString());
+      try {
+        Status s_error = createStatus(request, response);
+        s_error.setInternalError(e);
+        s_error.respond();
+      } catch (HBaseRestException f) {
+        response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      }
     }
-
-    // if we get to this point, then no handler was matched this request.
-    GenericHandler.doNotFound(response, "No handler for " + request.getPathInfo());
   }
-  
 
+  @SuppressWarnings("unchecked")
   @Override
   protected void doPut(HttpServletRequest request, HttpServletResponse response)
-  throws ServletException, IOException {
-    String [] pathSegments = getPathSegments(request);
-    
-    if (pathSegments.length == 1 && pathSegments[0].length() > 0) {
-      // if it has only table name
-      tableHandler.doPut(request, response, pathSegments);
-    } else {
-      // Equate PUT with a POST.
-      doPost(request, response);
+      throws ServletException, IOException {
+    try {
+      byte[][] pathSegments = getPathSegments(request);
+      if(pathSegments.length == 0) {
+        throw new HBaseRestException("method not supported");
+      } else if (pathSegments.length == 1 && pathSegments[0].length > 0) {
+        // if it has only table name
+        Status s = createStatus(request, response);
+        Map<String, String[]> queryMap = request.getParameterMap();
+        IHBaseRestParser parser = this.getParser(request);
+        byte[] input = readInputBuffer(request);
+        tableController.put(s, pathSegments, queryMap, input, parser);
+      } else {
+        // Equate PUT with a POST.
+        doPost(request, response);
+      }
+    } catch (HBaseRestException e) {
+      try {
+        Status s_error = createStatus(request, response);
+        s_error.setInternalError(e);
+        s_error.respond();
+      } catch (HBaseRestException f) {
+        response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      }
     }
   }
 
+  @SuppressWarnings("unchecked")
   @Override
   protected void doDelete(HttpServletRequest request,
-      HttpServletResponse response)
-  throws IOException, ServletException {
-    String [] pathSegments = getPathSegments(request);
-    
-    if (pathSegments.length == 1 && pathSegments[0].length() > 0) {
-      // if it only has only table name
-      tableHandler.doDelete(request, response, pathSegments);
-      return;
-    } else if (pathSegments.length >= 3 && pathSegments[0].length() > 0) {
-      // must be at least two path segments (table name and row or scanner)
-      if (pathSegments[1].toLowerCase().equals(SCANNER) &&
-          pathSegments.length == 3 && pathSegments[2].length() > 0) {
-        // DELETE to a scanner requires at least three path segments
-        scannerHandler.doDelete(request, response, pathSegments);
-        return;
-      } else if (pathSegments[1].toLowerCase().equals(ROW) &&
-          pathSegments.length >= 3) {
-        rowHandler.doDelete(request, response, pathSegments);
+      HttpServletResponse response) throws IOException, ServletException {
+    try {
+      Status s = createStatus(request, response);
+      byte[][] pathSegments = getPathSegments(request);
+      Map<String, String[]> queryMap = request.getParameterMap();
+      
+      if(pathSegments.length == 0) {
+        throw new HBaseRestException("method not supported");
+      } else if (pathSegments.length == 1 && pathSegments[0].length > 0) {
+        // if it only has only table name
+        tableController.delete(s, pathSegments, queryMap);
         return;
+      } else if (pathSegments.length >= 3 && pathSegments[0].length > 0) {
+        // must be at least two path segments (table name and row or scanner)
+        if (Bytes.toString(pathSegments[1]).toLowerCase().equals(
+            RESTConstants.SCANNER)
+            && pathSegments.length == 3 && pathSegments[2].length > 0) {
+          // DELETE to a scanner requires at least three path segments
+          scannercontroller.delete(s, pathSegments, queryMap);
+          return;
+        } else if (Bytes.toString(pathSegments[1]).toLowerCase().equals(
+            RESTConstants.ROW)
+            && pathSegments.length >= 3) {
+          rowController.delete(s, pathSegments, queryMap);
+          return;
+        } else if (pathSegments.length == 4) {
+          tsController.delete(s, pathSegments, queryMap);
+        }
+      }
+    } catch (HBaseRestException e) {
+      LOG.debug("POST - Error: " + e.toString());
+      try {
+        Status s_error = createStatus(request, response);
+        s_error.setInternalError(e);
+        s_error.respond();
+      } catch (HBaseRestException f) {
+        response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
       }
     }
-    
-    // if we reach this point, then no handler exists for this request.
-    GenericHandler.doNotFound(response, "No handler");
   }
-  
-  /*
+
+  /**
+   * This method will get the path segments from the HttpServletRequest.  Please
+   * note that if the first segment of the path is /api this is removed from the 
+   * returning byte array.
+   * 
    * @param request
+   * 
    * @return request pathinfo split on the '/' ignoring the first '/' so first
    * element in pathSegment is not the empty string.
    */
-  private String [] getPathSegments(final HttpServletRequest request) {
+  protected byte[][] getPathSegments(final HttpServletRequest request) {
     int context_len = request.getContextPath().length() + 1;
-    return request.getRequestURI().substring(context_len).split("/");
+    
+    byte[][] pathSegments = Bytes.toByteArrays(request.getRequestURI().substring(context_len)
+        .split("/"));
+    byte[] apiAsBytes = "api".getBytes();
+    if (Arrays.equals(apiAsBytes, pathSegments[0])) {
+      byte[][] newPathSegments = new byte[pathSegments.length - 1][];
+      for(int i = 0; i < newPathSegments.length; i++) {
+        newPathSegments[i] = pathSegments[i + 1];
+      }
+      pathSegments = newPathSegments;
+    }
+    return pathSegments;
+  }
+
+  protected byte[] readInputBuffer(HttpServletRequest request)
+      throws HBaseRestException {
+    try {
+      String resultant = "";
+      BufferedReader r = request.getReader();
+
+      int maxLength = 5000; // tie to conf
+      int bufferLength = 640;
+
+      char[] c = new char[bufferLength]; // 40 characters * sizeof(UTF16)
+      // TODO make s maxLength and c size values in configuration
+      if (!r.ready()) {
+        Thread.sleep(1000); // If r is not ready wait 1 second
+        if (!r.ready()) { // If r still is not ready something is wrong, return
+          // blank.
+          return new byte[0];
+        }
+      }
+      while (r.ready()) {
+        int n = r.read(c, 0, bufferLength);
+        resultant += new String(c);
+        if (n != bufferLength) {
+          break;
+        } else if (resultant.length() > maxLength) {
+          resultant = resultant.substring(0, maxLength);
+          break;
+        }
+      }
+      return Bytes.toBytes(resultant.trim());
+    } catch (Exception e) {
+      throw new HBaseRestException(e);
+    }
+  }
+
+  protected IHBaseRestParser getParser(HttpServletRequest request)
+      throws HBaseRestException {
+    return HBaseRestParserFactory.getParser(ContentType.getContentType(request
+        .getHeader("content-type")));
+  }
+
+  protected Status createStatus(HttpServletRequest request,
+      HttpServletResponse response) throws HBaseRestException {
+    return new Status(response, RestSerializerFactory.getSerializer(request,
+        response), this.getPathSegments(request));
   }
 
   //
   // Main program and support routines
   //
-  
-  private static void printUsageAndExit() {
+  protected static void printUsageAndExit() {
     printUsageAndExit(null);
   }
-  
-  private static void printUsageAndExit(final String message) {
+
+  protected static void printUsageAndExit(final String message) {
     if (message != null) {
       System.err.println(message);
     }
-    System.out.println("Usage: java org.apache.hadoop.hbase.rest.Dispatcher " +
-      "--help | [--port=PORT] [--bind=ADDR] start");
+    System.out.println("Usage: java org.apache.hadoop.hbase.rest.Dispatcher "
+        + "--help | [--port=PORT] [--bind=ADDR] start");
     System.out.println("Arguments:");
     System.out.println(" start Start REST server");
     System.out.println(" stop  Stop REST server");
     System.out.println("Options:");
     System.out.println(" port  Port to listen on. Default: 60050.");
     System.out.println(" bind  Address to bind on. Default: 0.0.0.0.");
-    System.out.println(" max-num-threads  The maximum number of threads for Jetty to run. Defaults to 256.");
+    System.out
+        .println(" max-num-threads  The maximum number of threads for Jetty to run. Defaults to 256.");
     System.out.println(" help  Print this message and exit.");
 
     System.exit(0);
@@ -230,9 +428,10 @@
 
   /*
    * Start up the REST servlet in standalone mode.
+   * 
    * @param args
    */
-  protected static void doMain(final String [] args) throws Exception {
+  protected static void doMain(final String[] args) throws Exception {
     if (args.length < 1) {
       printUsageAndExit();
     }
@@ -246,7 +445,7 @@
     final String addressArgKey = "--bind=";
     final String portArgKey = "--port=";
     final String numThreadsKey = "--max-num-threads=";
-    for (String cmd: args) {
+    for (String cmd : args) {
       if (cmd.startsWith(addressArgKey)) {
         bindAddress = cmd.substring(addressArgKey.length());
         continue;
@@ -258,14 +457,14 @@
       } else if (cmd.equals("start")) {
         continue;
       } else if (cmd.equals("stop")) {
-        printUsageAndExit("To shutdown the REST server run " +
-          "bin/hbase-daemon.sh stop rest or send a kill signal to " +
-          "the REST server pid");
-      } else if (cmd.startsWith(numThreadsKey)) { 
+        printUsageAndExit("To shutdown the REST server run "
+            + "bin/hbase-daemon.sh stop rest or send a kill signal to "
+            + "the REST server pid");
+      } else if (cmd.startsWith(numThreadsKey)) {
         numThreads = Integer.parseInt(cmd.substring(numThreadsKey.length()));
         continue;
       }
-      
+
       // Print out usage if we get to here.
       printUsageAndExit();
     }
@@ -278,20 +477,17 @@
     NCSARequestLog ncsa = new NCSARequestLog();
     ncsa.setLogLatency(true);
     webServer.setRequestLog(ncsa);
-    WebApplicationContext context =
-      webServer.addWebApplication("/api", InfoServer.getWebAppDir("rest"));
-    context.addServlet("stacks", "/stacks",
-      StatusHttpServer.StackServlet.class.getName());
-    context.addServlet("logLevel", "/logLevel",
-      org.apache.hadoop.log.LogLevel.Servlet.class.getName());
+    WebApplicationContext context = webServer.addWebApplication("/", InfoServer
+        .getWebAppDir("rest"));
     webServer.start();
   }
-  
+
   /**
    * @param args
-   * @throws Exception 
+   * @throws Exception
    */
-  public static void main(String [] args) throws Exception {
+  public static void main(String[] args) throws Exception {
+    System.out.println("Starting restServer");
     doMain(args);
   }
 }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java?rev=736503&r1=736502&r2=736503&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java Wed Jan 21 16:21:16 2009
@@ -1,342 +0,0 @@
-/**
- * Copyright 2007 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.rest;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
-import java.util.Map;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.znerd.xmlenc.LineBreak;
-import org.znerd.xmlenc.XMLOutputter;
-
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.io.Cell;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * GenericHandler contains some basic common stuff that all the individual
- * REST handler types take advantage of.
- */
-public abstract class GenericHandler {
-  protected HBaseConfiguration conf;
-  protected HBaseAdmin admin;
-
-  protected static final String ACCEPT = "accept";
-  protected static final String COLUMN = "column";
-  protected static final String TIMESTAMP = "timestamp";
-  protected static final String START_ROW = "start_row";
-  protected static final String END_ROW = "end_row";
-  protected static final String CONTENT_TYPE = "content-type";
-  protected static final String ROW = "row";
-  protected static final String REGIONS = "regions";
-  protected static final String VERSION = "version";
-  protected static final String OFFSET = "offset";
-  protected static final String LIMIT = "limit";
-  
-  protected final Log LOG = LogFactory.getLog(this.getClass());
-
-  public GenericHandler(HBaseConfiguration conf, HBaseAdmin admin)  {
-    this.conf = conf;
-    this.admin = admin;
-  }
-
-  /*
-   * Supported content types as enums
-   */
-  protected enum ContentType {
-    XML("text/xml"),
-    PLAIN("text/plain"),
-    MIME("multipart/related"),
-    NOT_ACCEPTABLE("");
-    
-    private final String type;
-    
-    private ContentType(final String t) {
-      this.type = t;
-    }
-    
-    @Override
-    public String toString() {
-      return this.type;
-    }
-    
-    /**
-     * Utility method used looking at Accept header content.
-     * @param t The content type to examine.
-     * @return The enum that matches the prefix of <code>t</code> or
-     * the default enum if <code>t</code> is empty.  If unsupported type, we
-     * return NOT_ACCEPTABLE.
-     */
-    public static ContentType getContentType(final String t) {
-      // Default to text/plain. Curl sends */*.
-      if (t == null || t.equals("*/*")) { 
-        return ContentType.XML;
-      }
-      String lowerCased = t.toLowerCase();
-      ContentType [] values = ContentType.values();
-      ContentType result = null;
-      for (int i = 0; i < values.length; i++) {
-        if (lowerCased.startsWith(values[i].type)) {
-          result = values[i];
-          break;
-        }
-      }
-      return result == null? NOT_ACCEPTABLE: result;
-    }
-  }
-
-  
-  /*
-   * @param o
-   * @return XMLOutputter wrapped around <code>o</code>.
-   * @throws IllegalStateException
-   * @throws IOException
-   */
-  protected XMLOutputter getXMLOutputter(final PrintWriter o)
-  throws IllegalStateException, IOException {
-    XMLOutputter outputter = new XMLOutputter(o, HConstants.UTF8_ENCODING);
-    outputter.setLineBreak(LineBreak.UNIX);
-    outputter.setIndentation(" ");
-    outputter.declaration();
-    return outputter;
-  }
-  
-  /*
-   * Write an XML element.
-   * @param outputter
-   * @param name
-   * @param value
-   * @throws IllegalStateException
-   * @throws IOException
-   */
-  protected void doElement(final XMLOutputter outputter,
-      final String name, final String value)
-  throws IllegalStateException, IOException {
-    outputter.startTag(name);
-    if (value.length() > 0) {
-      outputter.pcdata(value);
-    }
-    outputter.endTag();
-  }
-  
-  /*
-   * Set content-type, encoding, and status on passed <code>response</code>
-   * @param response
-   * @param status
-   * @param contentType
-   */
-  public static void setResponseHeader(final HttpServletResponse response,
-      final int status, final String contentType) {
-    // Container adds the charset to the HTTP content-type header.
-    response.setContentType(contentType);
-    response.setCharacterEncoding(HConstants.UTF8_ENCODING);
-    response.setStatus(status);
-  }
-
-  /*
-   * If we can't do the specified Accepts header type.
-   * @param response
-   * @throws IOException
-   */
-  public static void doNotAcceptable(final HttpServletResponse response)
-  throws IOException {
-    response.sendError(HttpServletResponse.SC_NOT_ACCEPTABLE);
-  }
-
-  /*
-   * If we can't do the specified Accepts header type.
-   * @param response
-   * @param message
-   * @throws IOException
-   */
-  public static void doNotAcceptable(final HttpServletResponse response,
-      final String message)
-  throws IOException {
-    response.sendError(HttpServletResponse.SC_NOT_ACCEPTABLE, message);
-  }
-  
-  /*
-   * Resource not found.
-   * @param response
-   * @throws IOException
-   */
-  public static void doNotFound(final HttpServletResponse response)
-  throws IOException {
-    response.sendError(HttpServletResponse.SC_NOT_FOUND);
-  }
-  
-  /*
-   * Resource not found.
-   * @param response
-   * @param msg
-   * @throws IOException
-   */
-  public static void doNotFound(final HttpServletResponse response, final String msg)
-  throws IOException {
-    response.sendError(HttpServletResponse.SC_NOT_FOUND, msg);
-  }
-
-  /*
-   * Unimplemented method.
-   * @param response
-   * @param message to send
-   * @throws IOException
-   */
-  public static void doMethodNotAllowed(final HttpServletResponse response,
-      final String message)
-  throws IOException {
-    response.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED, message);
-  }
-
-  protected String getTableName(final String [] pathSegments)
-  throws UnsupportedEncodingException {
-    // Get table name?  First part of passed segment.  It can't be empty string
-    // or null because we should have tested for that before coming in here.
-    return URLDecoder.decode(pathSegments[0], HConstants.UTF8_ENCODING);
-  }
- 
-  /*
-   * Output row columns 
-   * @param outputter
-   * @param m
-   * @throws IllegalStateException
-   * @throws IllegalArgumentException
-   * @throws IOException
-   */
-  protected void outputColumnsXml(final XMLOutputter outputter, final Map<byte [], Cell> m)
-  throws IllegalStateException, IllegalArgumentException, IOException {
-    outputColumnsXml(null, outputter, m);
-  }
-  
-  protected void outputColumnsXml(final HttpServletRequest request,
-      final XMLOutputter outputter, final Map<byte [], Cell> m)
-  throws IllegalStateException, IllegalArgumentException, IOException {
-    int offset = 0, limit = Integer.MAX_VALUE;
-    if (request != null) {
-      String offset_string = request.getParameter(OFFSET);
-      if (offset_string != null && !offset_string.equals(""))
-        offset = Integer.parseInt(offset_string);
-      String limit_string = request.getParameter(LIMIT);
-      if (limit_string != null && !limit_string.equals("")) {
-        limit = Integer.parseInt(limit_string);
-      }
-    }
-    
-    for (Map.Entry<byte [], Cell> e: m.entrySet()) {
-      if (offset > 0) {
-        --offset;
-        continue;
-      }
-      if (limit < 1) {
-        break;
-      } else {
-        --limit;
-      }
-      outputter.startTag(COLUMN);
-      doElement(outputter, "name", 
-        org.apache.hadoop.hbase.util.Base64.encodeBytes(e.getKey()));
-      outputCellXml(outputter, e.getValue());
-      outputter.endTag();
-    }
-  }
-
-  protected void outputColumnsWithMultiVersionsXml(final XMLOutputter outputter, 
-      final Map<byte [], Cell[]> m)
-  throws IllegalStateException, IllegalArgumentException, IOException {
-    outputColumnsWithMultiVersionsXml(null, outputter, m);
-  }
-  
-  protected void outputColumnsWithMultiVersionsXml(final HttpServletRequest request, 
-      final XMLOutputter outputter, final Map<byte [], Cell[]> m)
-  throws IllegalStateException, IllegalArgumentException, IOException {
-    int offset = 0, limit = Integer.MAX_VALUE;
-    if (request != null) {
-      String offset_string = request.getParameter(OFFSET);
-      if (offset_string != null && !offset_string.equals(""))
-        offset = Integer.parseInt(offset_string);
-      String limit_string = request.getParameter(LIMIT);
-      if (limit_string != null && !limit_string.equals("")) {
-        limit = Integer.parseInt(limit_string);
-      }
-    }
-    
-    for (Map.Entry<byte [], Cell[]> e: m.entrySet()) {
-      if (offset > 0) {
-        --offset;
-        continue;
-      }
-      if (limit < 1) {
-        break;
-      } else {
-        --limit;
-      }
-      for (Cell c : e.getValue()) {
-        outputter.startTag(COLUMN);
-        doElement(outputter, "name", 
-            org.apache.hadoop.hbase.util.Base64.encodeBytes(e.getKey())); 
-        outputCellXml(outputter, c);
-        outputter.endTag();       
-      }
-    }
-  }
-  
-  protected void outputCellXml(final XMLOutputter outputter, Cell c) 
-  throws IllegalStateException, IllegalArgumentException, IOException {
-    // We don't know String from binary data so we always base64 encode.
-    doElement(outputter, "value",
-        org.apache.hadoop.hbase.util.Base64.encodeBytes(c.getValue()));
-    doElement(outputter, "timestamp", String.valueOf(c.getTimestamp()));    
-  }
-//  Commented - multipart support is currently nonexistant.
-//  protected void outputColumnsMime(final MultiPartResponse mpr,
-//     final Map<Text, Cell> m)
-//   throws IOException {
-//     for (Map.Entry<Text, Cell> e: m.entrySet()) {
-//       mpr.startPart("application/octet-stream",
-//         new String [] {"Content-Description: " + e.getKey().toString(),
-//           "Content-Transfer-Encoding: binary",
-//           "Content-Length: " + e.getValue().getValue().length});
-//       mpr.getOut().write(e.getValue().getValue());
-//     }  
-//   }
- 
-  /*
-   * Get an HTable instance by it's table name.
-   */
-  protected HTable getTable(final String tableName) throws IOException {
-    return new HTable(this.conf, Bytes.toBytes(tableName));
-  }
-  
-  protected String makeColumnName(String column) {
-   if (column.indexOf(':') == -1)
-     column += ':';
-   return column;
-  }
-}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java?rev=736503&r1=736502&r2=736503&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java Wed Jan 21 16:21:16 2009
@@ -1,108 +0,0 @@
-/**
- * Copyright 2007 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.rest;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.znerd.xmlenc.XMLOutputter;
-
-
-/**
- * MetaHandler fields all requests for metadata at the instance level. At the
- * moment this is only GET requests to /.
- */
-public class MetaHandler extends GenericHandler {
-
-  public MetaHandler(HBaseConfiguration conf, HBaseAdmin admin) 
-  throws ServletException{
-    super(conf, admin);
-  }
-   
-   
-  public void doGet(HttpServletRequest request, HttpServletResponse response, 
-    String[] pathSegments)
-  throws ServletException, IOException {
-    getTables(request, response);
-  }
-  
-  public void doPost(HttpServletRequest request, HttpServletResponse response, 
-    String[] pathSegments)
-  throws ServletException, IOException {
-    doMethodNotAllowed(response, "POST not allowed at /");
-  }
-  
-  public void doPut(HttpServletRequest request, HttpServletResponse response, 
-    String[] pathSegments)
-  throws ServletException, IOException {
-    doMethodNotAllowed(response, "PUT not allowed at /");    
-  }
-  
-  public void doDelete(HttpServletRequest request, HttpServletResponse response, 
-    String[] pathSegments)
-  throws ServletException, IOException {
-    doMethodNotAllowed(response, "DELETE not allowed at /");
-  }
-     
-  /*
-   * Return list of tables. 
-   * @param request
-   * @param response
-   */
-  private void getTables(final HttpServletRequest request,
-    final HttpServletResponse response)
-  throws IOException {
-    HTableDescriptor [] tables = this.admin.listTables();
-
-    switch (ContentType.getContentType(request.getHeader(ACCEPT))) {
-      case XML:
-        setResponseHeader(response, tables.length > 0? 200: 204,
-            ContentType.XML.toString());
-          XMLOutputter outputter = getXMLOutputter(response.getWriter());
-          outputter.startTag("tables");
-          for (int i = 0; i < tables.length; i++) {
-            doElement(outputter, "table", Bytes.toString(tables[i].getName()));
-          }
-          outputter.endTag();
-          outputter.endDocument();
-          outputter.getWriter().close();
-        break;
-      case PLAIN:
-        setResponseHeader(response, tables.length > 0? 200: 204,
-            ContentType.PLAIN.toString());
-          PrintWriter out = response.getWriter();
-          for (int i = 0; i < tables.length; i++) {
-            out.println(Bytes.toString(tables[i].getName()));
-          }
-          out.close();
-        break;
-      default:
-        doNotAcceptable(response);
-    }
-  }
-}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RESTConstants.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RESTConstants.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RESTConstants.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RESTConstants.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,111 @@
+/**
+ * Copyright 2008 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.rest;
+
+import org.apache.hadoop.hbase.rest.filter.RowFilterSetFactory;
+import org.apache.hadoop.hbase.rest.filter.StopRowFilterFactory;
+import org.apache.hadoop.hbase.rest.filter.WhileMatchRowFilterFactory;
+import org.apache.hadoop.hbase.rest.filter.PageRowFilterFactory;
+import org.apache.hadoop.hbase.rest.filter.ColumnValueFilterFactory;
+import org.apache.hadoop.hbase.rest.filter.RegExpRowFilterFactory;
+import org.apache.hadoop.hbase.rest.filter.InclusiveStopRowFilterFactory;
+import java.util.HashMap;
+import org.apache.hadoop.hbase.rest.filter.FilterFactory;
+
+public interface RESTConstants {
+  final static String TRUE = "true";
+  final static String FALSE = "false";
+  // Used for getting all data from a column specified in that order.
+  final static String COLUMNS = "columns";
+  final static String COLUMN = "column";
+  // Used with TableExists
+  final static String EXISTS = "exists";
+  // Maps to Transaction ID
+  final static String TRANSACTION = "transaction";
+  // Transaction Operation Key.
+  final static String TRANSACTION_OPERATION = "transaction_op";
+  // Transaction Operation Values
+  final static String TRANSACTION_OPERATION_COMMIT = "commit";
+  final static String TRANSACTION_OPERATION_CREATE = "create";
+  final static String TRANSACTION_OPERATION_ABORT = "abort";
+  // Filter Key
+  final static String FILTER = "filter";
+  final static String FILTER_TYPE = "type";
+  final static String FILTER_VALUE = "value";
+  final static String FILTER_RANK = "rank";
+  // Scanner Key
+  final static String SCANNER = "scanner";
+  // The amount of rows to return at one time.
+  final static String SCANNER_RESULT_SIZE = "result_size";
+  final static String SCANNER_START_ROW = "start_row";
+  final static String SCANNER_STOP_ROW = "stop_row";
+  final static String SCANNER_FILTER = "filter";
+  final static String SCANNER_TIMESTAMP = "timestamp";
+  final static String NUM_VERSIONS = "num_versions";
+  final static String SCANNER_COLUMN = "column";
+  // static items used on the path
+  static final String DISABLE = "disable";
+  static final String ENABLE = "enable";
+  static final String REGIONS = "regions";
+  static final String ROW = "row";
+  static final String TIME_STAMPS = "timestamps";
+  static final String METADATA = "metadata";
+
+  static final String NAME = "name";
+  static final String VALUE = "value";
+  static final String ROWS = "rows";
+
+  static final FactoryMap filterFactories = FactoryMap.getFactoryMap();
+  static final String LIMIT = "limit";
+
+  static class FactoryMap {
+
+    protected static boolean created = false;
+    protected HashMap<String, FilterFactory> map = new HashMap<String, FilterFactory>();
+
+    protected FactoryMap() {
+    }
+
+    public static FactoryMap getFactoryMap() {
+      if (!created) {
+        created = true;
+        FactoryMap f = new FactoryMap();
+        f.initialize();
+        return f;
+      } else {
+        return null;
+      }
+    }
+
+    public FilterFactory get(String c) {
+      return map.get(c);
+    }
+
+    protected void initialize() {
+      map.put("ColumnValueFilter", new ColumnValueFilterFactory());
+      map.put("InclusiveStopRowFilter", new InclusiveStopRowFilterFactory());
+      map.put("PageRowFilter", new PageRowFilterFactory());
+      map.put("RegExpRowFilter", new RegExpRowFilterFactory());
+      map.put("RowFilterSet", new RowFilterSetFactory());
+      map.put("StopRowFilter", new StopRowFilterFactory());
+      map.put("WhileMatchRowFilter", new WhileMatchRowFilterFactory());
+    }
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RowController.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RowController.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RowController.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/RowController.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,135 @@
+/**
+ * Copyright 2008 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.rest;
+
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.rest.descriptors.RowUpdateDescriptor;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.rest.parser.IHBaseRestParser;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public class RowController extends AbstractController {
+
+  @SuppressWarnings("unused")
+  private Log LOG = LogFactory.getLog(RowController.class);
+
+  protected RowModel getModel() {
+    return (RowModel) model;
+  }
+
+  @Override
+  protected AbstractModel generateModel(
+      @SuppressWarnings("hiding") HBaseConfiguration conf, HBaseAdmin admin) {
+    return new RowModel(conf, admin);
+  }
+
+  @Override
+  public void get(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap) throws HBaseRestException {
+    RowModel innerModel = getModel();
+    s.setNoQueryResults();
+
+    byte[] tableName;
+    byte[] rowName;
+
+    tableName = pathSegments[0];
+    rowName = pathSegments[2];
+    RowResult row = null;
+    
+    if (queryMap.size() == 0 && pathSegments.length <= 3) {
+      row = innerModel.get(tableName, rowName);
+    } else if (pathSegments.length == 4
+        && Bytes.toString(pathSegments[3]).toLowerCase().equals(
+            RESTConstants.TIME_STAMPS)) {
+      innerModel.getTimestamps(tableName, rowName);
+    } else {
+      row = innerModel.get(tableName, rowName, this.getColumnsFromQueryMap(queryMap));
+    }
+    if(row == null) {
+      throw new HBaseRestException("row not found");
+    }
+    s.setOK(row);
+    s.respond();
+  }
+
+  @Override
+  public void post(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap, byte[] input, IHBaseRestParser parser)
+      throws HBaseRestException {
+    RowModel innerModel = getModel();
+
+    BatchUpdate b;
+    RowUpdateDescriptor rud = parser
+        .getRowUpdateDescriptor(input, pathSegments);
+
+    if (input.length == 0) {
+      s.setUnsupportedMediaType("no data send with post request");
+      s.respond();
+      return;
+    }
+
+    b = new BatchUpdate(rud.getRowName());
+
+    for (byte[] key : rud.getColVals().keySet()) {
+      b.put(key, rud.getColVals().get(key));
+    }
+
+    try {
+      innerModel.post(rud.getTableName().getBytes(), b);
+      s.setOK();
+    } catch (HBaseRestException e) {
+      s.setUnsupportedMediaType(e.getMessage());
+    }
+    s.respond();
+  }
+
+  @Override
+  public void put(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap, byte[] input, IHBaseRestParser parser)
+      throws HBaseRestException {
+    s.setMethodNotImplemented();
+    s.respond();
+  }
+
+  @Override
+  public void delete(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap) throws HBaseRestException {
+    RowModel innerModel = getModel();
+    byte[] tableName;
+    byte[] rowName;
+
+    tableName = pathSegments[0];
+    rowName = pathSegments[2];
+    if(queryMap.size() == 0) {
+      innerModel.delete(tableName, rowName);
+    } else {
+      innerModel.delete(tableName, rowName, this.getColumnsFromQueryMap(queryMap));
+    }
+    s.setOK();
+    s.respond();
+  }
+}