hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From raw...@apache.org
Subject svn commit: r782178 [4/16] - in /hadoop/hbase/trunk: bin/ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/client/ src/java/org/apache/hadoop/hbase/client/tableindexed/ src/java/org/apache/hadoop/hbase/client/transactional/ src/java/o...
Date Sat, 06 Jun 2009 01:26:27 GMT
Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Result.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Result.java?rev=782178&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Result.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Result.java Sat Jun  6 01:26:21 2009
@@ -0,0 +1,479 @@
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.SplitKeyValue;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Single row result of a {@link Get} or {@link Scan} query.
+ */
+public class Result implements Writable {
+  private KeyValue [] kvs = null;
+  private NavigableMap<byte[], 
+     NavigableMap<byte[], NavigableMap<Long, byte[]>>> familyMap = null;
+  // We're not using java serialization.  Transient here is just a marker to say
+  // that this is where we cache row if we're ever asked for it.
+  private transient byte [] row = null;
+
+  /**
+   * Constructor used for Writable.
+   */
+  public Result() {}
+
+  /**
+   * Instantiate a Result with the specified array of KeyValues.
+   * @param kvs array of KeyValues
+   */
+  public Result(KeyValue [] kvs) {
+    if(kvs != null && kvs.length > 0) {
+      this.kvs = kvs;
+    }
+  }
+
+  /**
+   * Instantiate a Result with the specified List of KeyValues.
+   * @param kvs List of KeyValues
+   */
+  public Result(List<KeyValue> kvs) {
+    this(kvs.toArray(new KeyValue[0]));
+  }
+
+  /**
+   * Method for retrieving the row that this result is for
+   * @return row
+   */
+  public synchronized byte [] getRow() {
+    if (this.row == null) {
+      this.row =
+        this.kvs == null || this.kvs.length == 0? null: this.kvs[0].getRow();
+    }
+    return this.row;
+  }
+
+  /**
+   * Directly return the unsorted array of KeyValues in this Result.
+   * @return unsorted array of KeyValues
+   */
+  public KeyValue[] raw() {
+    return kvs;
+  }
+
+  /**
+   * Return a sorted list of the KeyValues in this result.
+   * @return
+   */
+  public List<KeyValue> list() {
+    return Arrays.asList(sorted());
+  }
+
+  /**
+   * Returns a sorted array of KeyValues in this Result.
+   * <p>
+   * Note: Sorting is done in place, so the backing array will be sorted
+   * after calling this method.
+   * @return sorted array of KeyValues
+   */
+  public KeyValue[] sorted() {
+    if(isEmpty()) {
+      return null;
+    }
+    Arrays.sort(kvs, (Comparator<KeyValue>)KeyValue.COMPARATOR);
+    return kvs;
+  }
+
+  /**
+   * Map of families to all versions of its qualifiers and values.
+   * <p>
+   * Returns a three level Map of the form: 
+   * <code>Map<family,Map<qualifier,Map<timestamp,value>>></code>
+   * <p>
+   * Note: All other map returning methods make use of this map internally. 
+   * @return map from families to qualifiers to versions
+   */
+  public NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> getMap() {
+    if(this.familyMap != null) {
+      return this.familyMap;
+    }
+    if(isEmpty()) {
+      return null;
+    }
+    this.familyMap =
+      new TreeMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>
+      (Bytes.BYTES_COMPARATOR);
+    for(KeyValue kv : this.kvs) {
+      SplitKeyValue splitKV = kv.split();
+      byte [] family = splitKV.getFamily();
+      NavigableMap<byte[], NavigableMap<Long, byte[]>> columnMap = 
+        familyMap.get(family);
+      if(columnMap == null) {
+        columnMap = new TreeMap<byte[], NavigableMap<Long, byte[]>>
+          (Bytes.BYTES_COMPARATOR);
+        familyMap.put(family, columnMap);
+      }
+      byte [] qualifier = splitKV.getQualifier();
+      NavigableMap<Long, byte[]> versionMap = columnMap.get(qualifier);
+      if(versionMap == null) {
+        versionMap = new TreeMap<Long, byte[]>(new Comparator<Long>() {
+          public int compare(Long l1, Long l2) {
+            return l2.compareTo(l1);
+          }
+        });
+        columnMap.put(qualifier, versionMap);
+      }
+      Long timestamp = Bytes.toLong(splitKV.getTimestamp());
+      byte [] value = splitKV.getValue();
+      versionMap.put(timestamp, value);
+    }
+    return this.familyMap;
+  }
+
+  /**
+   * Map of families to their most recent qualifiers and values.
+   * <p>
+   * Returns a two level Map of the form: <code>Map<family,Map<qualifier,value>></code>
+   * <p>
+   * The most recent version of each qualifier will be used.
+   * @return map from families to qualifiers and value
+   */
+  public NavigableMap<byte[], NavigableMap<byte[], byte[]>> getNoVersionMap() {
+    if(this.familyMap == null) {
+      getMap();
+    }
+    if(isEmpty()) {
+      return null;
+    }
+    NavigableMap<byte[], NavigableMap<byte[], byte[]>> returnMap = 
+      new TreeMap<byte[], NavigableMap<byte[], byte[]>>(Bytes.BYTES_COMPARATOR);
+    for(Map.Entry<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> 
+      familyEntry : familyMap.entrySet()) {
+      NavigableMap<byte[], byte[]> qualifierMap = 
+        new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
+      for(Map.Entry<byte[], NavigableMap<Long, byte[]>> qualifierEntry :
+        familyEntry.getValue().entrySet()) {
+        byte [] value = 
+          qualifierEntry.getValue().get(qualifierEntry.getValue().firstKey());
+        qualifierMap.put(qualifierEntry.getKey(), value);
+      }
+      returnMap.put(familyEntry.getKey(), qualifierMap);
+    }
+    return returnMap;
+  }
+
+  /**
+   * Map of qualifiers to values.
+   * <p>
+   * Returns a Map of the form: <code>Map<qualifier,value></code>
+   * @return map of qualifiers to values
+   */
+  public NavigableMap<byte[], byte[]> getFamilyMap(byte [] family) {
+    if(this.familyMap == null) {
+      getMap();
+    }
+    if(isEmpty()) {
+      return null;
+    }
+    NavigableMap<byte[], byte[]> returnMap = 
+      new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
+    NavigableMap<byte[], NavigableMap<Long, byte[]>> qualifierMap = 
+      familyMap.get(family);
+    if(qualifierMap == null) {
+      return returnMap;
+    }
+    for(Map.Entry<byte[], NavigableMap<Long, byte[]>> entry : 
+      qualifierMap.entrySet()) {
+      byte [] value = 
+        entry.getValue().get(entry.getValue().firstKey());
+      returnMap.put(entry.getKey(), value);
+    }
+    return returnMap;
+  }
+
+  /**
+   * Get the latest version of the specified column.
+   * @param family family name
+   * @param qualifier column qualifier
+   * @return value of latest version of column, null if none found
+   */
+  public byte [] getValue(byte [] family, byte [] qualifier) {
+    Map.Entry<Long,byte[]> entry = getKeyValue(family, qualifier);
+    return entry == null ?null :entry.getValue();
+  }
+
+  public Cell getCellValue(byte[] family, byte[] qualifier) {
+    Map.Entry<Long,byte[]> val = getKeyValue(family, qualifier);
+    return new Cell(val.getValue(), val.getKey());
+  }
+
+  /**
+   * @return First KeyValue in this Result as a Cell or null if empty.
+   */
+  public Cell getCellValue() {
+    return isEmpty()? null: new Cell(kvs[0].getValue(), kvs[0].getTimestamp());
+  }
+
+  /**
+   * @return This Result as array of Cells or null if empty.
+   */
+  public Cell [] getCellValues() {
+    if (isEmpty()) return null;
+    Cell [] results = new Cell[kvs.length];
+    for (int i = 0; i < kvs.length; i++) {
+      results[i] = new Cell(kvs[i].getValue(), kvs[i].getTimestamp());
+    }
+    return results;
+  }
+
+  private Map.Entry<Long,byte[]> getKeyValue(byte[] family, byte[] qualifier) {
+    if(this.familyMap == null) {
+      getMap();
+    }
+    if(isEmpty()) {
+      return null;
+    }
+    NavigableMap<byte [], NavigableMap<Long, byte[]>> qualifierMap =
+      familyMap.get(family);
+    if(qualifierMap == null) {
+      return null;
+    }
+    NavigableMap<Long, byte[]> versionMap =
+      getVersionMap(qualifierMap, qualifier);
+    if(versionMap == null) {
+      return null;
+    }
+    return versionMap.firstEntry();
+  }
+  
+  private NavigableMap<Long, byte[]> getVersionMap(
+      NavigableMap<byte [], NavigableMap<Long, byte[]>> qualifierMap, byte [] qualifier) {
+    if(qualifier != null) {
+      return qualifierMap.get(qualifier);
+    } else {
+      return qualifierMap.get(new byte[0]);
+    }
+  }
+  
+  /**
+   * Get the latest version of the specified column,
+   * using <pre>family:qualifier</pre> notation.
+   * @param column column in family:qualifier notation
+   * @return value of latest version of column, null if none found
+   */
+  public byte [] getValue(byte [] column) {
+    try {
+      byte [][] split = KeyValue.parseColumn(column);
+      return getValue(split[0], split[1]);
+    } catch(Exception e) {
+      return null;
+    }
+  }
+  
+  /**
+   * Checks for existence of the specified column.
+   * @param family family name
+   * @param qualifier column qualifier
+   * @return true if at least one value exists in the result, false if not
+   */
+  public boolean containsColumn(byte [] family, byte [] qualifier) {
+    if(this.familyMap == null) {
+      getMap();
+    }
+    if(isEmpty()) {
+      return false;
+    }
+    NavigableMap<byte [], NavigableMap<Long, byte[]>> qualifierMap =
+      familyMap.get(family);
+    if(qualifierMap == null) {
+      return false;
+    }
+    NavigableMap<Long, byte[]> versionMap = getVersionMap(qualifierMap, qualifier);
+    if(versionMap == null) {
+      return false;
+    }
+    return true;
+  }
+  
+  /**
+   * Returns this Result in the old return format, {@link RowResult}.
+   * @return a RowResult
+   */
+  public RowResult getRowResult() {
+    return RowResult.createRowResult(Arrays.asList(kvs));
+  }
+  
+  /**
+   * Returns the value of the first column in the Result.
+   * @return value of the first column
+   */
+  public byte [] value() {
+    if(isEmpty()) {
+      return null;
+    }
+    return kvs[0].getValue();
+  }
+  
+  /**
+   * Check if the underlying KeyValue [] is empty or not
+   * @return true if empty
+   */
+  public boolean isEmpty() {
+    return (this.kvs == null || this.kvs.length == 0);
+  }
+  
+  /**
+   * 
+   * @return the size of the underlying KeyValue []
+   */
+  public int size() {
+    return (this.kvs == null ? 0 : this.kvs.length);
+  }
+  
+  /**
+   * @return String
+   */
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append("keyvalues=");
+    if(isEmpty()) {
+      sb.append("NONE");
+      return sb.toString();
+    }
+    sb.append("{");
+    boolean moreThanOne = false;
+    for(KeyValue kv : this.kvs) {
+      if(moreThanOne) {
+        sb.append(", ");
+      } else {
+        moreThanOne = true;
+      }
+      sb.append(kv.toString());
+    }
+    sb.append("}");
+    return sb.toString();
+  }
+  
+  //Writable
+  public void readFields(final DataInput in)
+  throws IOException {
+    int numKeys = in.readInt();
+    this.kvs = new KeyValue[numKeys];
+    if(numKeys == 0) {
+      return;
+    }
+    int totalBuffer = in.readInt();
+    byte [] buf = new byte[totalBuffer];
+    int offset = 0;
+    for(int i=0; i<numKeys; i++) {
+      int keyLength = in.readInt();
+      in.readFully(buf, offset, keyLength);
+      kvs[i] = new KeyValue(buf, offset, keyLength);
+      offset += keyLength;
+    }
+  }
+  
+  public void write(final DataOutput out)
+  throws IOException {
+    if(isEmpty()) {
+      out.writeInt(0);
+    } else {
+      int len = this.kvs.length;
+      out.writeInt(len);
+      int totalLen = 0;
+      for(KeyValue kv : kvs) {
+        totalLen += kv.getLength();
+      }
+      out.writeInt(totalLen);
+      for(KeyValue kv : kvs) {
+        out.writeInt(kv.getLength());
+        out.write(kv.getBuffer(), kv.getOffset(), kv.getLength());
+      }
+    }
+  }
+  
+  public static void writeArray(final DataOutput out, Result [] results)
+  throws IOException {
+    if(results == null || results.length == 0) {
+      out.writeInt(0);
+      return;
+    }
+    out.writeInt(results.length);
+    int bufLen = 0;
+    for(Result result : results) {
+      if(result == null || result.isEmpty()) {
+        continue;
+      }
+      for(KeyValue key : result.raw()) {
+        bufLen += key.getLength();
+      }
+    }
+    out.writeInt(bufLen);
+    for(Result result : results) {
+      if(result == null || result.isEmpty()) {
+        out.writeInt(0);
+        continue;
+      }
+      out.writeInt(result.size());
+      for(KeyValue kv : result.raw()) {
+        out.writeInt(kv.getLength());
+        out.write(kv.getBuffer(), kv.getOffset(), kv.getLength());
+      }
+    }
+  }
+  
+  public static Result [] readArray(final DataInput in)
+  throws IOException {
+    int numResults = in.readInt();
+    if(numResults == 0) {
+      return new Result[0];
+    }
+    Result [] results = new Result[numResults];
+    int bufSize = in.readInt();
+    byte [] buf = new byte[bufSize];
+    int offset = 0;
+    for(int i=0;i<numResults;i++) {
+      int numKeys = in.readInt();
+      KeyValue [] keys = new KeyValue[numKeys];
+      for(int j=0;j<numKeys;j++) {
+        int keyLen = in.readInt();
+        in.readFully(buf, offset, keyLen);
+        keys[j] = new KeyValue(buf, offset, keyLen);
+        offset += keyLen;
+      }
+      results[i] = new Result(keys);
+    }
+    return results;
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ResultScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ResultScanner.java?rev=782178&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ResultScanner.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ResultScanner.java Sat Jun  6 01:26:21 2009
@@ -0,0 +1,52 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Result;
+
+/**
+ * Interface for client-side scanning.
+ * Go to {@link HTable} to obtain instances.
+ */
+public interface ResultScanner extends Closeable, Iterable<Result> {
+
+  /**
+   * Grab the next row's worth of values. The scanner will return a Result.
+   * @return Result object if there is another row, null if the scanner is
+   * exhausted.
+   * @throws IOException
+   */  
+  public Result next() throws IOException;
+ 
+  /**
+   * @param nbRows number of rows to return
+   * @return Between zero and <param>nbRows</param> Results
+   * @throws IOException
+   */
+  public Result [] next(int nbRows) throws IOException;
+ 
+  /**
+   * Closes the scanner and releases any resources it has allocated
+   */
+  public void close();  
+}
\ No newline at end of file

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scan.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scan.java?rev=782178&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scan.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scan.java Sat Jun  6 01:26:21 2009
@@ -0,0 +1,456 @@
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+import org.apache.hadoop.hbase.io.HbaseObjectWritable;
+import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Writable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+/**
+ * Used to perform Scan operations.
+ * <p>
+ * All operations are identical to {@link Get} with the exception of
+ * instantiation.  Rather than specifying a single row, an optional startRow
+ * and stopRow may be defined.  If rows are not specified, the Scanner will
+ * iterate over all rows.
+ * <p>
+ * To scan everything for each row, instantiate a Scan object.
+ * To further define the scope of what to get when scanning, perform additional 
+ * methods as outlined below.
+ * <p>
+ * To get all columns from specific families, execute {@link #addFamily(byte[]) addFamily}
+ * for each family to retrieve.
+ * <p>
+ * To get specific columns, execute {@link #addColumn(byte[], byte[]) addColumn}
+ * for each column to retrieve.
+ * <p>
+ * To only retrieve columns within a specific range of version timestamps,
+ * execute {@link #setTimeRange(long, long) setTimeRange}.
+ * <p>
+ * To only retrieve columns with a specific timestamp, execute
+ * {@link #setTimeStamp(long) setTimestamp}.
+ * <p>
+ * To limit the number of versions of each column to be returned, execute
+ * {@link #setMaxVersions(int) setMaxVersions}.
+ * <p>
+ * To add a filter, execute {@link #setFilter(org.apache.hadoop.hbase.filter.Filter) setFilter}.
+ */
+public class Scan implements Writable {
+  private byte [] startRow = HConstants.EMPTY_START_ROW;
+  private byte [] stopRow  = HConstants.EMPTY_END_ROW;
+  private int maxVersions = 1;
+  private Filter filter = null;
+  private RowFilterInterface oldFilter = null;
+  private TimeRange tr = new TimeRange();
+  private Map<byte [], NavigableSet<byte []>> familyMap =
+    new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
+  
+  /**
+   * Create a Scan operation across all rows.
+   */
+  public Scan() {}
+
+  public Scan(byte [] startRow, Filter filter) {
+    this(startRow);
+    this.filter = filter;
+
+  }
+  
+  /**
+   * Create a Scan operation starting at the specified row.
+   * <p>
+   * If the specified row does not exist, the Scanner will start from the
+   * next closest row after the specified row.
+   * @param startRow row to start scanner at or after
+   */
+  public Scan(byte [] startRow) {
+    this.startRow = startRow;
+  }
+  
+  /**
+   * Create a Scan operation for the range of rows specified.
+   * @param startRow row to start scanner at or after (inclusive)
+   * @param stopRow row to stop scanner before (exclusive)
+   */
+  public Scan(byte [] startRow, byte [] stopRow) {
+    this.startRow = startRow;
+    this.stopRow = stopRow;
+  }
+  
+  /**
+   * Get all columns from the specified family.
+   * <p>
+   * Overrides previous calls to addColumn for this family.
+   * @param family family name
+   */
+  public Scan addFamily(byte [] family) {
+    familyMap.remove(family);
+    familyMap.put(family, null);
+
+    return this;
+  }
+  
+  /**
+   * Get the column from the specified family with the specified qualifier.
+   * <p>
+   * Overrides previous calls to addFamily for this family.
+   * @param family family name
+   * @param qualifier column qualifier
+   */
+  public Scan addColumn(byte [] family, byte [] qualifier) {
+    NavigableSet<byte []> set = familyMap.get(family);
+    if(set == null) {
+      set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
+    }
+    set.add(qualifier);
+    familyMap.put(family, set);
+
+    return this;
+  }
+  
+  /**
+   * Adds an array of columns specified the old format, family:qualifier.
+   * <p>
+   * Overrides previous calls to addFamily for any families in the input.
+   * @param columns array of columns, formatted as <pre>family:qualifier</pre>
+   */
+  public Scan addColumns(byte [][] columns) {
+    for(int i=0; i<columns.length; i++) {
+      try {
+        byte [][] split = KeyValue.parseColumn(columns[i]);
+        addColumn(split[0], split[1]);
+      } catch(Exception e) {}
+    }
+    return this;
+  }
+  
+  /**
+   * Get versions of columns only within the specified timestamp range,
+   * [minStamp, maxStamp).
+   * @param minStamp minimum timestamp value, inclusive
+   * @param maxStamp maximum timestamp value, exclusive
+   * @throws IOException if invalid time range
+   */
+  public Scan setTimeRange(long minStamp, long maxStamp)
+  throws IOException {
+    tr = new TimeRange(minStamp, maxStamp);
+    return this;
+  }
+  
+  /**
+   * Get versions of columns with the specified timestamp.
+   * @param timestamp version timestamp  
+   */
+  public Scan setTimeStamp(long timestamp) {
+    try {
+      tr = new TimeRange(timestamp, timestamp+1);
+    } catch(IOException e) {
+      // Will never happen
+    }
+    return this;
+  }
+
+  /**
+   * Set the start row.
+   * @param startRow
+   */
+  public Scan setStartRow(byte [] startRow) {
+    this.startRow = startRow;
+    return this;
+  }
+  
+  /**
+   * Set the stop row.
+   * @param stopRow
+   */
+  public Scan setStopRow(byte [] stopRow) {
+    this.stopRow = stopRow;
+    return this;
+  }
+  
+  /**
+   * Get all available versions.
+   */
+  public Scan setMaxVersions() {
+  	this.maxVersions = Integer.MAX_VALUE;
+    return this;
+  }
+
+  /**
+   * Get up to the specified number of versions of each column.
+   * @param maxVersions maximum versions for each column
+   * @throws IOException if invalid number of versions
+   */
+  public Scan setMaxVersions(int maxVersions) {
+    this.maxVersions = maxVersions;
+    return this;
+  }
+  
+  /**
+   * Apply the specified server-side filter when performing the Scan.
+   * @param filter filter to run on the server
+   */
+  public Scan setFilter(Filter filter) {
+    this.filter = filter;
+    return this;
+  }
+
+  /**
+   * Set an old-style filter interface to use. Note: not all features of the
+   * old style filters are supported.
+   * @deprecated
+   * @param filter
+   * @return
+   */
+  public Scan setOldFilter(RowFilterInterface filter) {
+    oldFilter = filter;
+
+    return this;
+  }
+  
+  /**
+   * Setting the familyMap
+   * @param familyMap
+   */
+  public Scan setFamilyMap(Map<byte [], NavigableSet<byte []>> familyMap) {
+    this.familyMap = familyMap;
+
+    return this;
+  }
+  
+  /**
+   * Getting the familyMap
+   * @return familyMap
+   */
+  public Map<byte [], NavigableSet<byte []>> getFamilyMap() {
+    return this.familyMap;
+  }
+  
+  /**
+   * @return the number of families in familyMap
+   */
+  public int numFamilies() {
+    if(hasFamilies()) {
+      return this.familyMap.size();
+    }
+    return 0;
+  }
+
+  /**
+   * @return true if familyMap is non empty, false otherwise
+   */
+  public boolean hasFamilies() {
+    return !this.familyMap.isEmpty();
+  }
+  
+  /**
+   * @return the keys of the familyMap
+   */
+  public byte[][] getFamilies() {
+    if(hasFamilies()) {
+      return this.familyMap.keySet().toArray(new byte[0][0]);
+    }
+    return null;
+  }
+  
+  /**
+   * @return the startrow
+   */
+  public byte [] getStartRow() {
+    return this.startRow;
+  }
+
+  /**
+   * @return the stoprow
+   */
+  public byte [] getStopRow() {
+    return this.stopRow;
+  }
+  
+  /**
+   * @return the max number of versions to fetch
+   */
+  public int getMaxVersions() {
+    return this.maxVersions;
+  } 
+
+  /**
+   * @return TimeRange
+   */
+  public TimeRange getTimeRange() {
+    return this.tr;
+  } 
+  
+  /**
+   * @return RowFilter
+   */
+  public Filter getFilter() {
+    return filter;
+  }
+
+  /**
+   * Get the old style filter, if there is one.
+   * @deprecated
+   * @return null or instance
+   */
+  public RowFilterInterface getOldFilter() {
+    return oldFilter;
+  }
+  
+  /**
+   * @return true is a filter has been specified, false if not
+   */
+  public boolean hasFilter() {
+    return filter != null || oldFilter != null;
+  }
+  
+  /**
+   * @return String
+   */
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append("startRow=");
+    sb.append(Bytes.toString(this.startRow));
+    sb.append(", stopRow=");
+    sb.append(Bytes.toString(this.stopRow));
+    sb.append(", maxVersions=");
+    sb.append("" + this.maxVersions);
+    sb.append(", timeRange=");
+    sb.append("[" + this.tr.getMin() + "," + this.tr.getMax() + ")");
+    sb.append(", families=");
+    if(this.familyMap.size() == 0) {
+      sb.append("ALL");
+      return sb.toString();
+    }
+    boolean moreThanOne = false;
+    for(Map.Entry<byte [], NavigableSet<byte[]>> entry : this.familyMap.entrySet()) {
+      if(moreThanOne) {
+        sb.append("), ");
+      } else {
+        moreThanOne = true;
+        sb.append("{");
+      }
+      sb.append("(family=");
+      sb.append(Bytes.toString(entry.getKey()));
+      sb.append(", columns=");
+      if(entry.getValue() == null) {
+        sb.append("ALL");
+      } else {
+        sb.append("{");
+        boolean moreThanOneB = false;
+        for(byte [] column : entry.getValue()) {
+          if(moreThanOneB) {
+            sb.append(", ");
+          } else {
+            moreThanOneB = true;
+          }
+          sb.append(Bytes.toString(column));
+        }
+        sb.append("}");
+      }
+    }
+    sb.append("}");
+    return sb.toString();
+  }
+  
+  //Writable
+  public void readFields(final DataInput in)
+  throws IOException {
+    this.startRow = Bytes.readByteArray(in);
+    this.stopRow = Bytes.readByteArray(in);
+    this.maxVersions = in.readInt();
+    boolean hasFilter = in.readBoolean();
+    if(hasFilter) {
+      this.filter = (Filter)HbaseObjectWritable.readObject(in,
+        new Configuration());
+    }
+    boolean hasOldFilter = in.readBoolean();
+    if (hasOldFilter) {
+      this.oldFilter = (RowFilterInterface)HbaseObjectWritable.readObject(in,
+          new Configuration());
+    }
+    this.tr = new TimeRange();
+    tr.readFields(in);
+    int numFamilies = in.readInt();
+    this.familyMap = 
+      new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
+    for(int i=0; i<numFamilies; i++) {
+      byte [] family = Bytes.readByteArray(in);
+      int numColumns = in.readInt();
+      TreeSet<byte []> set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
+      for(int j=0; j<numColumns; j++) {
+        byte [] qualifier = Bytes.readByteArray(in);
+        set.add(qualifier);
+      }
+      this.familyMap.put(family, set);
+    }
+  }  
+
+  public void write(final DataOutput out)
+  throws IOException {
+    Bytes.writeByteArray(out, this.startRow);
+    Bytes.writeByteArray(out, this.stopRow);
+    out.writeInt(this.maxVersions);
+    if(this.filter == null) {
+      out.writeBoolean(false);
+    } else {
+      out.writeBoolean(true);
+      HbaseObjectWritable.writeObject(out, this.filter, 
+          Filter.class, null);
+    }
+    if (this.oldFilter == null) {
+      out.writeBoolean(false);
+    } else {
+      out.writeBoolean(true);
+      HbaseObjectWritable.writeObject(out, this.oldFilter,
+          RowFilterInterface.class, null);
+    }
+    tr.write(out);
+    out.writeInt(familyMap.size());
+    for(Map.Entry<byte [], NavigableSet<byte []>> entry : familyMap.entrySet()) {
+      Bytes.writeByteArray(out, entry.getKey());
+      NavigableSet<byte []> columnSet = entry.getValue();
+      if(columnSet != null){
+        out.writeInt(columnSet.size());
+        for(byte [] qualifier : columnSet) {
+          Bytes.writeByteArray(out, qualifier);
+        }
+      } else {
+        out.writeInt(0);
+      }
+    }
+  }
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scanner.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scanner.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scanner.java Sat Jun  6 01:26:21 2009
@@ -1,5 +1,5 @@
 /**
- * Copyright 2008 The Apache Software Foundation
+ * Copyright 2009 The Apache Software Foundation
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -21,34 +21,32 @@
 
 import java.io.Closeable;
 import java.io.IOException;
+
 import org.apache.hadoop.hbase.io.RowResult;
 
 /**
  * Interface for client-side scanning.
  * Go to {@link HTable} to obtain instances.
+ * @deprecated See {@link ResultScanner}
  */
 public interface Scanner extends Closeable, Iterable<RowResult> {
   /**
-   * Grab the next row's worth of values. The scanner will return a RowResult
-   * that contains both the row's key and a map of byte[] column names to Cell 
-   * value objects. The data returned will only contain the most recent data 
-   * value for each row that is not newer than the target time passed when the
-   * scanner was created.
+   * Grab the next row's worth of values.
    * @return RowResult object if there is another row, null if the scanner is
    * exhausted.
    * @throws IOException
    */  
   public RowResult next() throws IOException;
-  
+ 
   /**
    * @param nbRows number of rows to return
-   * @return Between zero and <param>nbRows</param> RowResults
+   * @return Between zero and <param>nbRows</param> Results
    * @throws IOException
    */
   public RowResult [] next(int nbRows) throws IOException;
-  
+ 
   /**
    * Closes the scanner and releases any resources it has allocated
    */
-  public void close();  
+  public void close();
 }
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java Sat Jun  6 01:26:21 2009
@@ -23,37 +23,31 @@
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.filter.RowFilterInterface;
-import org.apache.hadoop.hbase.io.RowResult;
 
 
 /**
  * Retries scanner operations such as create, next, etc.
- * Used by {@link Scanner}s made by {@link HTable}.
+ * Used by {@link ResultScanner}s made by {@link HTable}.
  */
-public class ScannerCallable extends ServerCallable<RowResult[]> {
+public class ScannerCallable extends ServerCallable<Result[]> {
   private long scannerId = -1L;
   private boolean instantiated = false;
   private boolean closed = false;
-  private final byte [][] columns;
-  private final long timestamp;
-  private final RowFilterInterface filter;
+  private Scan scan;
+  private byte [] startRow;
   private int caching = 1;
 
   /**
    * @param connection
    * @param tableName
-   * @param columns
    * @param startRow
-   * @param timestamp
-   * @param filter
+   * @param scan
    */
-  public ScannerCallable (HConnection connection, byte [] tableName, byte [][] columns,
-      byte [] startRow, long timestamp, RowFilterInterface filter) {
+  public ScannerCallable (HConnection connection, byte [] tableName,
+      byte [] startRow, Scan scan) {
     super(connection, tableName, startRow);
-    this.columns = columns;
-    this.timestamp = timestamp;
-    this.filter = filter;
+    this.scan = scan;
+    this.startRow = startRow;
   }
   
   /**
@@ -71,7 +65,7 @@
   /**
    * @see java.util.concurrent.Callable#call()
    */
-  public RowResult[] call() throws IOException {
+  public Result [] call() throws IOException {
     if (scannerId != -1L && closed) {
       server.close(scannerId);
       scannerId = -1L;
@@ -79,28 +73,19 @@
       // open the scanner
       scannerId = openScanner();
     } else {
-      RowResult [] rrs = server.next(scannerId, caching);
-      return rrs.length == 0 ? null : rrs;
+      Result [] rrs = server.next(scannerId, caching);
+      return rrs == null || rrs.length == 0? null: rrs;
     }
     return null;
   }
   
   protected long openScanner() throws IOException {
     return server.openScanner(
-        this.location.getRegionInfo().getRegionName(), columns, row,
-        timestamp, filter);
+        this.location.getRegionInfo().getRegionName(), scan);
   }
   
-  protected byte [][] getColumns() {
-    return columns;
-  }
-  
-  protected long getTimestamp() {
-    return timestamp;
-  }
-  
-  protected RowFilterInterface getFilter() {
-    return filter;
+  protected Scan getScan() {
+    return scan;
   }
   
   /**

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHColumnDescriptor.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHColumnDescriptor.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHColumnDescriptor.java Sat Jun  6 01:26:21 2009
@@ -56,14 +56,6 @@
   }
 
   /**
-   * @see org.apache.hadoop.hbase.HColumnDescriptor#setMaxValueLength(int)
-   */
-  @Override
-  public void setMaxValueLength(int maxLength) {
-    throw new UnsupportedOperationException("HColumnDescriptor is read-only");
-  }
-
-  /**
    * @see org.apache.hadoop.hbase.HColumnDescriptor#setTimeToLive(int)
    */
   @Override

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java Sat Jun  6 01:26:21 2009
@@ -22,7 +22,7 @@
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
+//import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification;
 
 /**
  * Read-only table descriptor.
@@ -37,10 +37,14 @@
    * Create an unmodifyable copy of an HTableDescriptor
    * @param desc
    */
+//  UnmodifyableHTableDescriptor(final HTableDescriptor desc) {
+//    super(desc.getName(), getUnmodifyableFamilies(desc), desc.getIndexes(), desc.getValues());
+//  }
   UnmodifyableHTableDescriptor(final HTableDescriptor desc) {
-    super(desc.getName(), getUnmodifyableFamilies(desc), desc.getIndexes(), desc.getValues());
+    super(desc.getName(), getUnmodifyableFamilies(desc), desc.getValues());
   }
   
+  
   /*
    * @param desc
    * @return Families as unmodifiable array.
@@ -122,11 +126,11 @@
     throw new UnsupportedOperationException("HTableDescriptor is read-only");
   }
 
-  /**
-   * @see org.apache.hadoop.hbase.HTableDescriptor#addIndex(org.apache.hadoop.hbase.client.tableindexed.IndexSpecification)
-   */
-  @Override
-  public void addIndex(IndexSpecification index) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only"); 
-  }
+//  /**
+//   * @see org.apache.hadoop.hbase.HTableDescriptor#addIndex(org.apache.hadoop.hbase.client.tableindexed.IndexSpecification)
+//   */
+//  @Override
+//  public void addIndex(IndexSpecification index) {
+//    throw new UnsupportedOperationException("HTableDescriptor is read-only"); 
+//  }
 }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java Sat Jun  6 01:26:21 2009
@@ -1,152 +0,0 @@
-/**
- * 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.client.transactional;
-
-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.HRegionLocation;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
-import org.apache.hadoop.ipc.RemoteException;
-
-/**
- * Transaction Manager. Responsible for committing transactions.
- * 
- */
-public class TransactionManager {
-  static final Log LOG = LogFactory.getLog(TransactionManager.class);
-
-  private final HConnection connection;
-  private final TransactionLogger transactionLogger;
-
-  /**
-   * @param conf
-   */
-  public TransactionManager(final HBaseConfiguration conf) {
-    this(LocalTransactionLogger.getInstance(), conf);
-  }
-
-  /**
-   * @param transactionLogger
-   * @param conf
-   */
-  public TransactionManager(final TransactionLogger transactionLogger,
-      final HBaseConfiguration conf) {
-    this.transactionLogger = transactionLogger;
-    connection = HConnectionManager.getConnection(conf);
-  }
-
-  /**
-   * Called to start a transaction.
-   * 
-   * @return new transaction state
-   */
-  public TransactionState beginTransaction() {
-    long transactionId = transactionLogger.createNewTransactionLog();
-    LOG.debug("Begining transaction " + transactionId);
-    return new TransactionState(transactionId);
-  }
-
-  /**
-   * Try and commit a transaction.
-   * 
-   * @param transactionState
-   * @throws IOException
-   * @throws CommitUnsuccessfulException
-   */
-  public void tryCommit(final TransactionState transactionState)
-      throws CommitUnsuccessfulException, IOException {
-    LOG.debug("atempting to commit trasaction: " + transactionState.toString());
-
-    try {
-      for (HRegionLocation location : transactionState
-          .getParticipatingRegions()) {
-        TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection
-            .getHRegionConnection(location.getServerAddress());
-        boolean canCommit = transactionalRegionServer.commitRequest(location
-            .getRegionInfo().getRegionName(), transactionState
-            .getTransactionId());
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Region ["
-              + location.getRegionInfo().getRegionNameAsString() + "] votes "
-              + (canCommit ? "to commit" : "to abort") + " transaction "
-              + transactionState.getTransactionId());
-        }
-
-        if (!canCommit) {
-          LOG.debug("Aborting [" + transactionState.getTransactionId() + "]");
-          abort(transactionState, location);
-          throw new CommitUnsuccessfulException();
-        }
-      }
-
-      LOG.debug("Commiting [" + transactionState.getTransactionId() + "]");
-
-      transactionLogger.setStatusForTransaction(transactionState
-          .getTransactionId(), TransactionLogger.TransactionStatus.COMMITTED);
-
-      for (HRegionLocation location : transactionState
-          .getParticipatingRegions()) {
-        TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection
-            .getHRegionConnection(location.getServerAddress());
-        transactionalRegionServer.commit(location.getRegionInfo()
-            .getRegionName(), transactionState.getTransactionId());
-      }
-    } catch (RemoteException e) {
-      LOG.debug("Commit of transaction [" + transactionState.getTransactionId()
-          + "] was unsucsessful", e);
-      // FIXME, think about the what ifs
-      throw new CommitUnsuccessfulException(e);
-    }
-    // Tran log can be deleted now ...
-  }
-
-  /**
-   * Abort a s transaction.
-   * 
-   * @param transactionState
-   * @throws IOException
-   */
-  public void abort(final TransactionState transactionState) throws IOException {
-    abort(transactionState, null);
-  }
-
-  private void abort(final TransactionState transactionState,
-      final HRegionLocation locationToIgnore) throws IOException {
-    transactionLogger.setStatusForTransaction(transactionState
-        .getTransactionId(), TransactionLogger.TransactionStatus.ABORTED);
-
-    for (HRegionLocation location : transactionState.getParticipatingRegions()) {
-      if (locationToIgnore != null && location.equals(locationToIgnore)) {
-        continue;
-      }
-
-      TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection
-          .getHRegionConnection(location.getServerAddress());
-
-      transactionalRegionServer.abort(location.getRegionInfo().getRegionName(),
-          transactionState.getTransactionId());
-    }
-  }
-}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java Sat Jun  6 01:26:21 2009
@@ -33,12 +33,15 @@
 import org.apache.hadoop.io.ObjectWritable;
 
 /**
+ * This filter is a no-op in HBase 0.20.  Don't use it.
+ * 
  * This filter is used to filter based on the value of a given column. It takes
  * an operator (equal, greater, not equal, etc) and either a byte [] value or a
  * byte [] comparator. If we have a byte [] value then we just do a
  * lexicographic compare. If this is not sufficient (eg you want to deserialize
  * a long and then compare it to a fixed long value), then you can pass in your
  * own comparator instead.
+ * @deprecated Use filters that are rooted on @{link Filter} instead
  */
 public class ColumnValueFilter implements RowFilterInterface {
   /** Comparison operators. */

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/Filter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/Filter.java?rev=782178&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/Filter.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/Filter.java Sat Jun  6 01:26:21 2009
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.filter;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.hbase.KeyValue;
+
+/**
+ * Interface for row and column filters directly applied within the regionserver.
+ * A filter can expect the following call sequence:
+ *<ul>
+ * <li>{@link #reset()}</li>
+ * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
+ * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row, 
+ * if false, we will also call</li>
+ * <li>{@link #filterKeyValue(KeyValue)} -> true to drop this key/value</li>
+ * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
+ * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
+ * </li>
+ * </ul>
+ *
+ * Filter instances are created one per region/scan.  This interface replaces
+ * the old RowFilterInterface.
+ */
+public interface Filter extends Writable {
+  /**
+   * Reset the state of the filter between rows.
+   */
+  public void reset();
+
+  /**
+   * Filters a row based on the row key. If this returns true, the entire
+   * row will be excluded.  If false, each KeyValue in the row will be
+   * passed to {@link #filterKeyValue(KeyValue)} below.
+   *
+   * @param buffer buffer containing row key
+   * @param offset offset into buffer where row key starts
+   * @param length length of the row key
+   * @return true, remove entire row, false, include the row (maybe).
+   */
+  public boolean filterRowKey(byte [] buffer, int offset, int length);
+
+  /**
+   * If this returns true, the scan will terminate.
+   *
+   * @return true to end scan, false to continue.
+   */
+  public boolean filterAllRemaining();
+
+  /**
+   * A way to filter based on the column family, column qualifier and/or the
+   * column value. Return code is described below.  This allows filters to
+   * filter only certain number of columns, then terminate without matching ever
+   * column.
+   *
+   * If your filter returns <code>ReturnCode.NEXT_ROW</code>, it should return
+   * <code>ReturnCode.NEXT_ROW</code> until {@link #reset()} is called
+   * just in case the caller calls for the next row.
+   *
+   * @param v the KeyValue in question
+   * @return code as described below
+   * @see {@link Filter.ReturnCode}
+   */
+  public ReturnCode filterKeyValue(KeyValue v);
+
+  /**
+   * Return codes for filterValue().
+   */
+  public enum ReturnCode {
+    /**
+     * Include the KeyValue
+     */
+    INCLUDE,
+    /**
+     * Skip this KeyValue
+     */
+    SKIP,
+    /**
+     * Done with columns, skip to next row. Note that filterRow() will
+     * still be called.
+     */
+    NEXT_ROW,
+  };
+
+  /**
+   * Last chance to veto row based on previous {@link #filterKeyValue(KeyValue)}
+   * calls. The filter needs to retain state then return a particular value for
+   * this call if they wish to exclude a row if a certain column is missing
+   * (for example).
+   * @return true to exclude row, false to include row.
+   */
+  public boolean filterRow();
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java Sat Jun  6 01:26:21 2009
@@ -24,8 +24,10 @@
 /**
  * Subclass of StopRowFilter that filters rows > the stop row,
  * making it include up to the last row but no further.
+ *
+ * @deprecated Use filters that are rooted on @{link Filter} instead
  */
-public class InclusiveStopRowFilter extends StopRowFilter{
+public class InclusiveStopRowFilter extends StopRowFilter {
   /**
    * Default constructor, filters nothing. Required though for RPC
    * deserialization.
@@ -46,12 +48,17 @@
    */
   @Override
   public boolean filterRowKey(final byte [] rowKey) {
+    return filterRowKey(rowKey, 0, rowKey.length);
+  }
+
+  public boolean filterRowKey(byte []rowKey, int offset, int length) {
     if (rowKey == null) {
       if (getStopRowKey() == null) {
         return true;
       }
       return false;
-    }    
-    return Bytes.compareTo(getStopRowKey(), rowKey) < 0;
+    }
+    return Bytes.compareTo(getStopRowKey(), 0, getStopRowKey().length,
+        rowKey, offset, length) < 0;
   }
 }

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageFilter.java?rev=782178&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageFilter.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageFilter.java Sat Jun  6 01:26:21 2009
@@ -0,0 +1,92 @@
+/**
+ * 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.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+/**
+ * Implementation of Filter interface that limits results to a specific page
+ * size. It terminates scanning once the number of filter-passed results is >=
+ * the given page size.
+ * 
+ * <p>
+ * Note that this filter cannot guarantee that the number of results returned
+ * to a client are <= page size. This is because the filter is applied
+ * separately on different region servers. It does however optimize the scan of
+ * individual HRegions by making sure that the page size is never exceeded
+ * locally.
+ * </p>
+ */
+public class PageFilter implements Filter {
+  private long pageSize = Long.MAX_VALUE;
+  private int rowsAccepted = 0;
+
+  /**
+   * Default constructor, filters nothing. Required though for RPC
+   * deserialization.
+   */
+  public PageFilter() {
+    super();
+  }
+
+  /**
+   * Constructor that takes a maximum page size.
+   * 
+   * @param pageSize Maximum result size.
+   */
+  public PageFilter(final long pageSize) {
+    this.pageSize = pageSize;
+  }
+
+  public void reset() {
+    rowsAccepted = 0;
+  }
+
+  public boolean filterAllRemaining() {
+    return this.rowsAccepted >= this.pageSize;
+  }
+
+  public boolean filterRowKey(byte[] rowKey, int offset, int length) {
+    return filterAllRemaining();
+  }
+
+  public void readFields(final DataInput in) throws IOException {
+    this.pageSize = in.readLong();
+  }
+
+  public void write(final DataOutput out) throws IOException {
+    out.writeLong(pageSize);
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue v) {
+    this.rowsAccepted++;
+    return filterAllRemaining()? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE;
+  }
+
+  @Override
+  public boolean filterRow() {
+    return filterAllRemaining();
+  }
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java Sat Jun  6 01:26:21 2009
@@ -40,6 +40,8 @@
  * individual HRegions by making sure that the page size is never exceeded
  * locally.
  * </p>
+ *
+ * @deprecated Use filters that are rooted on @{link Filter} instead
  */
 public class PageRowFilter implements RowFilterInterface {
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PrefixRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PrefixRowFilter.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PrefixRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PrefixRowFilter.java Sat Jun  6 01:26:21 2009
@@ -31,6 +31,8 @@
 
 /**
  * RowFilterInterface that filters everything that does not match a prefix
+ *
+ * @deprecated Use filters that are rooted on @{link Filter} instead
  */
 public class PrefixRowFilter implements RowFilterInterface {
   protected byte[] prefix;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java Sat Jun  6 01:26:21 2009
@@ -30,7 +30,18 @@
  * 
  * Interface used for row-level filters applied to HRegion.HScanner scan
  * results during calls to next().
- * TODO: Make Filters use proper comparator comparing rows.
+ *
+ * In HBase 0.20, not all of the functions will be called, thus filters which depend
+ * on them will not work as advertised!
+ *
+ * Specifically, you can only count on the following methods to be called:
+ * boolean filterRowKey(final byte [] rowKey, final int offset, final int length);
+ * boolean filterAllRemaining();
+ *
+ * Complex filters that depend in more need to be rewritten to work with @{link Filter}
+ *
+ * Write new filters to use the @{link Filter} API instead.
+ * @deprecated Use filters that are rooted on @{link Filter} instead
  */
 public interface RowFilterInterface extends Writable {
   /**

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java Sat Jun  6 01:26:21 2009
@@ -38,6 +38,10 @@
  * which will be evaluated with a specified boolean operator MUST_PASS_ALL 
  * (!AND) or MUST_PASS_ONE (!OR).  Since you can use RowFilterSets as children 
  * of RowFilterSet, you can create a hierarchy of filters to be evaluated.
+ *
+ * It is highly likely this construct will no longer work!
+ *
+ * @deprecated Use filters that are rooted on @{link Filter} instead
  */
 public class RowFilterSet implements RowFilterInterface {
 

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowInclusiveStopFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowInclusiveStopFilter.java?rev=782178&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowInclusiveStopFilter.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowInclusiveStopFilter.java Sat Jun  6 01:26:21 2009
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.filter;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.DataInput;
+
+/**
+ * A Filter that stops after the given row.  There is no "RowStopFilter" because the Scan
+ * spec allows you to specify a stop row.
+ *
+ * Use this filter to include the stop row, eg: [A,Z].
+ */
+public class RowInclusiveStopFilter implements Filter {
+  private byte [] stopRowKey;
+
+  public RowInclusiveStopFilter() {
+    super();
+  }
+
+  public RowInclusiveStopFilter(final byte [] stopRowKey) {
+    this.stopRowKey = stopRowKey;
+  }
+
+  @Override
+  public void reset() {
+    // noop, no state
+  }
+
+  @Override
+  public boolean filterRowKey(byte[] buffer, int offset, int length) {
+    if (buffer == null) {
+      if (this.stopRowKey == null) {
+        return true; //filter...
+      }
+      return false;
+    }
+    // if stopRowKey is <= buffer, then true, filter row.
+    return Bytes.compareTo(stopRowKey, 0, stopRowKey.length, buffer, offset, length) < 0;
+  }
+
+  @Override
+  public boolean filterAllRemaining() {
+    return false;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue v) {
+    // include everything.
+    return ReturnCode.INCLUDE;
+  }
+
+  @Override
+  public boolean filterRow() {
+    return false;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    Bytes.writeByteArray(out, this.stopRowKey);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    this.stopRowKey = Bytes.readByteArray(in);
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowPrefixFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowPrefixFilter.java?rev=782178&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowPrefixFilter.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowPrefixFilter.java Sat Jun  6 01:26:21 2009
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.filter;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.DataInput;
+
+public class RowPrefixFilter implements Filter {
+
+  protected byte [] prefix;
+
+  public RowPrefixFilter(final byte [] prefix) {
+    this.prefix = prefix;
+  }
+
+  public RowPrefixFilter() {
+  }
+
+  @Override
+  public void reset() {
+  }
+
+  @Override
+  public boolean filterRowKey(byte[] buffer, int offset, int length) {
+    if (buffer == null)
+      return true;
+    if (length < prefix.length)
+      return true;
+    // if they are equal, return false => pass row
+    // else return true, filter row
+    return Bytes.compareTo(buffer, offset, prefix.length, prefix, 0, prefix.length) != 0;
+  }
+
+  @Override
+  public boolean filterAllRemaining() {
+    return false;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue v) {
+    return ReturnCode.INCLUDE;
+  }
+
+  @Override
+  public boolean filterRow() {
+    return false;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    Bytes.writeByteArray(out, prefix);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    prefix = Bytes.readByteArray(in);
+  }
+}

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowWhileMatchFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowWhileMatchFilter.java?rev=782178&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowWhileMatchFilter.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowWhileMatchFilter.java Sat Jun  6 01:26:21 2009
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.filter;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.DataInput;
+
+/**
+ * A wrapper filter that filters everything after the first filtered row.
+ */
+public class RowWhileMatchFilter implements Filter {
+  private boolean filterAllRemaining = false;
+  private Filter filter;
+
+  public RowWhileMatchFilter() {
+    super();
+  }
+
+  public RowWhileMatchFilter(Filter filter) {
+    this.filter = filter;
+  }
+
+  @Override
+  public void reset() {
+    // no state.
+  }
+
+  private void changeFAR(boolean value) {
+    filterAllRemaining = filterAllRemaining || value;
+  }
+
+  @Override
+  public boolean filterRowKey(byte[] buffer, int offset, int length) {
+    changeFAR(filter.filterRowKey(buffer, offset, length));
+    return filterAllRemaining();
+  }
+
+  @Override
+  public boolean filterAllRemaining() {
+    return this.filterAllRemaining || this.filter.filterAllRemaining();
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue v) {
+    ReturnCode c = filter.filterKeyValue(v);
+    changeFAR(c != ReturnCode.INCLUDE);
+    return c;
+  }
+
+  @Override
+  public boolean filterRow() {
+    return false;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(this.filter.getClass().getName());
+    this.filter.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    String className = in.readUTF();
+    try {
+      this.filter = (Filter)(Class.forName(className).newInstance());
+      this.filter.readFields(in);
+    } catch (InstantiationException e) {
+      throw new RuntimeException("Failed deserialize.", e);
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException("Failed deserialize.", e);
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException("Failed deserialize.", e);
+    }
+  }
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java Sat Jun  6 01:26:21 2009
@@ -32,6 +32,8 @@
 /**
  * Implementation of RowFilterInterface that filters out rows greater than or 
  * equal to a specified rowKey.
+ *
+ * @deprecated Use filters that are rooted on @{link Filter} instead
  */
 public class StopRowFilter implements RowFilterInterface {
   private byte [] stopRowKey;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java Sat Jun  6 01:26:21 2009
@@ -34,6 +34,8 @@
  * filter(..) methods or filterNotNull(SortedMap<Text, byte[]>), this wrapper's 
  * filterAllRemaining() will return true.  All filtering methods will 
  * thereafter defer to the result of filterAllRemaining().
+ *
+ * @deprecated Use filters that are rooted on @{link Filter} instead
  */
 public class WhileMatchRowFilter implements RowFilterInterface {
   private boolean filterAllRemaining = false;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/package-info.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/package-info.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/package-info.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/package-info.java Sat Jun  6 01:26:21 2009
@@ -17,7 +17,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-/**Provides row-level filters applied to HRegion scan results during calls to {@link org.apache.hadoop.hbase.client.Scanner#next()}. 
+/**Provides row-level filters applied to HRegion scan results during calls to {@link org.apache.hadoop.hbase.client.ResultScanner#next()}. 
 
 <p>Use {@link org.apache.hadoop.hbase.filter.StopRowFilter} to stop the scan once rows exceed the supplied row key.
 Filters will not stop the scan unless hosted inside of a {@link org.apache.hadoop.hbase.filter.WhileMatchRowFilter}.

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchOperation.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchOperation.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchOperation.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchOperation.java Sat Jun  6 01:26:21 2009
@@ -33,7 +33,8 @@
  * This object is purposely bare-bones because many instances are created
  * during bulk uploads.  We have one class for DELETEs and PUTs rather than
  * a class per type because it makes the serialization easier.
- * @see BatchUpdate 
+ * @see BatchUpdate
+ * @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API.
  */
 public class BatchOperation implements Writable, HeapSize {
   /**

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java Sat Jun  6 01:26:21 2009
@@ -42,6 +42,7 @@
  * There is one BatchUpdate object per server, so a series of batch operations
  * can result in multiple BatchUpdate objects if the batch contains rows that
  * are served by multiple region servers.
+ * @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API.
  */
 public class BatchUpdate
 implements WritableComparable<BatchUpdate>, Iterable<BatchOperation>, HeapSize {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Cell.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Cell.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Cell.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Cell.java Sat Jun  6 01:26:21 2009
@@ -46,6 +46,7 @@
  * stored with together as a result for get and getRow methods. This promotes
  * the timestamp of a cell to a first-class value, making it easy to take note
  * of temporal data. Cell is used all the way from HStore up to HTable.
+ * @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API.
  */
 public class Cell implements Writable, Iterable<Map.Entry<Long, byte[]>>,
     ISerializable {
@@ -228,7 +229,8 @@
    * TODO: This is the glue between old way of doing things and the new.
    * Herein we are converting our clean KeyValues to Map of Cells.
    */
-  public static HbaseMapWritable<byte [], Cell> createCells(final List<KeyValue> results) {
+  public static HbaseMapWritable<byte [], Cell> createCells(
+      final List<KeyValue> results) {
     HbaseMapWritable<byte [], Cell> cells =
       new HbaseMapWritable<byte [], Cell>();
     // Walking backward through the list of results though it has no effect
@@ -275,4 +277,4 @@
       throws HBaseRestException {
     serializer.serializeCell(this);
   }
-}
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/CodeToClassAndBack.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/CodeToClassAndBack.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/CodeToClassAndBack.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/CodeToClassAndBack.java Sat Jun  6 01:26:21 2009
@@ -45,7 +45,7 @@
   /**
    * Class list for supported classes
    */
-  public Class[] classList = {byte[].class, Cell.class};
+  public Class<?>[] classList = {byte[].class, Cell.class};
   
   /**
    * The static loader that is used instead of the static constructor in
@@ -58,8 +58,8 @@
    * Class that loads the static maps with their values. 
    */
   public class InternalStaticLoader{
-    InternalStaticLoader(Class[] classList, Map<Byte, Class<?>> CODE_TO_CLASS,
-    Map<Class<?>, Byte> CLASS_TO_CODE){
+    InternalStaticLoader(Class<?>[] classList,
+        Map<Byte,Class<?>> CODE_TO_CLASS, Map<Class<?>, Byte> CLASS_TO_CODE){
       byte code = 1;
       for(int i=0; i<classList.length; i++){
         CLASS_TO_CODE.put(classList[i], code);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java Sat Jun  6 01:26:21 2009
@@ -32,11 +32,18 @@
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HMsg;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HServerAddress;
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.RowFilterInterface;
 import org.apache.hadoop.hbase.filter.RowFilterSet;
 import org.apache.hadoop.hbase.io.HbaseMapWritable;
@@ -47,6 +54,7 @@
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /** 
@@ -89,52 +97,49 @@
     addToMap(Float.TYPE, code++);
     addToMap(Double.TYPE, code++);
     addToMap(Void.TYPE, code++);
+    
     // Other java types
     addToMap(String.class, code++);
     addToMap(byte [].class, code++);
     addToMap(byte [][].class, code++);
+    
     // Hadoop types
     addToMap(Text.class, code++);
     addToMap(Writable.class, code++);
     addToMap(Writable [].class, code++);
     addToMap(HbaseMapWritable.class, code++);
     addToMap(NullInstance.class, code++);
-    try {
-      addToMap(Class.forName("[Lorg.apache.hadoop.io.Text;"), code++);
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-    }
+
     // Hbase types
-    addToMap(HServerInfo.class, code++);
-    addToMap(HMsg.class, code++);
-    addToMap(HTableDescriptor.class, code++);
     addToMap(HColumnDescriptor.class, code++);
+    addToMap(HConstants.Modify.class, code++);
+    addToMap(HMsg.class, code++);
+    addToMap(HMsg[].class, code++);
     addToMap(RowFilterInterface.class, code++);
     addToMap(RowFilterSet.class, code++);
+    addToMap(HRegion.class, code++);
+    addToMap(HRegion[].class, code++);
     addToMap(HRegionInfo.class, code++);
-    addToMap(BatchUpdate.class, code++);
-    addToMap(HServerAddress.class, code++);
-    try {
-      addToMap(Class.forName("[Lorg.apache.hadoop.hbase.HMsg;"), code++);
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-    }
-    addToMap(Cell.class, code++);
-    try {
-      addToMap(Class.forName("[Lorg.apache.hadoop.hbase.io.Cell;"), code++);
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-    }
-    addToMap(RowResult.class, code++);
     addToMap(HRegionInfo[].class, code++);
+    addToMap(HServerAddress.class, code++);
+    addToMap(HServerInfo.class, code++);
+    addToMap(HTableDescriptor.class, code++);
     addToMap(MapWritable.class, code++);
-    try {
-      addToMap(Class.forName("[Lorg.apache.hadoop.hbase.io.RowResult;"), code++);
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-    }
-    addToMap(BatchUpdate[].class, code++);
+    
+    //
+    // HBASE-880
+    //
     addToMap(ClusterStatus.class, code++);
+    addToMap(Delete.class, code++);
+    addToMap(Get.class, code++);
+    addToMap(KeyValue.class, code++);
+    addToMap(KeyValue[].class, code++);
+    addToMap(Put.class, code++);
+    addToMap(Put[].class, code++);
+    addToMap(Result.class, code++);
+    addToMap(Result[].class, code++);
+    addToMap(Scan.class, code++);
+    
   }
   
   private Class<?> declaredClass;
@@ -228,6 +233,12 @@
     Byte code = CLASS_TO_CODE.get(c);
     if (code == null) {
       LOG.error("Unsupported type " + c);
+      StackTraceElement[] els = new Exception().getStackTrace();
+      for(StackTraceElement elem : els) {
+        LOG.error(elem.getMethodName());
+      }
+//          new Exception().getStackTrace()[0].getMethodName());
+//      throw new IOException(new Exception().getStackTrace()[0].getMethodName());
       throw new UnsupportedOperationException("No code for unexpected " + c);
     }
     out.writeByte(code);
@@ -261,6 +272,8 @@
       // byte-at-a-time we were previously doing.
       if (declClass.equals(byte [].class)) {
         Bytes.writeByteArray(out, (byte [])instanceObj);
+      } else if(declClass.equals(Result [].class)) {
+        Result.writeArray(out, (Result [])instanceObj);
       } else {
         int length = Array.getLength(instanceObj);
         out.writeInt(length);
@@ -363,6 +376,8 @@
     } else if (declaredClass.isArray()) {              // array
       if (declaredClass.equals(byte [].class)) {
         instance = Bytes.readByteArray(in);
+      } else if(declaredClass.equals(Result [].class)) {
+        instance = Result.readArray(in);
       } else {
         int length = in.readInt();
         instance = Array.newInstance(declaredClass.getComponentType(), length);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HeapSize.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HeapSize.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HeapSize.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HeapSize.java Sat Jun  6 01:26:21 2009
@@ -21,11 +21,24 @@
 
 /**
  * Implementations can be asked for an estimate of their size in bytes.
+ * <p>
  * Useful for sizing caches.  Its a given that implementation approximations
- * probably do not account for 32 vs 64 bit nor for different VM implemenations.
+ * do not account for 32 vs 64 bit nor for different VM implementations.
+ * <p>
+ * An Object's size is determined by the non-static data members in it,
+ * as well as the fixed {@link OBJECT} overhead.
+ * <p>
+ * For example:
+ * <pre>
+ * public class SampleObject implements HeapSize {
+ *   
+ *   int [] numbers;
+ *   int x;
+ * }
+ * </pre>
  */
 public interface HeapSize {
-  
+
   /** Reference size is 8 bytes on 64-bit, 4 bytes on 32-bit */
   static final int REFERENCE = 8;
   
@@ -49,10 +62,12 @@
   static final int LONG = 8;
   
   /** Array overhead */
-  static final int BYTE_ARRAY = REFERENCE;
   static final int ARRAY = 3 * REFERENCE;
   static final int MULTI_ARRAY = (4 * REFERENCE) + ARRAY;
   
+  /** Byte arrays are fixed size below plus its length, 8 byte aligned */
+  static final int BYTE_ARRAY = 3 * REFERENCE;
+  
   static final int BLOCK_SIZE_TAX = 8;
 
   static final int BYTE_BUFFER = 56;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java Sat Jun  6 01:26:21 2009
@@ -122,10 +122,22 @@
     return this.length;
   }
  
+  /**
+   * @return the current length of the buffer. same as getSize()
+   */
+  //Should probably deprecate getSize() so that we keep the same calls for all
+  //byte []
   public int getLength() {
-    return getSize();
+    if (this.bytes == null) {
+      throw new IllegalStateException("Uninitialiized. Null constructor " +
+        "called w/o accompaying readFields invocation");
+    }
+    return this.length;
   }
-
+  
+  /**
+   * @return offset
+   */
   public int getOffset(){
     return this.offset;
   }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Reference.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Reference.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Reference.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/Reference.java Sat Jun  6 01:26:21 2009
@@ -65,10 +65,17 @@
     this(null, Range.bottom);
   }
 
+  /**
+   * 
+   * @return Range
+   */
   public Range getFileRegion() {
     return this.region;
   }
 
+  /**
+   * @return splitKey
+   */
   public byte [] getSplitKey() {
     return splitkey;
   }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/RowResult.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/RowResult.java?rev=782178&r1=782177&r2=782178&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/RowResult.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/RowResult.java Sat Jun  6 01:26:21 2009
@@ -32,7 +32,6 @@
 import java.util.SortedMap;
 import java.util.TreeSet;
 
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.rest.descriptors.RestCell;
 import org.apache.hadoop.hbase.rest.exception.HBaseRestException;
@@ -46,11 +45,13 @@
 
 /**
  * Holds row name and then a map of columns to cells.
+ * @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API.
  */
 public class RowResult implements Writable, SortedMap<byte [], Cell>,
   Comparable<RowResult>, ISerializable {
   private byte [] row = null;
   private final HbaseMapWritable<byte [], Cell> cells;
+  private final byte [] COL_REGIONINFO = Bytes.toBytes("info:regioninfo");
 
   /** default constructor for writable */
   public RowResult() {
@@ -102,6 +103,11 @@
     return cells.containsKey(key);
   }
   
+  /**
+   * Check if the key can be found in this RowResult
+   * @param key
+   * @return true if key id found, false if not
+   */
   public boolean containsKey(String key) {
     return cells.containsKey(Bytes.toBytes(key));
   }
@@ -175,6 +181,16 @@
   public Cell get(String key) {
     return get(Bytes.toBytes(key));
   }
+
+  /**
+   * Get a cell using seperate family, columnQualifier arguments.
+   * @param family
+   * @param columnQualifier
+   * @return
+   */
+  public Cell get(byte [] family, byte [] columnQualifier) {
+    return get(Bytes.add(family, KeyValue.COLUMN_FAMILY_DELIM_ARRAY, columnQualifier));
+  }
   
 
   public Comparator<? super byte[]> comparator() {
@@ -245,7 +261,7 @@
       sb.append(Long.toString(e.getValue().getTimestamp()));
       sb.append(", value=");
       byte [] v = e.getValue().getValue();
-      if (Bytes.equals(e.getKey(), HConstants.COL_REGIONINFO)) {
+      if (Bytes.equals(e.getKey(), this.COL_REGIONINFO)) {
         try {
           sb.append(Writables.getHRegionInfo(v).toString());
         } catch (IOException ioe) {



Mime
View raw message