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 [3/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...

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableModel.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableModel.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableModel.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,280 @@
+/**
+ * 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.ArrayList;
+
+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.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Scanner;
+import org.apache.hadoop.hbase.io.RowResult;
+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 agilejson.TOJSON;
+
+public class TableModel extends AbstractModel {
+
+  @SuppressWarnings("unused")
+  private Log LOG = LogFactory.getLog(TableModel.class);
+
+  public TableModel(HBaseConfiguration config, HBaseAdmin admin) {
+    super.initialize(config, admin);
+  }
+
+  // Get Methods
+  public RowResult[] get(byte[] tableName) throws HBaseRestException {
+    return get(tableName, getColumns(tableName));
+  }
+
+  /**
+   * Returns all cells from all rows from the given table in the given columns.
+   * The output is in the order that the columns are given.
+   * 
+   * @param tableName
+   *          table name
+   * @param columnNames
+   *          column names
+   * @return resultant rows
+   * @throws org.apache.hadoop.hbase.rest.exception.HBaseRestException
+   */
+  public RowResult[] get(byte[] tableName, byte[][] columnNames)
+      throws HBaseRestException {
+    try {
+      ArrayList<RowResult> a = new ArrayList<RowResult>();
+      HTable table = new HTable(tableName);
+
+      Scanner s = table.getScanner(columnNames);
+      RowResult r;
+
+      while ((r = s.next()) != null) {
+        a.add(r);
+      }
+
+      return a.toArray(new RowResult[0]);
+    } catch (Exception e) {
+      throw new HBaseRestException(e);
+    }
+  }
+
+  protected boolean doesTableExist(byte[] tableName) throws HBaseRestException {
+    try {
+      return this.admin.tableExists(tableName);
+    } catch (IOException e) {
+      throw new HBaseRestException(e);
+    }
+  }
+  
+  protected void disableTable(byte[] tableName) throws HBaseRestException {
+    try {
+      this.admin.disableTable(tableName);
+    } catch (IOException e) {
+      throw new HBaseRestException("IOException disabling table", e);
+    }
+  }
+  
+  protected void enableTable(byte[] tableName) throws HBaseRestException {
+    try {
+      this.admin.enableTable(tableName);
+    } catch (IOException e) {
+      throw new HBaseRestException("IOException enabiling table", e);
+    }
+  }
+
+  public boolean updateTable(String tableName,
+      ArrayList<HColumnDescriptor> columns) throws HBaseRestException {
+    HTableDescriptor htc = null;
+    try {
+      htc = this.admin.getTableDescriptor(tableName);
+    } catch (IOException e) {
+      throw new HBaseRestException("Table does not exist");
+    }
+
+    for (HColumnDescriptor column : columns) {
+      if (htc.hasFamily(Bytes.toBytes(column.getNameAsString()))) {
+        try {
+          this.admin.disableTable(tableName);
+          this.admin.modifyColumn(tableName, column.getNameAsString(), column);
+          this.admin.enableTable(tableName);
+        } catch (IOException e) {
+          throw new HBaseRestException("unable to modify column "
+              + column.getNameAsString(), e);
+        }
+      } else {
+        try {
+          this.admin.disableTable(tableName);
+          this.admin.addColumn(tableName, column);
+          this.admin.enableTable(tableName);
+        } catch (IOException e) {
+          throw new HBaseRestException("unable to add column "
+              + column.getNameAsString(), e);
+        }
+      }
+    }
+
+    return true;
+
+  }
+
+  /**
+   * Get table metadata.
+   * 
+   * @param request
+   * @param response
+   * @param tableName
+   * @throws IOException
+   */
+  public HTableDescriptor getTableMetadata(final String tableName)
+      throws HBaseRestException {
+    HTableDescriptor descriptor = null;
+    try {
+      HTableDescriptor[] tables = this.admin.listTables();
+      for (int i = 0; i < tables.length; i++) {
+        if (Bytes.toString(tables[i].getName()).equals(tableName)) {
+          descriptor = tables[i];
+          break;
+        }
+      }
+      if (descriptor == null) {
+
+      } else {
+        return descriptor;
+      }
+    } catch (IOException e) {
+      throw new HBaseRestException("error processing request.");
+    }
+    return descriptor;
+  }
+
+  /**
+   * Return region offsets.
+   * 
+   * @param request
+   * @param response
+   */
+  public Regions getTableRegions(final String tableName)
+      throws HBaseRestException {
+    try {
+      HTable table = new HTable(this.conf, tableName);
+      // Presumption is that this.table has already been focused on target
+      // table.
+      Regions regions = new Regions(table.getStartKeys());
+      // Presumption is that this.table has already been set against target
+      // table
+      return regions;
+    } catch (IOException e) {
+      throw new HBaseRestException("Unable to get regions from table");
+    }
+  }
+
+  // Post Methods
+  /**
+   * Creates table tableName described by the json in input.
+   * 
+   * @param tableName
+   *          table name
+   * @param htd
+   *          HBaseTableDescriptor for the table to be created
+   * 
+   * @return true if operation does not fail due to a table with the given
+   *         tableName not existing.
+   * @throws org.apache.hadoop.hbase.rest.exception.HBaseRestException
+   */
+  public boolean post(byte[] tableName, HTableDescriptor htd)
+      throws HBaseRestException {
+    try {
+      if (!this.admin.tableExists(tableName)) {
+        this.admin.createTable(htd);
+        return true;
+      }
+    } catch (IOException e) {
+      throw new HBaseRestException(e);
+    }
+    return false;
+  }
+
+  /**
+   * Deletes table tableName
+   * 
+   * @param tableName
+   *          name of the table.
+   * @return true if table exists and deleted, false if table does not exist.
+   * @throws org.apache.hadoop.hbase.rest.exception.HBaseRestException
+   */
+  public boolean delete(byte[] tableName) throws HBaseRestException {
+    try {
+      if (this.admin.tableExists(tableName)) {
+        this.admin.disableTable(tableName);
+        this.admin.deleteTable(tableName);
+        return true;
+      }
+      return false;
+    } catch (Exception e) {
+      throw new HBaseRestException(e);
+    }
+  }
+
+  public class Regions implements ISerializable {
+    byte[][] regionKey;
+
+    public Regions(byte[][] bs) {
+      super();
+      this.regionKey = bs;
+    }
+
+    @SuppressWarnings("unused")
+    private Regions() {
+    }
+
+    /**
+     * @return the regionKey
+     */
+    @TOJSON(fieldName = "region")
+    public byte[][] getRegionKey() {
+      return regionKey;
+    }
+
+    /**
+     * @param regionKey
+     *          the regionKey to set
+     */
+    public void setRegionKey(byte[][] regionKey) {
+      this.regionKey = regionKey;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see org.apache.hadoop.hbase.rest.xml.IOutputXML#toXML()
+     */
+    public void restSerialize(IRestSerializer serializer)
+        throws HBaseRestException {
+      serializer.serializeRegionData(this);
+    }
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TimestampController.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TimestampController.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TimestampController.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TimestampController.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,139 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+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;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public class TimestampController extends AbstractController {
+
+  @SuppressWarnings("unused")
+  private Log LOG = LogFactory.getLog(TimestampController.class);
+
+  protected TimestampModel getModel() {
+    return (TimestampModel) model;
+  }
+
+  @Override
+  protected AbstractModel generateModel(
+      @SuppressWarnings("hiding") HBaseConfiguration conf, HBaseAdmin admin) {
+    return new TimestampModel(conf, admin);
+  }
+
+  @Override
+  public void get(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap) throws HBaseRestException {
+    TimestampModel innerModel = getModel();
+
+    byte[] tableName;
+    byte[] rowName;
+    long timestamp;
+
+    tableName = pathSegments[0];
+    rowName = pathSegments[2];
+    timestamp = Bytes.toLong(pathSegments[3]);
+
+    if (queryMap.size() == 0) {
+      s.setOK(innerModel.get(tableName, rowName, timestamp));
+    } else {
+      // get the column names if any were passed in
+      String[] column_params = queryMap.get(RESTConstants.COLUMN);
+      byte[][] columns = null;
+
+      if (column_params != null && column_params.length > 0) {
+        List<String> available_columns = new ArrayList<String>();
+        for (String column_param : column_params) {
+          available_columns.add(column_param);
+        }
+        columns = Bytes.toByteArrays(available_columns.toArray(new String[0]));
+      }
+      s.setOK(innerModel.get(tableName, rowName, columns, timestamp));
+    }
+    s.respond();
+  }
+
+  @Override
+  public void post(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap, byte[] input, IHBaseRestParser parser)
+      throws HBaseRestException {
+    TimestampModel innerModel = getModel();
+
+    byte[] tableName;
+    byte[] rowName;
+    byte[] columnName;
+    long timestamp;
+
+    tableName = pathSegments[0];
+    rowName = pathSegments[1];
+    columnName = pathSegments[2];
+    timestamp = Bytes.toLong(pathSegments[3]);
+
+    try {
+      if (queryMap.size() == 0) {
+        innerModel.post(tableName, rowName, columnName, timestamp, input);
+        s.setOK();
+      } else {
+        s.setUnsupportedMediaType("Unknown Query.");
+      }
+    } 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 {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public void delete(Status s, byte[][] pathSegments,
+      Map<String, String[]> queryMap) throws HBaseRestException {
+    TimestampModel innerModel = getModel();
+
+    byte[] tableName;
+    byte[] rowName;
+    long timestamp;
+
+    tableName = pathSegments[0];
+    rowName = pathSegments[2];
+    timestamp = Bytes.toLong(pathSegments[3]);
+
+    if (queryMap.size() == 0) {
+      innerModel.delete(tableName, rowName, timestamp);
+    } else {
+      innerModel.delete(tableName, rowName, this
+          .getColumnsFromQueryMap(queryMap), timestamp);
+    }
+    s.setAccepted();
+    s.respond();
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TimestampModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TimestampModel.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TimestampModel.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TimestampModel.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,126 @@
+/**
+ * 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.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+
+public class TimestampModel extends AbstractModel {
+
+  @SuppressWarnings("unused")
+  private Log LOG = LogFactory.getLog(TimestampModel.class);
+
+  public TimestampModel(HBaseConfiguration conf, HBaseAdmin admin) {
+    super.initialize(conf, admin);
+  }
+
+  public void delete(byte[] tableName, byte[] rowName, long timestamp)
+      throws HBaseRestException {
+    try {
+      HTable table = new HTable(tableName);
+      table.deleteAll(rowName, timestamp);
+    } catch (IOException e) {
+      throw new HBaseRestException(e);
+    }
+  }
+
+  public void delete(byte[] tableName, byte[] rowName, byte[][] columns,
+      long timestamp) throws HBaseRestException {
+    try {
+      HTable table = new HTable(tableName);
+      for (byte[] column : columns) {
+        table.deleteAll(rowName, column, timestamp);
+      }
+    } catch (IOException e) {
+      throw new HBaseRestException(e);
+    }
+  }
+
+  public Cell get(byte[] tableName, byte[] rowName, byte[] columnName,
+      long timestamp) throws HBaseRestException {
+    try {
+      HTable table = new HTable(tableName);
+      return table.get(rowName, columnName, timestamp, 1)[0];
+    } catch (IOException e) {
+      throw new HBaseRestException(e);
+    }
+  }
+
+  public Cell[] get(byte[] tableName, byte[] rowName, byte[] columnName,
+      long timestamp, int numVersions) throws HBaseRestException {
+    try {
+      HTable table = new HTable(tableName);
+      return table.get(rowName, columnName, timestamp, numVersions);
+    } catch (IOException e) {
+      throw new HBaseRestException(e);
+    }
+  }
+
+  public RowResult get(byte[] tableName, byte[] rowName, byte[][] columns,
+      long timestamp) throws HBaseRestException {
+    try {
+      HTable table = new HTable(tableName);
+      return table.getRow(rowName, columns, timestamp);
+    } catch (IOException e) {
+      throw new HBaseRestException(e);
+    }
+  }
+
+  /**
+   * @param tableName
+   * @param rowName
+   * @param timestamp
+   * @return
+   * @throws HBaseRestException
+   */
+  public RowResult get(byte[] tableName, byte[] rowName, long timestamp)
+      throws HBaseRestException {
+    try {
+      HTable table = new HTable(tableName);
+      return table.getRow(rowName, timestamp);
+    } catch (IOException e) {
+      throw new HBaseRestException(e);
+    }
+  }
+
+  public void post(byte[] tableName, byte[] rowName, byte[] columnName,
+      long timestamp, byte[] value) throws HBaseRestException {
+    try {
+      HTable table;
+      BatchUpdate b;
+
+      table = new HTable(tableName);
+      b = new BatchUpdate(rowName, timestamp);
+
+      b.put(columnName, value);
+      table.commit(b);
+    } catch (IOException e) {
+      throw new HBaseRestException(e);
+    }
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/RestCell.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/RestCell.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/RestCell.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/RestCell.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,103 @@
+/**
+ * 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.descriptors;
+
+import org.apache.hadoop.hbase.io.Cell;
+
+import agilejson.TOJSON;
+
+/**
+ * 
+ */
+public class RestCell extends Cell {
+
+  byte[] name;
+  
+  
+
+  /**
+   * 
+   */
+  public RestCell() {
+    super();
+    // TODO Auto-generated constructor stub
+  }
+  
+  /**
+   * 
+   */
+  public RestCell(byte[] name, Cell cell) {
+    super(cell.getValue(), cell.getTimestamp());
+    this.name = name;
+  }
+
+  /**
+   * @param value
+   * @param timestamp
+   */
+  public RestCell(byte[] value, long timestamp) {
+    super(value, timestamp);
+    // TODO Auto-generated constructor stub
+  }
+
+  /**
+   * @param vals
+   * @param ts
+   */
+  public RestCell(byte[][] vals, long[] ts) {
+    super(vals, ts);
+    // TODO Auto-generated constructor stub
+  }
+
+  /**
+   * @param value
+   * @param timestamp
+   */
+  public RestCell(String value, long timestamp) {
+    super(value, timestamp);
+    // TODO Auto-generated constructor stub
+  }
+
+  /**
+   * @param vals
+   * @param ts
+   */
+  public RestCell(String[] vals, long[] ts) {
+    super(vals, ts);
+    // TODO Auto-generated constructor stub
+  }
+
+  /**
+   * @return the name
+   */
+  @TOJSON(base64=true)
+  public byte[] getName() {
+    return name;
+  }
+
+  /**
+   * @param name the name to set
+   */
+  public void setName(byte[] name) {
+    this.name = name;
+  }
+  
+  
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/RowUpdateDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/RowUpdateDescriptor.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/RowUpdateDescriptor.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/RowUpdateDescriptor.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,74 @@
+/**
+ * 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.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ *
+ */
+public class RowUpdateDescriptor {
+  private String tableName;
+  private String rowName;
+  private Map<byte[], byte[]> colvals = new HashMap<byte[], byte[]>();
+  
+  public RowUpdateDescriptor(String tableName, String rowName) {
+    this.tableName = tableName;
+    this.rowName = rowName;
+  }
+  
+  public RowUpdateDescriptor() {}
+
+  /**
+   * @return the tableName
+   */
+  public String getTableName() {
+    return tableName;
+  }
+
+  /**
+   * @param tableName the tableName to set
+   */
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  /**
+   * @return the rowName
+   */
+  public String getRowName() {
+    return rowName;
+  }
+
+  /**
+   * @param rowName the rowName to set
+   */
+  public void setRowName(String rowName) {
+    this.rowName = rowName;
+  }
+
+  /**
+   * @return the test
+   */
+  public Map<byte[], byte[]> getColVals() {
+    return colvals;
+  }  
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/ScannerDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/ScannerDescriptor.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/ScannerDescriptor.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/ScannerDescriptor.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,130 @@
+/**
+ * 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.descriptors;
+
+/**
+ * 
+ */
+public class ScannerDescriptor {
+  byte[][] columns;
+  long timestamp;
+  byte[] startRow;
+  byte[] stopRow;
+  String filters;
+
+  /**
+   * @param columns
+   * @param timestamp
+   * @param startRow
+   * @param stopRow
+   * @param filters
+   */
+  public ScannerDescriptor(byte[][] columns, long timestamp, byte[] startRow,
+      byte[] stopRow, String filters) {
+    super();
+    this.columns = columns;
+    this.timestamp = timestamp;
+    this.startRow = startRow;
+    this.stopRow = stopRow;
+    this.filters = filters;
+    
+    if(this.startRow == null) {
+      this.startRow = new byte[0];
+    }
+    if(this.stopRow == null) {
+      this.stopRow = new byte[0];
+    }
+  }
+
+  /**
+   * @return the columns
+   */
+  public byte[][] getColumns() {
+    return columns;
+  }
+
+  /**
+   * @param columns
+   *          the columns to set
+   */
+  public void setColumns(byte[][] columns) {
+    this.columns = columns;
+  }
+
+  /**
+   * @return the timestamp
+   */
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  /**
+   * @param timestamp
+   *          the timestamp to set
+   */
+  public void setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
+  }
+
+  /**
+   * @return the startRow
+   */
+  public byte[] getStartRow() {
+    return startRow;
+  }
+
+  /**
+   * @param startRow
+   *          the startRow to set
+   */
+  public void setStartRow(byte[] startRow) {
+    this.startRow = startRow;
+  }
+
+  /**
+   * @return the stopRow
+   */
+  public byte[] getStopRow() {
+    return stopRow;
+  }
+
+  /**
+   * @param stopRow
+   *          the stopRow to set
+   */
+  public void setStopRow(byte[] stopRow) {
+    this.stopRow = stopRow;
+  }
+
+  /**
+   * @return the filters
+   */
+  public String getFilters() {
+    return filters;
+  }
+
+  /**
+   * @param filters
+   *          the filters to set
+   */
+  public void setFilters(String filters) {
+    this.filters = filters;
+  }
+}
\ No newline at end of file

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/ScannerIdentifier.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/ScannerIdentifier.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/ScannerIdentifier.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/ScannerIdentifier.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,96 @@
+/**
+ * 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.descriptors;
+
+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 ScannerIdentifier implements ISerializable {
+  Integer id;
+  Long numRows;
+
+  /**
+   * @param id
+   */
+  public ScannerIdentifier(Integer id) {
+    super();
+    this.id = id;
+  }
+
+  /**
+   * @param id
+   * @param numRows
+   */
+  public ScannerIdentifier(Integer id, Long numRows) {
+    super();
+    this.id = id;
+    this.numRows = numRows;
+  }
+
+  /**
+   * @return the id
+   */
+  @TOJSON
+  public Integer getId() {
+    return id;
+  }
+
+  /**
+   * @param id
+   *          the id to set
+   */
+  public void setId(Integer id) {
+    this.id = id;
+  }
+
+  /**
+   * @return the numRows
+   */
+  public Long getNumRows() {
+    return numRows;
+  }
+
+  /**
+   * @param numRows
+   *          the numRows to set
+   */
+  public void setNumRows(Long numRows) {
+    this.numRows = numRows;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.hbase.rest.xml.IOutputXML#toXML(org.apache.hadoop.hbase
+   * .rest.serializer.IRestSerializer)
+   */
+  public void restSerialize(IRestSerializer serializer)
+      throws HBaseRestException {
+    serializer.serializeScannerIdentifier(this);
+  }
+
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/TimestampsDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/TimestampsDescriptor.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/TimestampsDescriptor.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/descriptors/TimestampsDescriptor.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,67 @@
+/**
+ * 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.descriptors;
+
+import java.util.HashMap;
+import java.util.Map;
+
+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;
+
+/**
+ * 
+ */
+public class TimestampsDescriptor implements ISerializable {
+  Map<Long, String> timestamps = new HashMap<Long, String>();
+
+  public void add(long timestamp, byte[] tableName, byte[] rowName) {
+    StringBuilder sb = new StringBuilder();
+    sb.append('/');
+    sb.append(Bytes.toString(tableName));
+    sb.append("/row/");
+    sb.append(Bytes.toString(rowName));
+    sb.append('/');
+    sb.append(timestamp);
+
+    timestamps.put(timestamp, sb.toString());
+  }
+
+  /**
+   * @return the timestamps
+   */
+  public Map<Long, String> getTimestamps() {
+    return timestamps;
+  }
+
+  /*
+   * (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.serializeTimestamps(this);
+  }
+
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/exception/HBaseRestException.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/exception/HBaseRestException.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/exception/HBaseRestException.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/exception/HBaseRestException.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,86 @@
+/**
+ * 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.exception;
+
+import agilejson.TOJSON;
+
+public class HBaseRestException extends Exception {
+
+  /**
+   * 
+   */
+  private static final long serialVersionUID = 8481585437124298646L;
+  private Exception innerException;
+  private String innerClass;
+  private String innerMessage;
+
+  public HBaseRestException() {
+
+  }
+
+  public HBaseRestException(Exception e) throws HBaseRestException {
+    if (HBaseRestException.class.isAssignableFrom(e.getClass())) {
+      throw ((HBaseRestException) e);
+    }
+    setInnerException(e);
+    innerClass = e.getClass().toString();
+    innerMessage = e.getMessage();
+  }
+
+  /**
+   * @param message
+   */
+  public HBaseRestException(String message) {
+    super(message);
+    innerMessage = message;
+  }
+
+  public HBaseRestException(String message, Exception exception) {
+    super(message, exception);
+    setInnerException(exception);
+    innerClass = exception.getClass().toString();
+    innerMessage = message;
+  }
+
+  @TOJSON
+  public String getInnerClass() {
+    return this.innerClass;
+  }
+
+  @TOJSON
+  public String getInnerMessage() {
+    return this.innerMessage;
+  }
+
+  /**
+   * @param innerException
+   *          the innerException to set
+   */
+  public void setInnerException(Exception innerException) {
+    this.innerException = innerException;
+  }
+
+  /**
+   * @return the innerException
+   */
+  public Exception getInnerException() {
+    return innerException;
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/ColumnValueFilterFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/ColumnValueFilterFactory.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/ColumnValueFilterFactory.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/ColumnValueFilterFactory.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,66 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.hbase.filter.ColumnValueFilter;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+/**
+ * FilterFactory that constructs a ColumnValueFilter from a JSON arg String.
+ * Expects a Stringified JSON argument with the following form:
+ * 
+ * { "column_name" : "MY_COLUMN_NAME", "compare_op" : "INSERT_COMPARE_OP_HERE",
+ * "value" : "MY_COMPARE_VALUE" }
+ * 
+ * The current valid compare ops are: equal, greater, greater_or_equal, less,
+ * less_or_equal, not_equal
+ */
+public class ColumnValueFilterFactory implements FilterFactory {
+
+  public RowFilterInterface getFilterFromJSON(String args)
+      throws HBaseRestException {
+    JSONObject innerJSON;
+    String columnName;
+    String compareOp;
+    String value;
+
+    try {
+      innerJSON = new JSONObject(args);
+    } catch (JSONException e) {
+      throw new HBaseRestException(e);
+    }
+
+    if ((columnName = innerJSON.optString(COLUMN_NAME)) == null) {
+      throw new MalformedFilterException();
+    }
+    if ((compareOp = innerJSON.optString(COMPARE_OP)) == null) {
+      throw new MalformedFilterException();
+    }
+    if ((value = innerJSON.optString(VALUE)) == null) {
+      throw new MalformedFilterException();
+    }
+
+    return new ColumnValueFilter(columnName.getBytes(),
+        ColumnValueFilter.CompareOp.valueOf(compareOp), value.getBytes());
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/FilterFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/FilterFactory.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/FilterFactory.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/FilterFactory.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,71 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+
+/**
+ * Constructs Filters from JSON. Filters are defined
+ * as JSON Objects of the form:
+ * {
+ *  "type" : "FILTER_CLASS_NAME",
+ * "args" : "FILTER_ARGUMENTS"
+ * }
+ * 
+ * For Filters like WhileMatchRowFilter,
+ * nested Filters are supported. Just serialize a different
+ * filter in the form (for instance if you wanted to use WhileMatchRowFilter
+ * with a StopRowFilter:
+ * 
+ * {
+ *  "type" : "WhileMatchRowFilter",
+ * "args" : {
+ *              "type" : "StopRowFilter",
+ *              "args" : "ROW_KEY_TO_STOP_ON"
+ *            }
+ * }
+ * 
+ * For filters like RowSetFilter, nested Filters AND Filter arrays
+ * are supported. So for instance If one wanted to do a RegExp
+ * RowFilter UNIONed with a WhileMatchRowFilter(StopRowFilter),
+ * you would look like this:
+ * 
+ * {
+ *   "type" : "RowFilterSet",
+ *   "args" : [
+ *                {
+ *                  "type" : "RegExpRowFilter",
+ *                  "args" : "MY_REGULAR_EXPRESSION"
+ *                },
+ *                {
+ *                  "type" : "WhileMatchRowFilter"
+ *                  "args" : {
+ *                                "type" : "StopRowFilter"
+ *                                "args" : "MY_STOP_ROW_EXPRESSION"
+ *                             }
+ *                }
+ *              ]
+ * }
+ */
+public interface FilterFactory extends FilterFactoryConstants {
+  public RowFilterInterface getFilterFromJSON(String args)
+      throws HBaseRestException;
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/FilterFactoryConstants.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/FilterFactoryConstants.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/FilterFactoryConstants.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/FilterFactoryConstants.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,41 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+
+public interface FilterFactoryConstants {
+  static String TYPE = "type";
+  static String ARGUMENTS = "args";
+  static String COLUMN_NAME = "column_name";
+  static String COMPARE_OP = "compare_op";
+  static String VALUE = "value";
+
+  static class MalformedFilterException extends HBaseRestException {
+
+    public MalformedFilterException() {
+    }
+
+    @Override
+    public String toString() {
+      return "malformed filter expression";
+    }
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/InclusiveStopRowFilterFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/InclusiveStopRowFilterFactory.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/InclusiveStopRowFilterFactory.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/InclusiveStopRowFilterFactory.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,37 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.hbase.filter.InclusiveStopRowFilter;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * FilterFactory that construct a InclusiveStopRowFilter
+ * from a JSON argument String.
+ * 
+ * It expects that the whole input string consists of only
+ * the rowKey that you wish to stop on.
+ */
+public class InclusiveStopRowFilterFactory implements FilterFactory {
+  public RowFilterInterface getFilterFromJSON(String args) {
+    return new InclusiveStopRowFilter(Bytes.toBytes(args));
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/PageRowFilterFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/PageRowFilterFactory.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/PageRowFilterFactory.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/PageRowFilterFactory.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,34 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.hbase.filter.PageRowFilter;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+
+/**
+ * Constructs a PageRowFilter from a JSON argument String.
+ * Expects the entire JSON argument string to consist
+ * of the long that is the length of the pages that you want.
+ */
+public class PageRowFilterFactory implements FilterFactory {
+  public RowFilterInterface getFilterFromJSON(String args) {
+    return new PageRowFilter(Long.parseLong(args));
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/RegExpRowFilterFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/RegExpRowFilterFactory.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/RegExpRowFilterFactory.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/RegExpRowFilterFactory.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,34 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.hbase.filter.RegExpRowFilter;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+
+/**
+ * Constructs a RegExpRowFilter from a JSON argument string.
+ * Expects the entire JSON arg string to consist of the
+ * entire regular expression to be used.
+ */
+public class RegExpRowFilterFactory implements FilterFactory {
+  public RowFilterInterface getFilterFromJSON(String args) {
+    return new RegExpRowFilter(args);
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/RowFilterSetFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/RowFilterSetFactory.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/RowFilterSetFactory.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/RowFilterSetFactory.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,115 @@
+/**
+ * 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.filter;
+
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+import org.apache.hadoop.hbase.filter.RowFilterSet;
+import org.apache.hadoop.hbase.rest.RESTConstants;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.rest.filter.FilterFactoryConstants.MalformedFilterException;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+/**
+ * Constructs a RowFilterSet from a JSON argument String.
+ * 
+ * Assumes that the input is a JSONArray consisting of JSON Object version of
+ * the filters that you wish to mash together in an AND statement.
+ * 
+ * The Syntax for the individual inner filters are defined by their respective
+ * FilterFactory. If a filter factory for said Factory does not exist, a
+ * MalformedFilterJSONException will be thrown.
+ * 
+ * Currently OR Statements are not supported even though at a later iteration
+ * they could be supported easily.
+ */
+public class RowFilterSetFactory implements FilterFactory {
+
+  public RowFilterInterface getFilterFromJSON(String args)
+      throws HBaseRestException {
+    JSONArray filterArray;
+    Set<RowFilterInterface> set;
+    JSONObject filter;
+
+    try {
+      filterArray = new JSONArray(args);
+    } catch (JSONException e) {
+      throw new HBaseRestException(e);
+    }
+
+    // If only 1 Row, just return the row.
+    if (filterArray.length() == 1) {
+      return getRowFilter(filterArray.optJSONObject(0));
+    }
+
+    // Otherwise continue
+    set = new HashSet<RowFilterInterface>();
+
+    for (int i = 0; i < filterArray.length(); i++) {
+
+      // Get FIlter Object
+      if ((filter = filterArray.optJSONObject(i)) == null) {
+        throw new MalformedFilterException();
+      }
+
+      // Add newly constructed filter to the filter set;
+      set.add(getRowFilter(filter));
+    }
+
+    // Put set into a RowFilterSet and return.
+    return new RowFilterSet(set);
+  }
+
+  /**
+   * A refactored method that encapsulates the creation of a RowFilter given a
+   * JSONObject with a correct form of: { "type" : "MY_TYPE", "args" : MY_ARGS,
+   * }
+   * 
+   * @param filter
+   * @return
+   * @throws org.apache.hadoop.hbase.rest.exception.HBaseRestException
+   */
+  protected RowFilterInterface getRowFilter(JSONObject filter)
+      throws HBaseRestException {
+    FilterFactory f;
+    String filterType;
+    String filterArgs;
+
+    // Get Filter's Type
+    if ((filterType = filter.optString(FilterFactoryConstants.TYPE)) == null) {
+      throw new MalformedFilterException();
+    }
+
+    // Get Filter Args
+    if ((filterArgs = filter.optString(FilterFactoryConstants.ARGUMENTS)) == null) {
+      throw new MalformedFilterException();
+    }
+
+    // Get Filter Factory for given Filter Type
+    if ((f = RESTConstants.filterFactories.get(filterType)) == null) {
+      throw new MalformedFilterException();
+    }
+
+    return f.getFilterFromJSON(filterArgs);
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/StopRowFilterFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/StopRowFilterFactory.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/StopRowFilterFactory.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/StopRowFilterFactory.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,37 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+import org.apache.hadoop.hbase.filter.StopRowFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * FilterFactory that construct a StopRowFilter
+ * from an Argument String.
+ * 
+ * It expects that the whole input string consists of only
+ * the rowKey that you wish to stop on.
+ */
+public class StopRowFilterFactory implements FilterFactory {
+  public RowFilterInterface getFilterFromJSON(String args) {
+    return new StopRowFilter(Bytes.toBytes(args));
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/WhileMatchRowFilterFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/WhileMatchRowFilterFactory.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/WhileMatchRowFilterFactory.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/filter/WhileMatchRowFilterFactory.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,61 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+import org.apache.hadoop.hbase.filter.WhileMatchRowFilter;
+import org.apache.hadoop.hbase.rest.RESTConstants;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+/**
+ * Factory to produce WhileMatchRowFilters from JSON
+ * Expects as an arguement a valid JSON Object in 
+ * String form of another RowFilterInterface.
+ */
+public class WhileMatchRowFilterFactory implements FilterFactory {
+  public RowFilterInterface getFilterFromJSON(String args)
+      throws HBaseRestException {
+    JSONObject innerFilterJSON;
+    FilterFactory f;
+    String innerFilterType;
+    String innerFilterArgs;
+
+    try {
+      innerFilterJSON = new JSONObject(args);
+    } catch (JSONException e) {
+      throw new HBaseRestException(e);
+    }
+
+    // Check if filter is correct
+    if ((innerFilterType = innerFilterJSON.optString(TYPE)) == null)
+      throw new MalformedFilterException();
+    if ((innerFilterArgs = innerFilterJSON.optString(ARGUMENTS)) == null)
+      throw new MalformedFilterException();
+
+    if ((f = RESTConstants.filterFactories.get(innerFilterType)) == null)
+      throw new MalformedFilterException();
+
+    RowFilterInterface innerFilter = f.getFilterFromJSON(innerFilterArgs);
+
+    return new WhileMatchRowFilter(innerFilter);
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/HBaseRestParserFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/HBaseRestParserFactory.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/HBaseRestParserFactory.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/HBaseRestParserFactory.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,56 @@
+/**
+ * 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.parser;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.rest.Dispatcher.ContentType;
+
+/**
+ * 
+ */
+public class HBaseRestParserFactory {
+
+  private static final Map<ContentType, Class<?>> parserMap = 
+    new HashMap<ContentType, Class<?>>();
+
+  static {
+    parserMap.put(ContentType.XML, XMLRestParser.class);
+    parserMap.put(ContentType.JSON, JsonRestParser.class);
+  }
+
+  public static IHBaseRestParser getParser(ContentType ct) {
+    IHBaseRestParser parser = null;
+
+    Class<?> clazz = parserMap.get(ct);
+    try {
+      parser = (IHBaseRestParser) clazz.newInstance();
+    } catch (InstantiationException e) {
+      // TODO Auto-generated catch block
+      e.printStackTrace();
+    } catch (IllegalAccessException e) {
+      // TODO Auto-generated catch block
+      e.printStackTrace();
+    }
+
+    return parser;
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/IHBaseRestParser.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/IHBaseRestParser.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/IHBaseRestParser.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/IHBaseRestParser.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,52 @@
+/**
+ * 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.parser;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.rest.descriptors.RowUpdateDescriptor;
+import org.apache.hadoop.hbase.rest.descriptors.ScannerDescriptor;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+
+/**
+ * 
+ */
+public interface IHBaseRestParser {
+  /**
+   * Parses a HTableDescriptor given the input array.
+   * 
+   * @param input
+   * @return
+   * @throws HBaseRestException
+   */
+  public HTableDescriptor getTableDescriptor(byte[] input)
+      throws HBaseRestException;
+
+  public ArrayList<HColumnDescriptor> getColumnDescriptors(byte[] input)
+      throws HBaseRestException;
+
+  public ScannerDescriptor getScannerDescriptor(byte[] input)
+      throws HBaseRestException;
+
+  public RowUpdateDescriptor getRowUpdateDescriptor(byte[] input,
+      byte[][] pathSegments) throws HBaseRestException;
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/JsonRestParser.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/JsonRestParser.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/JsonRestParser.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/JsonRestParser.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,235 @@
+/**
+ * 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.parser;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.rest.RESTConstants;
+import org.apache.hadoop.hbase.rest.descriptors.RowUpdateDescriptor;
+import org.apache.hadoop.hbase.rest.descriptors.ScannerDescriptor;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+/**
+ * 
+ */
+public class JsonRestParser implements IHBaseRestParser {
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.hbase.rest.parser.IHBaseRestParser#getTableDescriptor
+   * (byte[])
+   */
+  public HTableDescriptor getTableDescriptor(byte[] input)
+      throws HBaseRestException {
+    try {
+      JSONObject o;
+      HTableDescriptor h;
+      JSONArray columnDescriptorArray;
+      o = new JSONObject(new String(input));
+      columnDescriptorArray = o.getJSONArray("column_families");
+      h = new HTableDescriptor(o.getString("name"));
+
+      for (int i = 0; i < columnDescriptorArray.length(); i++) {
+        JSONObject json_columnDescriptor = columnDescriptorArray
+            .getJSONObject(i);
+        h.addFamily(this.getColumnDescriptor(json_columnDescriptor));
+      }
+      return h;
+    } catch (Exception e) {
+      throw new HBaseRestException(e);
+    }
+  }
+
+  private HColumnDescriptor getColumnDescriptor(JSONObject jsonObject)
+      throws JSONException {
+    String strTemp;
+    strTemp = jsonObject.getString("name");
+    if (strTemp.charAt(strTemp.length() - 1) != ':') {
+      strTemp += ":";
+    }
+
+    byte[] name = Bytes.toBytes(strTemp);
+
+    int maxVersions;
+    HColumnDescriptor.CompressionType cType;
+    boolean inMemory;
+    boolean blockCacheEnabled;
+    int maxValueLength;
+    int timeToLive;
+    boolean bloomfilter;
+
+    try {
+      bloomfilter = jsonObject.getBoolean("bloomfilter");
+    } catch (JSONException e) {
+      bloomfilter = false;
+    }
+
+    try {
+      maxVersions = jsonObject.getInt("max_versions");
+    } catch (JSONException e) {
+      maxVersions = 3;
+    }
+
+    try {
+      cType = HColumnDescriptor.CompressionType.valueOf(jsonObject
+          .getString("compression_type"));
+    } catch (JSONException e) {
+      cType = HColumnDescriptor.CompressionType.NONE;
+    }
+
+    try {
+      inMemory = jsonObject.getBoolean("in_memory");
+    } catch (JSONException e) {
+      inMemory = false;
+    }
+
+    try {
+      blockCacheEnabled = jsonObject.getBoolean("block_cache_enabled");
+    } catch (JSONException e) {
+      blockCacheEnabled = false;
+    }
+
+    try {
+      maxValueLength = jsonObject.getInt("max_value_length");
+    } catch (JSONException e) {
+      maxValueLength = 2147483647;
+    }
+
+    try {
+      timeToLive = jsonObject.getInt("time_to_live");
+    } catch (JSONException e) {
+      timeToLive = Integer.MAX_VALUE;
+    }
+
+    return new HColumnDescriptor(name, maxVersions, cType, inMemory,
+        blockCacheEnabled, maxValueLength, timeToLive, bloomfilter);
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.hbase.rest.parser.IHBaseRestParser#getColumnDescriptors
+   * (byte[])
+   */
+  public ArrayList<HColumnDescriptor> getColumnDescriptors(byte[] input)
+      throws HBaseRestException {
+    ArrayList<HColumnDescriptor> columns = new ArrayList<HColumnDescriptor>();
+    try {
+      JSONObject o;
+      JSONArray columnDescriptorArray;
+      o = new JSONObject(new String(input));
+      columnDescriptorArray = o.getJSONArray("column_families");
+
+      for (int i = 0; i < columnDescriptorArray.length(); i++) {
+        JSONObject json_columnDescriptor = columnDescriptorArray
+            .getJSONObject(i);
+        columns.add(this.getColumnDescriptor(json_columnDescriptor));
+      }
+    } catch (JSONException e) {
+      throw new HBaseRestException("Error Parsing json input", e);
+    }
+
+    return columns;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.hbase.rest.parser.IHBaseRestParser#getScannerDescriptor
+   * (byte[])
+   */
+  public ScannerDescriptor getScannerDescriptor(byte[] input)
+      throws HBaseRestException {
+    JSONObject scannerDescriptor;
+    JSONArray columnArray;
+
+    byte[][] columns = null;
+    long timestamp;
+    byte[] startRow;
+    byte[] stopRow;
+    String filters;
+
+    try {
+      scannerDescriptor = new JSONObject(new String(input));
+
+      columnArray = scannerDescriptor.optJSONArray(RESTConstants.COLUMNS);
+      timestamp = scannerDescriptor.optLong(RESTConstants.SCANNER_TIMESTAMP);
+      startRow = Bytes.toBytes(scannerDescriptor.optString(
+          RESTConstants.SCANNER_START_ROW, ""));
+      stopRow = Bytes.toBytes(scannerDescriptor.optString(
+          RESTConstants.SCANNER_STOP_ROW, ""));
+      filters = scannerDescriptor.optString(RESTConstants.SCANNER_FILTER);
+
+      if (columnArray != null) {
+        columns = new byte[columnArray.length()][];
+        for (int i = 0; i < columnArray.length(); i++) {
+          columns[i] = Bytes.toBytes(columnArray.optString(i));
+        }
+      }
+
+      return new ScannerDescriptor(columns, timestamp, startRow, stopRow,
+          filters);
+    } catch (JSONException e) {
+      throw new HBaseRestException("error parsing json string", e);
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.hbase.rest.parser.IHBaseRestParser#getRowUpdateDescriptor
+   * (byte[], byte[][])
+   */
+  public RowUpdateDescriptor getRowUpdateDescriptor(byte[] input,
+      byte[][] pathSegments) throws HBaseRestException {
+
+    RowUpdateDescriptor rud = new RowUpdateDescriptor();
+    JSONArray a;
+
+    rud.setTableName(Bytes.toString(pathSegments[0]));
+    rud.setRowName(Bytes.toString(pathSegments[2]));
+
+    try {
+      JSONObject updateObject = new JSONObject(new String(input));
+      a = updateObject.getJSONArray(RESTConstants.COLUMNS);
+      for (int i = 0; i < a.length(); i++) {
+        rud.getColVals().put(
+            Bytes.toBytes(a.getJSONObject(i).getString(RESTConstants.NAME)),
+            org.apache.hadoop.hbase.util.Base64.decode(a.getJSONObject(i)
+                .getString(RESTConstants.VALUE)));
+      }
+    } catch (JSONException e) {
+      throw new HBaseRestException("Error parsing row update json", e);
+    }
+    return rud;
+  }
+
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/XMLRestParser.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/XMLRestParser.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/XMLRestParser.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/parser/XMLRestParser.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,291 @@
+/**
+ * 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.parser;
+
+import java.io.ByteArrayInputStream;
+import java.util.ArrayList;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
+import org.apache.hadoop.hbase.rest.RESTConstants;
+import org.apache.hadoop.hbase.rest.descriptors.RowUpdateDescriptor;
+import org.apache.hadoop.hbase.rest.descriptors.ScannerDescriptor;
+import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
+import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+/**
+ * 
+ */
+public class XMLRestParser implements IHBaseRestParser {
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.hbase.rest.parser.IHBaseRestParser#getTableDescriptor
+   * (byte[])
+   */
+  public HTableDescriptor getTableDescriptor(byte[] input)
+      throws HBaseRestException {
+    DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory
+        .newInstance();
+    docBuilderFactory.setIgnoringComments(true);
+
+    DocumentBuilder builder = null;
+    Document doc = null;
+    HTableDescriptor htd = null;
+
+    try {
+      builder = docBuilderFactory.newDocumentBuilder();
+      ByteArrayInputStream is = new ByteArrayInputStream(input);
+      doc = builder.parse(is);
+    } catch (Exception e) {
+      throw new HBaseRestException(e);
+    }
+
+    try {
+      Node name_node = doc.getElementsByTagName("name").item(0);
+      String table_name = name_node.getFirstChild().getNodeValue();
+
+      htd = new HTableDescriptor(table_name);
+      NodeList columnfamily_nodes = doc.getElementsByTagName("columnfamily");
+      for (int i = 0; i < columnfamily_nodes.getLength(); i++) {
+        Element columnfamily = (Element) columnfamily_nodes.item(i);
+        htd.addFamily(this.getColumnDescriptor(columnfamily));
+      }
+    } catch (Exception e) {
+      throw new HBaseRestException(e);
+    }
+    return htd;
+  }
+
+  public HColumnDescriptor getColumnDescriptor(Element columnfamily) {
+    return this.getColumnDescriptor(columnfamily, null);
+  }
+
+  private HColumnDescriptor getColumnDescriptor(Element columnfamily,
+      HTableDescriptor currentTDesp) {
+    Node name_node = columnfamily.getElementsByTagName("name").item(0);
+    String colname = makeColumnName(name_node.getFirstChild().getNodeValue());
+
+    int max_versions = HColumnDescriptor.DEFAULT_VERSIONS;
+    CompressionType compression = HColumnDescriptor.DEFAULT_COMPRESSION;
+    boolean in_memory = HColumnDescriptor.DEFAULT_IN_MEMORY;
+    boolean block_cache = HColumnDescriptor.DEFAULT_BLOCKCACHE;
+    int max_cell_size = HColumnDescriptor.DEFAULT_LENGTH;
+    int ttl = HColumnDescriptor.DEFAULT_TTL;
+    boolean bloomfilter = HColumnDescriptor.DEFAULT_BLOOMFILTER;
+
+    if (currentTDesp != null) {
+      HColumnDescriptor currentCDesp = currentTDesp.getFamily(Bytes
+          .toBytes(colname));
+      if (currentCDesp != null) {
+        max_versions = currentCDesp.getMaxVersions();
+        // compression = currentCDesp.getCompression();
+        in_memory = currentCDesp.isInMemory();
+        block_cache = currentCDesp.isBlockCacheEnabled();
+        max_cell_size = currentCDesp.getMaxValueLength();
+        ttl = currentCDesp.getTimeToLive();
+        bloomfilter = currentCDesp.isBloomfilter();
+      }
+    }
+
+    NodeList max_versions_list = columnfamily
+        .getElementsByTagName("max-versions");
+    if (max_versions_list.getLength() > 0) {
+      max_versions = Integer.parseInt(max_versions_list.item(0).getFirstChild()
+          .getNodeValue());
+    }
+
+    NodeList compression_list = columnfamily
+        .getElementsByTagName("compression");
+    if (compression_list.getLength() > 0) {
+      compression = CompressionType.valueOf(compression_list.item(0)
+          .getFirstChild().getNodeValue());
+    }
+
+    NodeList in_memory_list = columnfamily.getElementsByTagName("in-memory");
+    if (in_memory_list.getLength() > 0) {
+      in_memory = Boolean.valueOf(in_memory_list.item(0).getFirstChild()
+          .getNodeValue());
+    }
+
+    NodeList block_cache_list = columnfamily
+        .getElementsByTagName("block-cache");
+    if (block_cache_list.getLength() > 0) {
+      block_cache = Boolean.valueOf(block_cache_list.item(0).getFirstChild()
+          .getNodeValue());
+    }
+
+    NodeList max_cell_size_list = columnfamily
+        .getElementsByTagName("max-cell-size");
+    if (max_cell_size_list.getLength() > 0) {
+      max_cell_size = Integer.valueOf(max_cell_size_list.item(0)
+          .getFirstChild().getNodeValue());
+    }
+
+    NodeList ttl_list = columnfamily.getElementsByTagName("time-to-live");
+    if (ttl_list.getLength() > 0) {
+      ttl = Integer.valueOf(ttl_list.item(0).getFirstChild().getNodeValue());
+    }
+
+    NodeList bloomfilter_list = columnfamily
+        .getElementsByTagName("bloomfilter");
+    if (bloomfilter_list.getLength() > 0) {
+      bloomfilter = Boolean.valueOf(bloomfilter_list.item(0).getFirstChild()
+          .getNodeValue());
+    }
+
+    HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(colname),
+        max_versions, compression, in_memory, block_cache, max_cell_size, ttl,
+        bloomfilter);
+
+    NodeList metadataList = columnfamily.getElementsByTagName("metadata");
+    for (int i = 0; i < metadataList.getLength(); i++) {
+      Element metadataColumn = (Element) metadataList.item(i);
+      // extract the name and value children
+      Node mname_node = metadataColumn.getElementsByTagName("name").item(0);
+      String mname = mname_node.getFirstChild().getNodeValue();
+      Node mvalue_node = metadataColumn.getElementsByTagName("value").item(0);
+      String mvalue = mvalue_node.getFirstChild().getNodeValue();
+      hcd.setValue(mname, mvalue);
+    }
+
+    return hcd;
+  }
+
+  protected String makeColumnName(String column) {
+    String returnColumn = column;
+    if (column.indexOf(':') == -1)
+      returnColumn += ':';
+    return returnColumn;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.hbase.rest.parser.IHBaseRestParser#getColumnDescriptors
+   * (byte[])
+   */
+  public ArrayList<HColumnDescriptor> getColumnDescriptors(byte[] input)
+      throws HBaseRestException {
+    DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory
+        .newInstance();
+    docBuilderFactory.setIgnoringComments(true);
+
+    DocumentBuilder builder = null;
+    Document doc = null;
+    ArrayList<HColumnDescriptor> columns = new ArrayList<HColumnDescriptor>();
+
+    try {
+      builder = docBuilderFactory.newDocumentBuilder();
+      ByteArrayInputStream is = new ByteArrayInputStream(input);
+      doc = builder.parse(is);
+    } catch (Exception e) {
+      throw new HBaseRestException(e);
+    }
+
+    NodeList columnfamily_nodes = doc.getElementsByTagName("columnfamily");
+    for (int i = 0; i < columnfamily_nodes.getLength(); i++) {
+      Element columnfamily = (Element) columnfamily_nodes.item(i);
+      columns.add(this.getColumnDescriptor(columnfamily));
+    }
+
+    return columns;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.hbase.rest.parser.IHBaseRestParser#getScannerDescriptor
+   * (byte[])
+   */
+  public ScannerDescriptor getScannerDescriptor(byte[] input)
+      throws HBaseRestException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.hbase.rest.parser.IHBaseRestParser#getRowUpdateDescriptor
+   * (byte[], byte[][])
+   */
+  public RowUpdateDescriptor getRowUpdateDescriptor(byte[] input,
+      byte[][] pathSegments) throws HBaseRestException {
+    RowUpdateDescriptor rud = new RowUpdateDescriptor();
+
+    rud.setTableName(Bytes.toString(pathSegments[0]));
+    rud.setRowName(Bytes.toString(pathSegments[2]));
+
+    DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory
+        .newInstance();
+    docBuilderFactory.setIgnoringComments(true);
+
+    DocumentBuilder builder = null;
+    Document doc = null;
+
+    try {
+      builder = docBuilderFactory.newDocumentBuilder();
+      ByteArrayInputStream is = new ByteArrayInputStream(input);
+      doc = builder.parse(is);
+    } catch (Exception e) {
+      throw new HBaseRestException(e.getMessage(), e);
+    }
+
+    NodeList cell_nodes = doc.getElementsByTagName(RESTConstants.COLUMN);
+    System.out.println("cell_nodes.length: " + cell_nodes.getLength());
+    for (int i = 0; i < cell_nodes.getLength(); i++) {
+      String columnName = null;
+      byte[] value = null;
+
+      Element cell = (Element) cell_nodes.item(i);
+
+      NodeList item = cell.getElementsByTagName(RESTConstants.NAME);
+      if (item.getLength() > 0) {
+        columnName = item.item(0).getFirstChild().getNodeValue();
+      }
+
+      NodeList item1 = cell.getElementsByTagName(RESTConstants.VALUE);
+      if (item1.getLength() > 0) {
+        value = org.apache.hadoop.hbase.util.Base64.decode(item1
+            .item(0).getFirstChild().getNodeValue());
+      }
+
+      if (columnName != null && value != null) {
+        rud.getColVals().put(columnName.getBytes(), value);
+      }
+    }
+    return rud;
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/serializer/AbstractRestSerializer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/serializer/AbstractRestSerializer.java?rev=736503&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/serializer/AbstractRestSerializer.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/serializer/AbstractRestSerializer.java Wed Jan 21 16:21:16 2009
@@ -0,0 +1,58 @@
+/**
+ * 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.serializer;
+
+import javax.servlet.http.HttpServletResponse;
+
+/**
+ * 
+ *         Abstract object that is used as the base of all serializers in the
+ *         REST based interface.
+ */
+public abstract class AbstractRestSerializer implements IRestSerializer {
+
+  // keep the response object to write back to the stream
+  protected final HttpServletResponse response;
+  // Used to denote if pretty printing of the output should be used
+  protected final boolean prettyPrint;
+
+  /**
+   * marking the default constructor as private so it will never be used.
+   */
+  @SuppressWarnings("unused")
+  private AbstractRestSerializer() {
+    response = null;
+    prettyPrint = false;
+  }
+
+  /**
+   * Public constructor for AbstractRestSerializer. This is the constructor that
+   * should be called whenever creating a RestSerializer object.
+   * 
+   * @param response
+   */
+  public AbstractRestSerializer(HttpServletResponse response,
+      boolean prettyPrint) {
+    super();
+    this.response = response;
+    this.prettyPrint = prettyPrint;
+  }
+
+}