You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2010/05/18 20:49:14 UTC

svn commit: r945816 [3/11] - in /hadoop/hbase/trunk: ./ contrib/ contrib/stargate/ core/ core/src/main/java/org/apache/hadoop/hbase/rest/ core/src/main/java/org/apache/hadoop/hbase/rest/client/ core/src/main/java/org/apache/hadoop/hbase/rest/metrics/ c...

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java?rev=945816&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java Tue May 18 18:49:12 2010
@@ -0,0 +1,237 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlAnyAttribute;
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.namespace.QName;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+
+/**
+ * Representation of a column family schema.
+ * 
+ * <pre>
+ * &lt;complexType name="ColumnSchema"&gt;
+ *   &lt;attribute name="name" type="string"&gt;&lt;/attribute&gt;
+ *   &lt;anyAttribute&gt;&lt;/anyAttribute&gt;
+ * &lt;/complexType&gt;
+ * </pre>
+ */
+@XmlRootElement(name="ColumnSchema")
+public class ColumnSchemaModel implements Serializable {
+  private static final long serialVersionUID = 1L;
+  private static QName BLOCKCACHE = new QName(HColumnDescriptor.BLOCKCACHE);
+  private static QName BLOCKSIZE = new QName(HColumnDescriptor.BLOCKSIZE);
+  private static QName BLOOMFILTER = new QName(HColumnDescriptor.BLOOMFILTER);
+  private static QName COMPRESSION = new QName(HColumnDescriptor.COMPRESSION);
+  private static QName IN_MEMORY = new QName(HConstants.IN_MEMORY);
+  private static QName TTL = new QName(HColumnDescriptor.TTL);
+  private static QName VERSIONS = new QName(HConstants.VERSIONS);
+
+  private String name;
+  private Map<QName,Object> attrs = new HashMap<QName,Object>();
+
+  /**
+   * Default constructor
+   */
+  public ColumnSchemaModel() {}
+
+  /**
+   * Add an attribute to the column family schema
+   * @param name the attribute name
+   * @param value the attribute value
+   */
+  public void addAttribute(String name, Object value) {
+    attrs.put(new QName(name), value);
+  }
+
+  /**
+   * @param name the attribute name
+   * @return the attribute value
+   */
+  public String getAttribute(String name) {
+    Object o = attrs.get(new QName(name));
+    return o != null ? o.toString(): null;
+  }
+
+  /**
+   * @return the column name
+   */
+  @XmlAttribute
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * @return the map for holding unspecified (user) attributes
+   */
+  @XmlAnyAttribute
+  public Map<QName,Object> getAny() {
+    return attrs;
+  }
+
+  /**
+   * @param name the table name
+   */
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("{ NAME => '");
+    sb.append(name);
+    sb.append('\'');
+    for (Map.Entry<QName,Object> e: attrs.entrySet()) {
+      sb.append(", ");
+      sb.append(e.getKey().getLocalPart());
+      sb.append(" => '");
+      sb.append(e.getValue().toString());
+      sb.append('\'');
+    }
+    sb.append(" }");
+    return sb.toString();
+  }
+
+  // getters and setters for common schema attributes
+
+  // cannot be standard bean type getters and setters, otherwise this would
+  // confuse JAXB
+
+  /**
+   * @return true if the BLOCKCACHE attribute is present and true
+   */
+  public boolean __getBlockcache() {
+    Object o = attrs.get(BLOCKCACHE);
+    return o != null ? 
+      Boolean.valueOf(o.toString()) : HColumnDescriptor.DEFAULT_BLOCKCACHE;
+  }
+
+  /**
+   * @return the value of the BLOCKSIZE attribute or its default if it is unset
+   */
+  public int __getBlocksize() {
+    Object o = attrs.get(BLOCKSIZE);
+    return o != null ? 
+      Integer.valueOf(o.toString()) : HColumnDescriptor.DEFAULT_BLOCKSIZE;
+  }
+
+  /**
+   * @return true if the BLOOMFILTER attribute is present and true
+   */
+  public boolean __getBloomfilter() {
+    Object o = attrs.get(BLOOMFILTER);
+    return o != null ? 
+      Boolean.valueOf(o.toString()) : HColumnDescriptor.DEFAULT_BLOOMFILTER;
+  }
+
+  /**
+   * @return the value of the COMPRESSION attribute or its default if it is unset
+   */
+  public String __getCompression() {
+    Object o = attrs.get(COMPRESSION);
+    return o != null ? o.toString() : HColumnDescriptor.DEFAULT_COMPRESSION;
+  }
+
+  /**
+   * @return true if the IN_MEMORY attribute is present and true
+   */
+  public boolean __getInMemory() {
+    Object o = attrs.get(IN_MEMORY);
+    return o != null ? 
+      Boolean.valueOf(o.toString()) : HColumnDescriptor.DEFAULT_IN_MEMORY;
+  }
+
+  /**
+   * @return the value of the TTL attribute or its default if it is unset
+   */
+  public int __getTTL() {
+    Object o = attrs.get(TTL);
+    return o != null ? 
+      Integer.valueOf(o.toString()) : HColumnDescriptor.DEFAULT_TTL;
+  }
+
+  /**
+   * @return the value of the VERSIONS attribute or its default if it is unset
+   */
+  public int __getVersions() {
+    Object o = attrs.get(VERSIONS);
+    return o != null ? 
+      Integer.valueOf(o.toString()) : HColumnDescriptor.DEFAULT_VERSIONS;
+  }
+
+  /**
+   * @param value the desired value of the BLOCKSIZE attribute
+   */
+  public void __setBlocksize(int value) {
+    attrs.put(BLOCKSIZE, Integer.toString(value));
+  }
+
+  /**
+   * @param value the desired value of the BLOCKCACHE attribute
+   */
+  public void __setBlockcache(boolean value) {
+    attrs.put(BLOCKCACHE, Boolean.toString(value));
+  }
+
+  public void __setBloomfilter(boolean value) {
+    attrs.put(BLOOMFILTER, Boolean.toString(value));
+  }
+
+  /**
+   * @param value the desired value of the COMPRESSION attribute
+   */
+  public void __setCompression(String value) {
+    attrs.put(COMPRESSION, value); 
+  }
+
+  /**
+   * @param value the desired value of the IN_MEMORY attribute
+   */
+  public void __setInMemory(boolean value) {
+    attrs.put(IN_MEMORY, Boolean.toString(value));
+  }
+
+  /**
+   * @param value the desired value of the TTL attribute
+   */
+  public void __setTTL(int value) {
+    attrs.put(TTL, Integer.toString(value));
+  }
+
+  /**
+   * @param value the desired value of the VERSIONS attribute
+   */
+  public void __setVersions(int value) {
+    attrs.put(VERSIONS, Integer.toString(value));
+  }
+}

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/RowModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/RowModel.java?rev=945816&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/RowModel.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/RowModel.java Tue May 18 18:49:12 2010
@@ -0,0 +1,142 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
+
+/**
+ * Representation of a row. A row is a related set of cells, grouped by common
+ * row key. RowModels do not appear in results by themselves. They are always
+ * encapsulated within CellSetModels.
+ * 
+ * <pre>
+ * &lt;complexType name="Row"&gt;
+ *   &lt;sequence&gt;
+ *     &lt;element name="key" type="base64Binary"&gt;&lt;/element&gt;
+ *     &lt;element name="cell" type="tns:Cell" 
+ *       maxOccurs="unbounded" minOccurs="1"&gt;&lt;/element&gt;
+ *   &lt;/sequence&gt;
+ * &lt;/complexType&gt;
+ * </pre>
+ */
+@XmlRootElement(name="Row")
+public class RowModel implements ProtobufMessageHandler, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private byte[] key;
+  private List<CellModel> cells = new ArrayList<CellModel>();
+
+  /**
+   * Default constructor
+   */
+  public RowModel() { }
+
+  /**
+   * Constructor
+   * @param key the row key
+   */
+  public RowModel(final String key) {
+    this(key.getBytes());
+  }
+  
+  /**
+   * Constructor
+   * @param key the row key
+   */
+  public RowModel(final byte[] key) {
+    this.key = key;
+    cells = new ArrayList<CellModel>();
+  }
+
+  /**
+   * Constructor
+   * @param key the row key
+   * @param cells the cells
+   */
+  public RowModel(final String key, final List<CellModel> cells) {
+    this(key.getBytes(), cells);
+  }
+  
+  /**
+   * Constructor
+   * @param key the row key
+   * @param cells the cells
+   */
+  public RowModel(final byte[] key, final List<CellModel> cells) {
+    this.key = key;
+    this.cells = cells;
+  }
+  
+  /**
+   * Adds a cell to the list of cells for this row
+   * @param cell the cell
+   */
+  public void addCell(CellModel cell) {
+    cells.add(cell);
+  }
+
+  /**
+   * @return the row key
+   */
+  @XmlAttribute
+  public byte[] getKey() {
+    return key;
+  }
+
+  /**
+   * @param key the row key
+   */
+  public void setKey(byte[] key) {
+    this.key = key;
+  }
+
+  /**
+   * @return the cells
+   */
+  @XmlElement(name="Cell")
+  public List<CellModel> getCells() {
+    return cells;
+  }
+
+  @Override
+  public byte[] createProtobufOutput() {
+    // there is no standalone row protobuf message
+    throw new UnsupportedOperationException(
+        "no protobuf equivalent to RowModel");
+  }
+
+  @Override
+  public ProtobufMessageHandler getObjectFromMessage(byte[] message)
+      throws IOException {
+    // there is no standalone row protobuf message
+    throw new UnsupportedOperationException(
+        "no protobuf equivalent to RowModel");
+  }
+}

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java?rev=945816&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java Tue May 18 18:49:12 2010
@@ -0,0 +1,628 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
+import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
+import org.apache.hadoop.hbase.filter.PageFilter;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.hbase.filter.RegexStringComparator;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.filter.SkipFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.filter.ValueFilter;
+import org.apache.hadoop.hbase.filter.WhileMatchFilter;
+import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
+import org.apache.hadoop.hbase.rest.protobuf.generated.ScannerMessage.Scanner;
+import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.protobuf.ByteString;
+
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.api.json.JSONJAXBContext;
+import com.sun.jersey.api.json.JSONMarshaller;
+import com.sun.jersey.api.json.JSONUnmarshaller;
+
+/**
+ * A representation of Scanner parameters.
+ * 
+ * <pre>
+ * &lt;complexType name="Scanner"&gt;
+ *   &lt;sequence>
+ *     &lt;element name="column" type="base64Binary" minOccurs="0" maxOccurs="unbounded"/&gt;
+ *   &lt;/sequence&gt;
+ *   &lt;element name="filter" type="string" minOccurs="0" maxOccurs="1"&gt;&lt;/element&gt;
+ *   &lt;attribute name="startRow" type="base64Binary"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="endRow" type="base64Binary"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="batch" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="startTime" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="endTime" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="maxVersions" type="int"&gt;&lt;/attribute&gt;
+ * &lt;/complexType&gt;
+ * </pre>
+ */
+@XmlRootElement(name="Scanner")
+public class ScannerModel implements ProtobufMessageHandler, Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  private byte[] startRow = HConstants.EMPTY_START_ROW;
+  private byte[] endRow = HConstants.EMPTY_END_ROW;;
+  private List<byte[]> columns = new ArrayList<byte[]>();
+  private int batch = Integer.MAX_VALUE;
+  private long startTime = 0;
+  private long endTime = Long.MAX_VALUE;
+  private String filter = null;
+  private int maxVersions = Integer.MAX_VALUE;
+
+  @XmlRootElement
+  static class FilterModel {
+    
+    @XmlRootElement
+    static class WritableByteArrayComparableModel {
+      @XmlAttribute public String type;
+      @XmlAttribute public String value;
+
+      static enum ComparatorType {
+        BinaryComparator,
+        BinaryPrefixComparator,
+        RegexStringComparator,
+        SubstringComparator    
+      }
+
+      public WritableByteArrayComparableModel() { }
+
+      public WritableByteArrayComparableModel(
+          WritableByteArrayComparable comparator) {
+        String typeName = comparator.getClass().getSimpleName();
+        ComparatorType type = ComparatorType.valueOf(typeName);
+        this.type = typeName;
+        switch (type) {
+          case BinaryComparator:
+          case BinaryPrefixComparator:
+            this.value = Base64.encodeBytes(comparator.getValue());
+            break;
+          case RegexStringComparator:
+          case SubstringComparator:
+            this.value = Bytes.toString(comparator.getValue());
+            break;
+          default:
+            throw new RuntimeException("unhandled filter type: " + type);
+        }
+      }
+
+      public WritableByteArrayComparable build() {
+        WritableByteArrayComparable comparator;
+        switch (ComparatorType.valueOf(type)) {
+          case BinaryComparator: {
+            comparator = new BinaryComparator(Base64.decode(value));
+          } break;
+          case BinaryPrefixComparator: {
+            comparator = new BinaryPrefixComparator(Base64.decode(value));
+          } break;
+          case RegexStringComparator: {
+            comparator = new RegexStringComparator(value);
+          } break;
+          case SubstringComparator: {
+            comparator = new SubstringComparator(value);
+          } break;
+          default: {
+            throw new RuntimeException("unhandled comparator type: " + type);
+          }
+        }
+        return comparator;
+      }
+
+    }
+
+    // a grab bag of fields, would have been a union if this were C
+    @XmlAttribute public String type = null;
+    @XmlAttribute public String op = null;
+    @XmlElement WritableByteArrayComparableModel comparator = null;
+    @XmlAttribute public String value = null;
+    @XmlElement public List<FilterModel> filters = null;
+    @XmlAttribute public Integer limit = null;
+    @XmlAttribute public String family = null;
+    @XmlAttribute public String qualifier = null;
+    @XmlAttribute public Boolean ifMissing = null;
+    @XmlAttribute public Boolean latestVersion = null;
+
+    static enum FilterType {
+      ColumnCountGetFilter,
+      FilterList,
+      FirstKeyOnlyFilter,
+      InclusiveStopFilter,
+      PageFilter,
+      PrefixFilter,
+      QualifierFilter,
+      RowFilter,
+      SingleColumnValueFilter,
+      SkipFilter,
+      ValueFilter,
+      WhileMatchFilter    
+    }
+
+    public FilterModel() { }
+    
+    public FilterModel(Filter filter) { 
+      String typeName = filter.getClass().getSimpleName();
+      FilterType type = FilterType.valueOf(typeName);
+      this.type = typeName;
+      switch (type) {
+        case ColumnCountGetFilter:
+          this.limit = ((ColumnCountGetFilter)filter).getLimit();
+          break;
+        case FilterList:
+          this.op = ((FilterList)filter).getOperator().toString();
+          this.filters = new ArrayList<FilterModel>();
+          for (Filter child: ((FilterList)filter).getFilters()) {
+            this.filters.add(new FilterModel(child));
+          }
+          break;
+        case FirstKeyOnlyFilter:
+          break;
+        case InclusiveStopFilter:
+          this.value = 
+            Base64.encodeBytes(((InclusiveStopFilter)filter).getStopRowKey());
+          break;
+        case PageFilter:
+          this.value = Long.toString(((PageFilter)filter).getPageSize());
+          break;
+        case PrefixFilter:
+          this.value = Base64.encodeBytes(((PrefixFilter)filter).getPrefix());
+          break;
+        case QualifierFilter:
+        case RowFilter:
+        case ValueFilter:
+          this.op = ((CompareFilter)filter).getOperator().toString();
+          this.comparator = 
+            new WritableByteArrayComparableModel(
+              ((CompareFilter)filter).getComparator());
+          break;
+        case SingleColumnValueFilter: {
+          SingleColumnValueFilter scvf = (SingleColumnValueFilter) filter;
+          this.family = Base64.encodeBytes(scvf.getFamily());
+          byte[] qualifier = scvf.getQualifier();
+          if (qualifier != null) {
+            this.qualifier = Base64.encodeBytes(qualifier);
+          }
+          this.op = scvf.getOperator().toString();
+          this.comparator = 
+            new WritableByteArrayComparableModel(scvf.getComparator());
+          if (scvf.getFilterIfMissing()) {
+            this.ifMissing = true;
+          }
+          if (scvf.getLatestVersionOnly()) {
+            this.latestVersion = true;
+          }
+        } break;
+        case SkipFilter:
+          this.filters = new ArrayList<FilterModel>();
+          this.filters.add(new FilterModel(((SkipFilter)filter).getFilter()));
+          break;
+        case WhileMatchFilter:
+          this.filters = new ArrayList<FilterModel>();
+          this.filters.add(
+            new FilterModel(((WhileMatchFilter)filter).getFilter()));
+          break;
+        default:
+          throw new RuntimeException("unhandled filter type " + type);
+      }
+    }
+
+    public Filter build() {
+      Filter filter;
+      switch (FilterType.valueOf(type)) {
+      case ColumnCountGetFilter: {
+        filter = new ColumnCountGetFilter(limit);
+      } break;
+      case FilterList: {
+        List<Filter> list = new ArrayList<Filter>();
+        for (FilterModel model: filters) {
+          list.add(model.build());
+        }
+        filter = new FilterList(FilterList.Operator.valueOf(op), list);
+      } break;
+      case FirstKeyOnlyFilter: {
+        filter = new FirstKeyOnlyFilter();
+      } break;
+      case InclusiveStopFilter: {
+        filter = new InclusiveStopFilter(Base64.decode(value));
+      } break;
+      case PageFilter: {
+        filter = new PageFilter(Long.valueOf(value));
+      } break;
+      case PrefixFilter: {
+        filter = new PrefixFilter(Base64.decode(value));
+      } break;
+      case QualifierFilter: {
+        filter = new QualifierFilter(CompareOp.valueOf(op), comparator.build());
+      } break;
+      case RowFilter: {
+        filter = new RowFilter(CompareOp.valueOf(op), comparator.build());
+      } break;
+      case SingleColumnValueFilter: {
+        filter = new SingleColumnValueFilter(Base64.decode(family),
+          qualifier != null ? Base64.decode(qualifier) : null,
+          CompareOp.valueOf(op), comparator.build());
+        if (ifMissing != null) {
+          ((SingleColumnValueFilter)filter).setFilterIfMissing(ifMissing);
+        }
+        if (latestVersion != null) {
+          ((SingleColumnValueFilter)filter).setLatestVersionOnly(latestVersion);
+        }
+      } break;
+      case SkipFilter: {
+        filter = new SkipFilter(filters.get(0).build());
+      } break;
+      case ValueFilter: {
+        filter = new ValueFilter(CompareOp.valueOf(op), comparator.build());
+      } break;
+      case WhileMatchFilter: {
+        filter = new WhileMatchFilter(filters.get(0).build());
+      } break;
+      default:
+        throw new RuntimeException("unhandled filter type: " + type);
+      }
+      return filter;
+    }
+
+  }
+
+  /**
+   * @param s the JSON representation of the filter
+   * @return the filter
+   * @throws Exception
+   */
+  public static Filter buildFilter(String s) throws Exception {
+    JSONJAXBContext context =
+      new JSONJAXBContext(JSONConfiguration.natural().build(),
+        FilterModel.class);
+    JSONUnmarshaller unmarshaller = context.createJSONUnmarshaller();
+    FilterModel model = unmarshaller.unmarshalFromJSON(new StringReader(s),
+      FilterModel.class);
+    return model.build();
+  }
+
+  /**
+   * @param filter the filter
+   * @return the JSON representation of the filter
+   * @throws Exception 
+   */
+  public static String stringifyFilter(final Filter filter) throws Exception {
+    JSONJAXBContext context =
+      new JSONJAXBContext(JSONConfiguration.natural().build(),
+        FilterModel.class);
+    JSONMarshaller marshaller = context.createJSONMarshaller();
+    StringWriter writer = new StringWriter();
+    marshaller.marshallToJSON(new FilterModel(filter), writer);
+    return writer.toString();
+  }
+
+  /**
+   * @param scan the scan specification
+   * @throws Exception 
+   */
+  public static ScannerModel fromScan(Scan scan) throws Exception {
+    ScannerModel model = new ScannerModel();
+    model.setStartRow(scan.getStartRow());
+    model.setEndRow(scan.getStopRow());
+    byte[][] families = scan.getFamilies();
+    if (families != null) {
+      for (byte[] column: families) {
+        model.addColumn(column);
+      }
+    }
+    model.setStartTime(scan.getTimeRange().getMin());
+    model.setEndTime(scan.getTimeRange().getMax());
+    int caching = scan.getCaching();
+    if (caching > 0) {
+      model.setBatch(caching);
+    }
+    int maxVersions = scan.getMaxVersions();
+    if (maxVersions > 0) {
+      model.setMaxVersions(maxVersions);
+    }
+    Filter filter = scan.getFilter();
+    if (filter != null) {
+      model.setFilter(stringifyFilter(filter));
+    }
+    return model;
+  }
+
+  /**
+   * Default constructor
+   */
+  public ScannerModel() {}
+
+  /**
+   * Constructor
+   * @param startRow the start key of the row-range
+   * @param endRow the end key of the row-range
+   * @param columns the columns to scan
+   * @param batch the number of values to return in batch
+   * @param endTime the upper bound on timestamps of values of interest
+   * @param maxVersions the maximum number of versions to return
+   * @param filter a filter specification
+   * (values with timestamps later than this are excluded)
+   */
+  public ScannerModel(byte[] startRow, byte[] endRow, List<byte[]> columns,
+      int batch, long endTime, int maxVersions, String filter) {
+    super();
+    this.startRow = startRow;
+    this.endRow = endRow;
+    this.columns = columns;
+    this.batch = batch;
+    this.endTime = endTime;
+    this.maxVersions = maxVersions;
+    this.filter = filter;
+  }
+
+  /**
+   * Constructor 
+   * @param startRow the start key of the row-range
+   * @param endRow the end key of the row-range
+   * @param columns the columns to scan
+   * @param batch the number of values to return in batch
+   * @param startTime the lower bound on timestamps of values of interest
+   * (values with timestamps earlier than this are excluded)
+   * @param endTime the upper bound on timestamps of values of interest
+   * (values with timestamps later than this are excluded)
+   * @param filter a filter specification
+   */
+  public ScannerModel(byte[] startRow, byte[] endRow, List<byte[]> columns,
+      int batch, long startTime, long endTime, String filter) {
+    super();
+    this.startRow = startRow;
+    this.endRow = endRow;
+    this.columns = columns;
+    this.batch = batch;
+    this.startTime = startTime;
+    this.endTime = endTime;
+    this.filter = filter;
+  }
+
+  /**
+   * Add a column to the column set
+   * @param column the column name, as &lt;column&gt;(:&lt;qualifier&gt;)?
+   */
+  public void addColumn(byte[] column) {
+    columns.add(column);
+  }
+
+  /**
+   * @return true if a start row was specified
+   */
+  public boolean hasStartRow() {
+    return !Bytes.equals(startRow, HConstants.EMPTY_START_ROW);
+  }
+
+  /**
+   * @return start row
+   */
+  @XmlAttribute
+  public byte[] getStartRow() {
+    return startRow;
+  }
+
+  /**
+   * @return true if an end row was specified
+   */
+  public boolean hasEndRow() {
+    return !Bytes.equals(endRow, HConstants.EMPTY_END_ROW);
+  }
+
+  /**
+   * @return end row
+   */
+  @XmlAttribute
+  public byte[] getEndRow() {
+    return endRow;
+  }
+
+  /**
+   * @return list of columns of interest in column:qualifier format, or empty for all
+   */
+  @XmlElement(name="column")
+  public List<byte[]> getColumns() {
+    return columns;
+  }
+  
+  /**
+   * @return the number of cells to return in batch
+   */
+  @XmlAttribute
+  public int getBatch() {
+    return batch;
+  }
+
+  /**
+   * @return the lower bound on timestamps of items of interest
+   */
+  @XmlAttribute
+  public long getStartTime() {
+    return startTime;
+  }
+
+  /**
+   * @return the upper bound on timestamps of items of interest
+   */
+  @XmlAttribute
+  public long getEndTime() {
+    return endTime;
+  }
+
+  /**
+   * @return maximum number of versions to return
+   */
+  @XmlAttribute
+  public int getMaxVersions() {
+    return maxVersions;
+  }
+
+  /**
+   * @return the filter specification
+   */
+  @XmlElement
+  public String getFilter() {
+    return filter;
+  }
+
+  /**
+   * @param startRow start row
+   */
+  public void setStartRow(byte[] startRow) {
+    this.startRow = startRow;
+  }
+
+  /**
+   * @param endRow end row
+   */
+  public void setEndRow(byte[] endRow) {
+    this.endRow = endRow;
+  }
+
+  /**
+   * @param columns list of columns of interest in column:qualifier format, or empty for all
+   */
+  public void setColumns(List<byte[]> columns) {
+    this.columns = columns;
+  }
+
+  /**
+   * @param batch the number of cells to return in batch
+   */
+  public void setBatch(int batch) {
+    this.batch = batch;
+  }
+
+  /**
+   * @param maxVersions maximum number of versions to return
+   */
+  public void setMaxVersions(int maxVersions) {
+    this.maxVersions = maxVersions;
+  }
+
+  /**
+   * @param startTime the lower bound on timestamps of values of interest
+   */
+  public void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }
+
+  /**
+   * @param endTime the upper bound on timestamps of values of interest
+   */
+  public void setEndTime(long endTime) {
+    this.endTime = endTime;
+  }
+
+  /**
+   * @param filter the filter specification
+   */
+  public void setFilter(String filter) {
+    this.filter = filter;
+  }
+
+  @Override
+  public byte[] createProtobufOutput() {
+    Scanner.Builder builder = Scanner.newBuilder();
+    if (!Bytes.equals(startRow, HConstants.EMPTY_START_ROW)) {
+      builder.setStartRow(ByteString.copyFrom(startRow));
+    }
+    if (!Bytes.equals(endRow, HConstants.EMPTY_START_ROW)) {
+      builder.setEndRow(ByteString.copyFrom(endRow));
+    }
+    for (byte[] column: columns) {
+      builder.addColumns(ByteString.copyFrom(column));
+    }
+    builder.setBatch(batch);
+    if (startTime != 0) {
+      builder.setStartTime(startTime);
+    }
+    if (endTime != 0) {
+      builder.setEndTime(endTime);
+    }
+    builder.setBatch(getBatch());
+    builder.setMaxVersions(maxVersions);
+    if (filter != null) {
+      builder.setFilter(filter);
+    }
+    return builder.build().toByteArray();
+  }
+
+  @Override
+  public ProtobufMessageHandler getObjectFromMessage(byte[] message)
+      throws IOException {
+    Scanner.Builder builder = Scanner.newBuilder();
+    builder.mergeFrom(message);
+    if (builder.hasStartRow()) {
+      startRow = builder.getStartRow().toByteArray();
+    }
+    if (builder.hasEndRow()) {
+      endRow = builder.getEndRow().toByteArray();
+    }
+    for (ByteString column: builder.getColumnsList()) {
+      addColumn(column.toByteArray());
+    }
+    if (builder.hasBatch()) {
+      batch = builder.getBatch();
+    }
+    if (builder.hasStartTime()) {
+      startTime = builder.getStartTime();
+    }
+    if (builder.hasEndTime()) {
+      endTime = builder.getEndTime();
+    }
+    if (builder.hasMaxVersions()) {
+      maxVersions = builder.getMaxVersions();
+    }
+    if (builder.hasFilter()) {
+      filter = builder.getFilter();
+    }
+    return this;
+  }
+
+}

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterStatusModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterStatusModel.java?rev=945816&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterStatusModel.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterStatusModel.java Tue May 18 18:49:12 2010
@@ -0,0 +1,620 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
+import org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.protobuf.ByteString;
+
+/**
+ * Representation of the status of a storage cluster:
+ * <p>
+ * <ul>
+ * <li>regions: the total number of regions served by the cluster</li>
+ * <li>requests: the total number of requests per second handled by the
+ * cluster in the last reporting interval</li>
+ * <li>averageLoad: the average load of the region servers in the cluster</li>
+ * <li>liveNodes: detailed status of the live region servers</li>
+ * <li>deadNodes: the names of region servers declared dead</li>
+ * </ul>
+ * 
+ * <pre>
+ * &lt;complexType name="StorageClusterStatus"&gt;
+ *   &lt;sequence&gt;
+ *     &lt;element name="liveNode" type="tns:Node"
+ *       maxOccurs="unbounded" minOccurs="0"&gt;
+ *     &lt;/element&gt;
+ *     &lt;element name="deadNode" type="string" maxOccurs="unbounded"
+ *       minOccurs="0"&gt;
+ *     &lt;/element&gt;
+ *   &lt;/sequence&gt;
+ *   &lt;attribute name="regions" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="requests" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="averageLoad" type="float"&gt;&lt;/attribute&gt;
+ * &lt;/complexType&gt;
+ *
+ * &lt;complexType name="Node"&gt;
+ *   &lt;sequence&gt;
+ *     &lt;element name="region" type="tns:Region" 
+ *       maxOccurs="unbounded" minOccurs="0"&gt;&lt;/element&gt;
+ *   &lt;/sequence&gt;
+ *   &lt;attribute name="name" type="string"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="startCode" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="requests" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="heapSizeMB" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="maxHeapSizeMB" type="int"&gt;&lt;/attribute&gt;
+ * &lt;/complexType&gt;
+ *
+ * &lt;complexType name="Region"&gt;
+ *   &lt;attribute name="name" type="base64Binary"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="stores" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="storefiles" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="storefileSizeMB" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="memstoreSizeMB" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="storefileIndexSizeMB" type="int"&gt;&lt;/attribute&gt;
+ * &lt;/complexType&gt;
+ * </pre>
+ */
+@XmlRootElement(name="ClusterStatus")
+public class StorageClusterStatusModel 
+    implements Serializable, ProtobufMessageHandler {
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * Represents a region server.
+	 */
+	public static class Node {
+	  
+	  /**
+	   * Represents a region hosted on a region server.
+	   */
+	  public static class Region {
+	    private byte[] name;
+	    private int stores;
+	    private int storefiles;
+	    private int storefileSizeMB;
+	    private int memstoreSizeMB;
+	    private int storefileIndexSizeMB;
+
+	    /**
+	     * Default constructor
+	     */
+	    public Region() {}
+
+	    /**
+	     * Constructor
+	     * @param name the region name
+	     */
+	    public Region(byte[] name) {
+	      this.name = name;
+	    }
+
+	    /**
+	     * Constructor
+	     * @param name the region name
+	     * @param stores the number of stores
+	     * @param storefiles the number of store files
+	     * @param storefileSizeMB total size of store files, in MB
+	     * @param memstoreSizeMB total size of memstore, in MB
+	     * @param storefileIndexSizeMB total size of store file indexes, in MB
+	     */
+	    public Region(byte[] name, int stores, int storefiles,
+          int storefileSizeMB, int memstoreSizeMB, int storefileIndexSizeMB) {
+        this.name = name;
+        this.stores = stores;
+        this.storefiles = storefiles;
+        this.storefileSizeMB = storefileSizeMB;
+        this.memstoreSizeMB = memstoreSizeMB;
+        this.storefileIndexSizeMB = storefileIndexSizeMB;
+      }
+
+      /**
+	     * @return the region name
+	     */
+	    @XmlAttribute
+	    public byte[] getName() {
+	      return name;
+	    }
+
+	    /**
+	     * @return the number of stores 
+	     */
+	    @XmlAttribute
+	    public int getStores() {
+        return stores;
+      }
+
+      /**
+       * @return the number of store files 
+       */
+      @XmlAttribute
+      public int getStorefiles() {
+        return storefiles;
+      }
+
+      /**
+       * @return the total size of store files, in MB
+       */
+      @XmlAttribute
+      public int getStorefileSizeMB() {
+        return storefileSizeMB;
+      }
+
+      /**
+       * @return memstore size, in MB
+       */
+      @XmlAttribute
+      public int getMemstoreSizeMB() {
+        return memstoreSizeMB;
+      }
+
+      /**
+       * @return the total size of store file indexes, in MB
+       */
+      @XmlAttribute
+      public int getStorefileIndexSizeMB() {
+        return storefileIndexSizeMB;
+      }
+
+      /**
+	     * @param name the region name
+	     */
+	    public void setName(byte[] name) {
+	      this.name = name;
+	    }
+
+	    /**
+	     * @param stores the number of stores
+	     */
+      public void setStores(int stores) {
+        this.stores = stores;
+      }
+
+      /**
+       * @param storefiles the number of store files
+       */
+      public void setStorefiles(int storefiles) {
+        this.storefiles = storefiles;
+      }
+
+      /**
+       * @param storefileSizeMB total size of store files, in MB
+       */
+      public void setStorefileSizeMB(int storefileSizeMB) {
+        this.storefileSizeMB = storefileSizeMB;
+      }
+
+      /**
+       * @param memstoreSizeMB memstore size, in MB
+       */
+      public void setMemstoreSizeMB(int memstoreSizeMB) {
+        this.memstoreSizeMB = memstoreSizeMB;
+      }
+
+      /**
+       * @param storefileIndexSizeMB total size of store file indexes, in MB
+       */
+      public void setStorefileIndexSizeMB(int storefileIndexSizeMB) {
+        this.storefileIndexSizeMB = storefileIndexSizeMB;
+      }
+	  }
+
+	  private String name;
+    private long startCode;
+    private int requests;
+    private int heapSizeMB;
+    private int maxHeapSizeMB;
+    private List<Region> regions = new ArrayList<Region>();
+
+    /**
+     * Add a region name to the list
+     * @param name the region name
+     */
+    public void addRegion(byte[] name, int stores, int storefiles,
+        int storefileSizeMB, int memstoreSizeMB, int storefileIndexSizeMB) {
+      regions.add(new Region(name, stores, storefiles, storefileSizeMB,
+        memstoreSizeMB, storefileIndexSizeMB));
+    }
+
+    /**
+     * @param index the index
+     * @return the region name
+     */
+    public Region getRegion(int index) {
+      return regions.get(index);
+    }
+
+    /**
+     * Default constructor
+     */
+    public Node() {}
+
+    /**
+     * Constructor
+     * @param name the region server name
+     * @param startCode the region server's start code
+     */
+    public Node(String name, long startCode) {
+      this.name = name;
+      this.startCode = startCode;
+    }
+
+    /**
+     * @return the region server's name
+     */
+    @XmlAttribute
+    public String getName() {
+      return name;
+    }
+
+    /**
+     * @return the region server's start code
+     */
+    @XmlAttribute
+    public long getStartCode() {
+      return startCode;
+    }
+
+    /**
+     * @return the current heap size, in MB
+     */
+    @XmlAttribute
+    public int getHeapSizeMB() {
+      return heapSizeMB;
+    }
+
+    /**
+     * @return the maximum heap size, in MB
+     */
+    @XmlAttribute
+    public int getMaxHeapSizeMB() {
+      return maxHeapSizeMB;
+    }
+
+    /**
+     * @return the list of regions served by the region server
+     */
+    @XmlElement(name="Region")
+    public List<Region> getRegions() {
+      return regions;
+    }
+
+    /**
+     * @return the number of requests per second processed by the region server
+     */
+    @XmlAttribute
+    public int getRequests() {
+      return requests;
+    }
+
+    /**
+     * @param name the region server's hostname
+     */
+    public void setName(String name) {
+      this.name = name;
+    }
+
+    /**
+     * @param startCode the region server's start code
+     */
+    public void setStartCode(long startCode) {
+      this.startCode = startCode;
+    }
+
+    /**
+     * @param heapSizeMB the current heap size, in MB
+     */
+    public void setHeapSizeMB(int heapSizeMB) {
+      this.heapSizeMB = heapSizeMB;
+    }
+
+    /**
+     * @param maxHeapSizeMB the maximum heap size, in MB
+     */
+    public void setMaxHeapSizeMB(int maxHeapSizeMB) {
+      this.maxHeapSizeMB = maxHeapSizeMB;
+    }
+
+    /**
+     * @param regions a list of regions served by the region server
+     */
+    public void setRegions(List<Region> regions) {
+      this.regions = regions;
+    }
+
+    /**
+     * @param requests the number of requests per second processed by the
+     * region server
+     */
+    public void setRequests(int requests) {
+      this.requests = requests;
+    }
+	}
+
+	private List<Node> liveNodes = new ArrayList<Node>();
+	private List<String> deadNodes = new ArrayList<String>();
+	private int regions;
+	private int requests;
+	private double averageLoad;
+
+	/**
+	 * Add a live node to the cluster representation.
+	 * @param name the region server name
+	 * @param startCode the region server's start code
+	 * @param heapSizeMB the current heap size, in MB
+	 * @param maxHeapSizeMB the maximum heap size, in MB
+	 */
+	public Node addLiveNode(String name, long startCode, int heapSizeMB,
+	    int maxHeapSizeMB) {
+	  Node node = new Node(name, startCode);
+	  node.setHeapSizeMB(heapSizeMB);
+	  node.setMaxHeapSizeMB(maxHeapSizeMB);
+	  liveNodes.add(node);
+	  return node;
+	}
+
+	/**
+	 * @param index the index
+	 * @return the region server model
+	 */
+	public Node getLiveNode(int index) {
+	  return liveNodes.get(index);
+	}
+
+	/**
+	 * Add a dead node to the cluster representation.
+	 * @param node the dead region server's name
+	 */
+	public void addDeadNode(String node) {
+	  deadNodes.add(node);
+	}
+	
+	/**
+	 * @param index the index
+	 * @return the dead region server's name
+	 */
+	public String getDeadNode(int index) {
+	  return deadNodes.get(index);
+	}
+
+	/**
+	 * Default constructor
+	 */
+	public StorageClusterStatusModel() {}
+
+	/**
+	 * @return the list of live nodes
+	 */
+	@XmlElement(name="Node")
+	@XmlElementWrapper(name="LiveNodes")
+	public List<Node> getLiveNodes() {
+	  return liveNodes;
+	}
+
+	/**
+	 * @return the list of dead nodes
+	 */
+  @XmlElement(name="Node")
+  @XmlElementWrapper(name="DeadNodes")
+  public List<String> getDeadNodes() {
+    return deadNodes;
+  }
+
+  /**
+   * @return the total number of regions served by the cluster
+   */
+  @XmlAttribute
+  public int getRegions() {
+    return regions;
+  }
+
+  /**
+   * @return the total number of requests per second handled by the cluster in
+   * the last reporting interval
+   */
+  @XmlAttribute
+  public int getRequests() {
+    return requests;
+  }
+
+  /**
+   * @return the average load of the region servers in the cluster
+   */
+  @XmlAttribute
+  public double getAverageLoad() {
+    return averageLoad;
+  }
+
+  /**
+   * @param nodes the list of live node models
+   */
+  public void setLiveNodes(List<Node> nodes) {
+    this.liveNodes = nodes;
+  }
+
+  /**
+   * @param nodes the list of dead node names
+   */
+  public void setDeadNodes(List<String> nodes) {
+    this.deadNodes = nodes;
+  }
+
+  /**
+   * @param regions the total number of regions served by the cluster
+   */
+  public void setRegions(int regions) {
+    this.regions = regions;
+  }
+
+  /**
+   * @param requests the total number of requests per second handled by the
+   * cluster
+   */
+  public void setRequests(int requests) {
+    this.requests = requests;
+  }
+
+  /**
+   * @param averageLoad the average load of region servers in the cluster
+   */
+  public void setAverageLoad(double averageLoad) {
+    this.averageLoad = averageLoad;
+  }
+
+	/* (non-Javadoc)
+	 * @see java.lang.Object#toString()
+	 */
+	@Override
+	public String toString() {
+	  StringBuilder sb = new StringBuilder();
+	  sb.append(String.format("%d live servers, %d dead servers, " + 
+      "%.4f average load\n\n", liveNodes.size(), deadNodes.size(),
+      averageLoad));
+    if (!liveNodes.isEmpty()) {
+      sb.append(liveNodes.size());
+      sb.append(" live servers\n");
+      for (Node node: liveNodes) {
+        sb.append("    ");
+        sb.append(node.name);
+        sb.append(' ');
+        sb.append(node.startCode);
+        sb.append("\n        requests=");
+        sb.append(node.requests);
+        sb.append(", regions=");
+        sb.append(node.regions.size());
+        sb.append("\n        heapSizeMB=");
+        sb.append(node.heapSizeMB);
+        sb.append("\n        maxHeapSizeMB=");
+        sb.append(node.maxHeapSizeMB);
+        sb.append("\n\n");
+        for (Node.Region region: node.regions) {
+          sb.append("        ");
+          sb.append(Bytes.toString(region.name));
+          sb.append("\n            stores=");
+          sb.append(region.stores);
+          sb.append("\n            storefiless=");
+          sb.append(region.storefiles);
+          sb.append("\n            storefileSizeMB=");
+          sb.append(region.storefileSizeMB);
+          sb.append("\n            memstoreSizeMB=");
+          sb.append(region.memstoreSizeMB);
+          sb.append("\n            storefileIndexSizeMB=");
+          sb.append(region.storefileIndexSizeMB);
+          sb.append('\n');
+        }
+        sb.append('\n');
+      }
+    }
+    if (!deadNodes.isEmpty()) {
+      sb.append('\n');
+      sb.append(deadNodes.size());
+      sb.append(" dead servers\n");
+      for (String node: deadNodes) {
+        sb.append("    ");
+        sb.append(node);
+        sb.append('\n');
+      }
+    }
+	  return sb.toString();
+	}
+
+  @Override
+  public byte[] createProtobufOutput() {
+    StorageClusterStatus.Builder builder = StorageClusterStatus.newBuilder();
+    builder.setRegions(regions);
+    builder.setRequests(requests);
+    builder.setAverageLoad(averageLoad);
+    for (Node node: liveNodes) {
+      StorageClusterStatus.Node.Builder nodeBuilder = 
+        StorageClusterStatus.Node.newBuilder();
+      nodeBuilder.setName(node.name);
+      nodeBuilder.setStartCode(node.startCode);
+      nodeBuilder.setRequests(node.requests);
+      nodeBuilder.setHeapSizeMB(node.heapSizeMB);
+      nodeBuilder.setMaxHeapSizeMB(node.maxHeapSizeMB);
+      for (Node.Region region: node.regions) {
+        StorageClusterStatus.Region.Builder regionBuilder =
+          StorageClusterStatus.Region.newBuilder();
+        regionBuilder.setName(ByteString.copyFrom(region.name));
+        regionBuilder.setStores(region.stores);
+        regionBuilder.setStorefiles(region.storefiles);
+        regionBuilder.setStorefileSizeMB(region.storefileSizeMB);
+        regionBuilder.setMemstoreSizeMB(region.memstoreSizeMB);
+        regionBuilder.setStorefileIndexSizeMB(region.storefileIndexSizeMB);
+        nodeBuilder.addRegions(regionBuilder);
+      }
+      builder.addLiveNodes(nodeBuilder);
+    }
+    for (String node: deadNodes) {
+      builder.addDeadNodes(node);
+    }
+    return builder.build().toByteArray();
+  }
+
+  @Override
+  public ProtobufMessageHandler getObjectFromMessage(byte[] message)
+      throws IOException {
+    StorageClusterStatus.Builder builder = StorageClusterStatus.newBuilder();
+    builder.mergeFrom(message);
+    if (builder.hasRegions()) {
+      regions = builder.getRegions();
+    }
+    if (builder.hasRequests()) {
+      requests = builder.getRequests();
+    }
+    if (builder.hasAverageLoad()) {
+      averageLoad = builder.getAverageLoad();
+    }
+    for (StorageClusterStatus.Node node: builder.getLiveNodesList()) {
+      long startCode = node.hasStartCode() ? node.getStartCode() : -1;
+      StorageClusterStatusModel.Node nodeModel = 
+        addLiveNode(node.getName(), startCode, node.getHeapSizeMB(),
+          node.getMaxHeapSizeMB());
+      int requests = node.hasRequests() ? node.getRequests() : 0;
+      nodeModel.setRequests(requests);
+      for (StorageClusterStatus.Region region: node.getRegionsList()) {
+        nodeModel.addRegion(
+          region.getName().toByteArray(),
+          region.getStores(),
+          region.getStorefiles(),
+          region.getStorefileSizeMB(),
+          region.getMemstoreSizeMB(),
+          region.getStorefileIndexSizeMB());
+      }
+    }
+    for (String node: builder.getDeadNodesList()) {
+      addDeadNode(node);
+    }
+    return this;
+  }
+}

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterVersionModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterVersionModel.java?rev=945816&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterVersionModel.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterVersionModel.java Tue May 18 18:49:12 2010
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.Serializable;
+
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlValue;
+
+/**
+ * Simple representation of the version of the storage cluster
+ * 
+ * <pre>
+ * &lt;complexType name="StorageClusterVersion"&gt;
+ *   &lt;attribute name="version" type="string"&gt;&lt;/attribute&gt;
+ * &lt;/complexType&gt;
+ * </pre>
+ */
+@XmlRootElement(name="ClusterVersion")
+public class StorageClusterVersionModel implements Serializable {
+	private static final long serialVersionUID = 1L;
+
+	private String version;
+
+	/**
+	 * @return the storage cluster version
+	 */
+	@XmlValue
+	public String getVersion() {
+	  return version;
+	}
+
+	/**
+	 * @param version the storage cluster version
+	 */
+	public void setVersion(String version) {
+	  this.version = version;
+	}
+
+	/* (non-Javadoc)
+	 * @see java.lang.Object#toString()
+	 */
+	@Override
+	public String toString() {
+	  return version;
+	}
+}

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableInfoModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableInfoModel.java?rev=945816&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableInfoModel.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableInfoModel.java Tue May 18 18:49:12 2010
@@ -0,0 +1,159 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
+import org.apache.hadoop.hbase.rest.protobuf.generated.TableInfoMessage.TableInfo;
+
+import com.google.protobuf.ByteString;
+
+/**
+ * Representation of a list of table regions. 
+ * 
+ * <pre>
+ * &lt;complexType name="TableInfo"&gt;
+ *   &lt;sequence&gt;
+ *     &lt;element name="region" type="tns:TableRegion" 
+ *       maxOccurs="unbounded" minOccurs="1"&gt;&lt;/element&gt;
+ *   &lt;/sequence&gt;
+ *   &lt;attribute name="name" type="string"&gt;&lt;/attribute&gt;
+ * &lt;/complexType&gt;
+ * </pre>
+ */
+@XmlRootElement(name="TableInfo")
+public class TableInfoModel implements Serializable, ProtobufMessageHandler {
+  private static final long serialVersionUID = 1L;
+
+  private String name;
+  private List<TableRegionModel> regions = new ArrayList<TableRegionModel>();
+
+  /**
+   * Default constructor
+   */
+  public TableInfoModel() {}
+
+  /**
+   * Constructor
+   * @param name
+   */
+  public TableInfoModel(String name) {
+    this.name = name;
+  }
+
+  /**
+   * Add a region model to the list
+   * @param region the region
+   */
+  public void add(TableRegionModel region) {
+    regions.add(region);
+  }
+
+  /**
+   * @param index the index
+   * @return the region model
+   */
+  public TableRegionModel get(int index) {
+    return regions.get(index);
+  }
+
+  /**
+   * @return the table name
+   */
+  @XmlAttribute
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * @return the regions
+   */
+  @XmlElement(name="Region")
+  public List<TableRegionModel> getRegions() {
+    return regions;
+  }
+
+  /**
+   * @param name the table name
+   */
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  /**
+   * @param regions the regions to set
+   */
+  public void setRegions(List<TableRegionModel> regions) {
+    this.regions = regions;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    for(TableRegionModel aRegion : regions) {
+      sb.append(aRegion.toString());
+      sb.append('\n');
+    }
+    return sb.toString();
+  }
+
+  @Override
+  public byte[] createProtobufOutput() {
+    TableInfo.Builder builder = TableInfo.newBuilder();
+    builder.setName(name);
+    for (TableRegionModel aRegion: regions) {
+      TableInfo.Region.Builder regionBuilder = TableInfo.Region.newBuilder();
+      regionBuilder.setName(aRegion.getName());
+      regionBuilder.setId(aRegion.getId());
+      regionBuilder.setStartKey(ByteString.copyFrom(aRegion.getStartKey()));
+      regionBuilder.setEndKey(ByteString.copyFrom(aRegion.getEndKey()));
+      regionBuilder.setLocation(aRegion.getLocation());
+      builder.addRegions(regionBuilder);
+    }
+    return builder.build().toByteArray();
+  }
+
+  @Override
+  public ProtobufMessageHandler getObjectFromMessage(byte[] message) 
+      throws IOException {
+    TableInfo.Builder builder = TableInfo.newBuilder();
+    builder.mergeFrom(message);
+    setName(builder.getName());
+    for (TableInfo.Region region: builder.getRegionsList()) {
+      add(new TableRegionModel(builder.getName(), region.getId(), 
+          region.getStartKey().toByteArray(),
+          region.getEndKey().toByteArray(),
+          region.getLocation()));
+    }
+    return this;
+  }
+}

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableListModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableListModel.java?rev=945816&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableListModel.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableListModel.java Tue May 18 18:49:12 2010
@@ -0,0 +1,112 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlElementRef;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
+import org.apache.hadoop.hbase.rest.protobuf.generated.TableListMessage.TableList;
+
+/**
+ * Simple representation of a list of table names.
+ */
+@XmlRootElement(name="TableList")
+public class TableListModel implements Serializable, ProtobufMessageHandler {
+
+	private static final long serialVersionUID = 1L;
+
+	private List<TableModel> tables = new ArrayList<TableModel>();
+
+	/**
+	 * Default constructor
+	 */
+	public TableListModel() {}
+
+	/**
+	 * Add the table name model to the list
+	 * @param table the table model
+	 */
+	public void add(TableModel table) {
+		tables.add(table);
+	}
+	
+	/**
+	 * @param index the index
+	 * @return the table model
+	 */
+	public TableModel get(int index) {
+		return tables.get(index);
+	}
+
+	/**
+	 * @return the tables
+	 */
+	@XmlElementRef(name="table")
+	public List<TableModel> getTables() {
+		return tables;
+	}
+
+	/**
+	 * @param tables the tables to set
+	 */
+	public void setTables(List<TableModel> tables) {
+		this.tables = tables;
+	}
+
+	/* (non-Javadoc)
+	 * @see java.lang.Object#toString()
+	 */
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+		for(TableModel aTable : tables) {
+			sb.append(aTable.toString());
+			sb.append('\n');
+		}
+		return sb.toString();
+	}
+
+	@Override
+	public byte[] createProtobufOutput() {
+		TableList.Builder builder = TableList.newBuilder();
+		for (TableModel aTable : tables) {
+			builder.addName(aTable.getName());
+		}
+		return builder.build().toByteArray();
+	}
+
+  @Override
+  public ProtobufMessageHandler getObjectFromMessage(byte[] message)
+      throws IOException {
+    TableList.Builder builder = TableList.newBuilder();
+    builder.mergeFrom(message);
+    for (String table: builder.getNameList()) {
+      this.add(new TableModel(table));
+    }
+    return this;
+  }
+}

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableModel.java?rev=945816&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableModel.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableModel.java Tue May 18 18:49:12 2010
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.Serializable;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Simple representation of a table name.
+ * 
+ * <pre>
+ * &lt;complexType name="Table"&gt;
+ *   &lt;sequence&gt;
+ *     &lt;element name="name" type="string"&gt;&lt;/element&gt;
+ *   &lt;/sequence&gt;
+ * &lt;/complexType&gt;
+ * </pre>
+ */
+@XmlRootElement(name="table")
+public class TableModel implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+	
+	private String name;
+	
+	/**
+	 * Default constructor
+	 */
+	public TableModel() {}
+
+	/**
+	 * Constructor
+	 * @param name
+	 */
+	public TableModel(String name) {
+		super();
+		this.name = name;
+	}
+
+	/**
+	 * @return the name
+	 */
+	@XmlAttribute
+	public String getName() {
+		return name;
+	}
+
+	/**
+	 * @param name the name to set
+	 */
+	public void setName(String name) {
+		this.name = name;
+	}
+
+	/* (non-Javadoc)
+	 * @see java.lang.Object#toString()
+	 */
+	@Override
+	public String toString() {
+		return this.name;
+	}
+}

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableRegionModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableRegionModel.java?rev=945816&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableRegionModel.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableRegionModel.java Tue May 18 18:49:12 2010
@@ -0,0 +1,193 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.Serializable;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Representation of a region of a table and its current location on the
+ * storage cluster.
+ * 
+ * <pre>
+ * &lt;complexType name="TableRegion"&gt;
+ *   &lt;attribute name="name" type="string"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="id" type="int"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="startKey" type="base64Binary"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="endKey" type="base64Binary"&gt;&lt;/attribute&gt;
+ *   &lt;attribute name="location" type="string"&gt;&lt;/attribute&gt;
+ *  &lt;/complexType&gt;
+ * </pre>
+ */
+@XmlRootElement(name="Region")
+public class TableRegionModel implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  private String table;
+  private long id;
+  private byte[] startKey; 
+  private byte[] endKey;
+  private String location;
+
+  /**
+   * Constructor
+   */
+  public TableRegionModel() {}
+
+  /**
+   * Constructor
+   * @param table the table name
+   * @param id the encoded id of the region
+   * @param startKey the start key of the region
+   * @param endKey the end key of the region
+   */
+  public TableRegionModel(String table, long id, byte[] startKey,
+      byte[] endKey) {
+    this(table, id, startKey, endKey, null);
+  }
+
+  /**
+   * Constructor
+   * @param table the table name
+   * @param id the encoded id of the region
+   * @param startKey the start key of the region
+   * @param endKey the end key of the region
+   * @param location the name and port of the region server hosting the region
+   */
+  public TableRegionModel(String table, long id, byte[] startKey,
+      byte[] endKey, String location) {
+    this.table = table;
+    this.id = id;
+    this.startKey = startKey;
+    this.endKey = endKey;
+    this.location = location;
+  }
+
+  /**
+   * @return the region name
+   */
+  @XmlAttribute
+  public String getName() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(table);
+    sb.append(',');
+    sb.append(Bytes.toString(startKey));
+    sb.append(',');
+    sb.append(id);
+    return sb.toString();
+  }
+
+  /**
+   * @return the encoded region id
+   */
+  @XmlAttribute 
+  public long getId() {
+    return id;
+  }
+
+  /**
+   * @return the start key
+   */
+  @XmlAttribute 
+  public byte[] getStartKey() {
+    return startKey;
+  }
+
+  /**
+   * @return the end key
+   */
+  @XmlAttribute 
+  public byte[] getEndKey() {
+    return endKey;
+  }
+
+  /**
+   * @return the name and port of the region server hosting the region
+   */
+  @XmlAttribute 
+  public String getLocation() {
+    return location;
+  }
+
+  /**
+   * @param name region printable name
+   */
+  public void setName(String name) {
+    String split[] = name.split(",");
+    table = split[0];
+    startKey = Bytes.toBytes(split[1]);
+    id = Long.valueOf(split[2]);
+  }
+
+  /**
+   * @param id the region's encoded id
+   */
+  public void setId(long id) {
+    this.id = id;
+  }
+
+  /**
+   * @param startKey the start key
+   */
+  public void setStartKey(byte[] startKey) {
+    this.startKey = startKey;
+  }
+
+  /**
+   * @param endKey the end key
+   */
+  public void setEndKey(byte[] endKey) {
+    this.endKey = endKey;
+  }
+
+  /**
+   * @param location the name and port of the region server hosting the region
+   */
+  public void setLocation(String location) {
+    this.location = location;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(getName());
+    sb.append(" [\n  id=");
+    sb.append(id);
+    sb.append("\n  startKey='");
+    sb.append(Bytes.toString(startKey));
+    sb.append("'\n  endKey='");
+    sb.append(Bytes.toString(endKey));
+    if (location != null) {
+      sb.append("'\n  location='");
+      sb.append(location);
+    }
+    sb.append("'\n]\n");
+    return sb.toString();
+  }
+}

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java?rev=945816&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java Tue May 18 18:49:12 2010
@@ -0,0 +1,353 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlAnyAttribute;
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.namespace.QName;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
+import org.apache.hadoop.hbase.rest.protobuf.generated.ColumnSchemaMessage.ColumnSchema;
+import org.apache.hadoop.hbase.rest.protobuf.generated.TableSchemaMessage.TableSchema;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * A representation of HBase table descriptors.
+ * 
+ * <pre>
+ * &lt;complexType name="TableSchema"&gt;
+ *   &lt;sequence&gt;
+ *     &lt;element name="column" type="tns:ColumnSchema" 
+ *       maxOccurs="unbounded" minOccurs="1"&gt;&lt;/element&gt;
+ *   &lt;/sequence&gt;
+ *   &lt;attribute name="name" type="string"&gt;&lt;/attribute&gt;
+ *   &lt;anyAttribute&gt;&lt;/anyAttribute&gt;
+ * &lt;/complexType&gt;
+ * </pre>
+ */
+@XmlRootElement(name="TableSchema")
+public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
+  private static final long serialVersionUID = 1L;
+  private static final QName IS_META = new QName(HTableDescriptor.IS_META);
+  private static final QName IS_ROOT = new QName(HTableDescriptor.IS_ROOT);
+  private static final QName READONLY = new QName(HTableDescriptor.READONLY);
+  private static final QName TTL = new QName(HColumnDescriptor.TTL);
+  private static final QName VERSIONS = new QName(HConstants.VERSIONS);
+  private static final QName COMPRESSION = 
+    new QName(HColumnDescriptor.COMPRESSION);
+
+  private String name;
+  private Map<QName,Object> attrs = new HashMap<QName,Object>();
+  private List<ColumnSchemaModel> columns = new ArrayList<ColumnSchemaModel>();
+  
+  /**
+   * Default constructor.
+   */
+  public TableSchemaModel() {}
+
+  /**
+   * Constructor
+   * @param htd the table descriptor
+   */
+  public TableSchemaModel(HTableDescriptor htd) {
+    setName(htd.getNameAsString());
+    for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
+        htd.getValues().entrySet()) {
+      addAttribute(Bytes.toString(e.getKey().get()), 
+        Bytes.toString(e.getValue().get()));
+    }
+    for (HColumnDescriptor hcd: htd.getFamilies()) {
+      ColumnSchemaModel columnModel = new ColumnSchemaModel();
+      columnModel.setName(hcd.getNameAsString());
+      for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
+          hcd.getValues().entrySet()) {
+        columnModel.addAttribute(Bytes.toString(e.getKey().get()), 
+            Bytes.toString(e.getValue().get()));
+      }
+      addColumnFamily(columnModel);
+    }
+  }
+
+  /**
+   * Add an attribute to the table descriptor
+   * @param name attribute name
+   * @param value attribute value
+   */
+  public void addAttribute(String name, Object value) {
+    attrs.put(new QName(name), value);
+  }
+
+  /**
+   * Return a table descriptor value as a string. Calls toString() on the
+   * object stored in the descriptor value map.
+   * @param name the attribute name
+   * @return the attribute value
+   */
+  public String getAttribute(String name) {
+    Object o = attrs.get(new QName(name));
+    return o != null ? o.toString() : null;
+  }
+
+  /**
+   * Add a column family to the table descriptor
+   * @param family the column family model
+   */
+  public void addColumnFamily(ColumnSchemaModel family) {
+    columns.add(family);
+  }
+
+  /**
+   * Retrieve the column family at the given index from the table descriptor
+   * @param index the index
+   * @return the column family model
+   */
+  public ColumnSchemaModel getColumnFamily(int index) {
+    return columns.get(index);
+  }
+
+  /**
+   * @return the table name
+   */
+  @XmlAttribute
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * @return the map for holding unspecified (user) attributes
+   */
+  @XmlAnyAttribute
+  public Map<QName,Object> getAny() {
+    return attrs;
+  }
+
+  /**
+   * @return the columns
+   */
+  @XmlElement(name="ColumnSchema")
+  public List<ColumnSchemaModel> getColumns() {
+    return columns;
+  }
+
+  /**
+   * @param name the table name
+   */
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  /**
+   * @param columns the columns to set
+   */
+  public void setColumns(List<ColumnSchemaModel> columns) {
+    this.columns = columns;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("{ NAME=> '");
+    sb.append(name);
+    sb.append('\'');
+    for (Map.Entry<QName,Object> e: attrs.entrySet()) {
+      sb.append(", ");
+      sb.append(e.getKey().getLocalPart());
+      sb.append(" => '");
+      sb.append(e.getValue().toString());
+      sb.append('\'');
+    }
+    sb.append(", COLUMNS => [ ");
+    Iterator<ColumnSchemaModel> i = columns.iterator();
+    while (i.hasNext()) {
+      ColumnSchemaModel family = i.next();
+      sb.append(family.toString());
+      if (i.hasNext()) {
+        sb.append(',');
+      }
+      sb.append(' ');
+    }
+    sb.append("] }");
+    return sb.toString();
+  }
+
+  // getters and setters for common schema attributes
+
+  // cannot be standard bean type getters and setters, otherwise this would
+  // confuse JAXB
+
+  /**
+   * @return true if IS_META attribute exists and is truel
+   */
+  public boolean __getIsMeta() {
+    Object o = attrs.get(IS_META);
+    return o != null ? Boolean.valueOf(o.toString()) : false;
+  }
+
+  /**
+   * @return true if IS_ROOT attribute exists and is truel
+   */
+  public boolean __getIsRoot() {
+    Object o = attrs.get(IS_ROOT);
+    return o != null ? Boolean.valueOf(o.toString()) : false;
+  }
+
+  /**
+   * @return true if READONLY attribute exists and is truel
+   */
+  public boolean __getReadOnly() {
+    Object o = attrs.get(READONLY);
+    return o != null ? 
+      Boolean.valueOf(o.toString()) : HTableDescriptor.DEFAULT_READONLY;
+  }
+
+  /**
+   * @param value desired value of IS_META attribute
+   */
+  public void __setIsMeta(boolean value) {
+    attrs.put(IS_META, Boolean.toString(value));
+  }
+
+  /**
+   * @param value desired value of IS_ROOT attribute
+   */
+  public void __setIsRoot(boolean value) {
+    attrs.put(IS_ROOT, Boolean.toString(value));
+  }
+
+  /**
+   * @param value desired value of READONLY attribute
+   */
+  public void __setReadOnly(boolean value) {
+    attrs.put(READONLY, Boolean.toString(value));
+  }
+
+  @Override
+  public byte[] createProtobufOutput() {
+    TableSchema.Builder builder = TableSchema.newBuilder();
+    builder.setName(name);
+    for (Map.Entry<QName, Object> e: attrs.entrySet()) {
+      TableSchema.Attribute.Builder attrBuilder = 
+        TableSchema.Attribute.newBuilder();
+      attrBuilder.setName(e.getKey().getLocalPart());
+      attrBuilder.setValue(e.getValue().toString());
+      builder.addAttrs(attrBuilder);
+    }
+    for (ColumnSchemaModel family: columns) {
+      Map<QName, Object> familyAttrs = family.getAny();
+      ColumnSchema.Builder familyBuilder = ColumnSchema.newBuilder();
+      familyBuilder.setName(family.getName());
+      for (Map.Entry<QName, Object> e: familyAttrs.entrySet()) {
+        ColumnSchema.Attribute.Builder attrBuilder = 
+          ColumnSchema.Attribute.newBuilder();
+        attrBuilder.setName(e.getKey().getLocalPart());
+        attrBuilder.setValue(e.getValue().toString());
+        familyBuilder.addAttrs(attrBuilder);
+      }
+      if (familyAttrs.containsKey(TTL)) {
+        familyBuilder.setTtl(
+          Integer.valueOf(familyAttrs.get(TTL).toString()));
+      }
+      if (familyAttrs.containsKey(VERSIONS)) {
+        familyBuilder.setMaxVersions(
+          Integer.valueOf(familyAttrs.get(VERSIONS).toString()));
+      }
+      if (familyAttrs.containsKey(COMPRESSION)) {
+        familyBuilder.setCompression(familyAttrs.get(COMPRESSION).toString());
+      }
+      builder.addColumns(familyBuilder);
+    }
+    if (attrs.containsKey(READONLY)) {
+      builder.setReadOnly(
+        Boolean.valueOf(attrs.get(READONLY).toString()));
+    }
+    return builder.build().toByteArray();
+  }
+
+  @Override
+  public ProtobufMessageHandler getObjectFromMessage(byte[] message) 
+      throws IOException {
+    TableSchema.Builder builder = TableSchema.newBuilder();
+    builder.mergeFrom(message);
+    this.setName(builder.getName());
+    for (TableSchema.Attribute attr: builder.getAttrsList()) {
+      this.addAttribute(attr.getName(), attr.getValue());
+    }
+    if (builder.hasReadOnly()) {
+      this.addAttribute(HTableDescriptor.READONLY, builder.getReadOnly());
+    }
+    for (ColumnSchema family: builder.getColumnsList()) {
+      ColumnSchemaModel familyModel = new ColumnSchemaModel();
+      familyModel.setName(family.getName());
+      for (ColumnSchema.Attribute attr: family.getAttrsList()) {
+        familyModel.addAttribute(attr.getName(), attr.getValue());
+      }
+      if (family.hasTtl()) {
+        familyModel.addAttribute(HColumnDescriptor.TTL, family.getTtl());
+      }
+      if (family.hasMaxVersions()) {
+        familyModel.addAttribute(HConstants.VERSIONS,
+          family.getMaxVersions());
+      }
+      if (family.hasCompression()) {
+        familyModel.addAttribute(HColumnDescriptor.COMPRESSION,
+          family.getCompression());
+      }
+      this.addColumnFamily(familyModel);
+    }
+    return this;
+  }
+
+  /**
+   * @return a table descriptor
+   */
+  public HTableDescriptor getTableDescriptor() {
+    HTableDescriptor htd = new HTableDescriptor(getName());
+    for (Map.Entry<QName, Object> e: getAny().entrySet()) {
+      htd.setValue(e.getKey().getLocalPart(), e.getValue().toString());
+    }
+    for (ColumnSchemaModel column: getColumns()) {
+      HColumnDescriptor hcd = new HColumnDescriptor(column.getName());
+      for (Map.Entry<QName, Object> e: column.getAny().entrySet()) {
+        hcd.setValue(e.getKey().getLocalPart(), e.getValue().toString());
+      }
+      htd.addFamily(hcd);
+    }
+    return htd;
+  }
+
+}

Added: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/VersionModel.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/VersionModel.java?rev=945816&view=auto
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/VersionModel.java (added)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/rest/model/VersionModel.java Tue May 18 18:49:12 2010
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+import javax.servlet.ServletContext;
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
+import org.apache.hadoop.hbase.rest.RESTServlet;
+import org.apache.hadoop.hbase.rest.protobuf.generated.VersionMessage.Version;
+
+import com.sun.jersey.spi.container.servlet.ServletContainer;
+
+/**
+ * A representation of the collection of versions of the REST gateway software
+ * components.
+ * <ul>
+ * <li>restVersion: REST gateway revision</li>
+ * <li>jvmVersion: the JVM vendor and version information</li>
+ * <li>osVersion: the OS type, version, and hardware architecture</li>
+ * <li>serverVersion: the name and version of the servlet container</li>
+ * <li>jerseyVersion: the version of the embedded Jersey framework</li>
+ * </ul>
+ */
+@XmlRootElement(name="Version")
+public class VersionModel implements Serializable, ProtobufMessageHandler {
+
+	private static final long serialVersionUID = 1L;
+
+	private String restVersion;
+  private String jvmVersion;
+  private String osVersion;
+  private String serverVersion;
+  private String jerseyVersion;
+
+  /**
+   * Default constructor. Do not use.
+   */
+  public VersionModel() {}
+  
+  /**
+   * Constructor
+   * @param context the servlet context
+   */
+	public VersionModel(ServletContext context) {
+	  restVersion = RESTServlet.VERSION_STRING;
+	  jvmVersion = System.getProperty("java.vm.vendor") + ' ' +
+      System.getProperty("java.version") + '-' +
+      System.getProperty("java.vm.version");
+	  osVersion = System.getProperty("os.name") + ' ' +
+      System.getProperty("os.version") + ' ' +
+      System.getProperty("os.arch");
+	  serverVersion = context.getServerInfo();
+	  jerseyVersion = ServletContainer.class.getPackage()
+      .getImplementationVersion();
+	}
+
+	/**
+	 * @return the REST gateway version
+	 */
+	@XmlAttribute(name="REST")
+	public String getRESTVersion() {
+    return restVersion;
+  }
+
+	/**
+	 * @return the JVM vendor and version
+	 */
+  @XmlAttribute(name="JVM")
+  public String getJVMVersion() {
+    return jvmVersion;
+  }
+
+  /**
+   * @return the OS name, version, and hardware architecture
+   */
+  @XmlAttribute(name="OS")
+  public String getOSVersion() {
+    return osVersion;
+  }
+
+  /**
+   * @return the servlet container version
+   */
+  @XmlAttribute(name="Server")
+  public String getServerVersion() {
+    return serverVersion;
+  }
+
+  /**
+   * @return the version of the embedded Jersey framework
+   */
+  @XmlAttribute(name="Jersey")
+  public String getJerseyVersion() {
+    return jerseyVersion;
+  }
+
+  /**
+   * @param version the REST gateway version string
+   */
+  public void setRESTVersion(String version) {
+    this.restVersion = version;
+  }
+
+  /**
+   * @param version the OS version string
+   */
+  public void setOSVersion(String version) {
+    this.osVersion = version;
+  }
+
+  /**
+   * @param version the JVM version string
+   */
+  public void setJVMVersion(String version) {
+    this.jvmVersion = version;
+  }
+
+  /**
+   * @param version the servlet container version string
+   */
+  public void setServerVersion(String version) {
+    this.serverVersion = version;
+  }
+
+  /**
+   * @param version the Jersey framework version string
+   */
+  public void setJerseyVersion(String version) {
+    this.jerseyVersion = version;
+  }
+
+  /* (non-Javadoc)
+	 * @see java.lang.Object#toString()
+	 */
+	@Override
+	public String toString() {
+	  StringBuilder sb = new StringBuilder();
+	  sb.append("rest ");
+	  sb.append(restVersion);
+	  sb.append(" [JVM: ");
+	  sb.append(jvmVersion);
+	  sb.append("] [OS: ");
+	  sb.append(osVersion);
+	  sb.append("] [Server: ");
+	  sb.append(serverVersion);
+	  sb.append("] [Jersey: ");
+    sb.append(jerseyVersion);
+	  sb.append("]\n");
+	  return sb.toString();
+	}
+
+	@Override
+  public byte[] createProtobufOutput() {
+	  Version.Builder builder = Version.newBuilder();
+	  builder.setRestVersion(restVersion);
+	  builder.setJvmVersion(jvmVersion);
+	  builder.setOsVersion(osVersion);
+	  builder.setServerVersion(serverVersion);
+	  builder.setJerseyVersion(jerseyVersion);
+	  return builder.build().toByteArray();
+  }
+
+  @Override
+  public ProtobufMessageHandler getObjectFromMessage(byte[] message)
+      throws IOException {
+    Version.Builder builder = Version.newBuilder();
+    builder.mergeFrom(message);
+    if (builder.hasRestVersion()) {
+      restVersion = builder.getRestVersion();
+    }
+    if (builder.hasJvmVersion()) {
+      jvmVersion = builder.getJvmVersion();
+    }
+    if (builder.hasOsVersion()) {
+      osVersion = builder.getOsVersion();
+    }
+    if (builder.hasServerVersion()) {
+      serverVersion = builder.getServerVersion();
+    }
+    if (builder.hasJerseyVersion()) {
+      jerseyVersion = builder.getJerseyVersion();
+    }
+    return this;
+  }
+}