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

svn commit: r764289 [7/8] - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/client/ src/java/org/apache/hadoop/hbase/filter/ src/java/org/apache/hadoop/hbase/io/ src/java/org/apache/hadoop/hbase/io/hfile/ s...

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRegExpRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRegExpRowFilter.java?rev=764289&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRegExpRowFilter.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRegExpRowFilter.java Sun Apr 12 10:39:55 2009
@@ -0,0 +1,200 @@
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.UnsupportedEncodingException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.regionserver.HLogEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Tests for regular expression row filter
+ */
+public class DisabledTestRegExpRowFilter extends TestCase {
+  TreeMap<byte [], Cell> colvalues;
+  RowFilterInterface mainFilter;
+  static final char FIRST_CHAR = 'a';
+  static final char LAST_CHAR = 'e';
+  static final String HOST_PREFIX = "org.apache.site-";
+  static byte [] GOOD_BYTES = null;
+
+  static {
+    try {
+      GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING);
+    } catch (UnsupportedEncodingException e) {
+      fail();
+    }
+  }
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    this.colvalues = new TreeMap<byte [], Cell>(Bytes.BYTES_COMPARATOR);
+    for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
+      colvalues.put(Bytes.toBytes(new String(new char [] {c})),
+          new Cell(GOOD_BYTES, HConstants.LATEST_TIMESTAMP));
+    }
+    this.mainFilter = new RegExpRowFilter(HOST_PREFIX + ".*", colvalues);
+  }
+  
+  /**
+   * Tests filtering using a regex on the row key
+   * @throws Exception
+   */
+  public void testRegexOnRow() throws Exception {
+    regexRowTests(mainFilter);
+  }
+
+  /**
+   * Tests filtering using a regex on row and colum
+   * @throws Exception
+   */
+  public void testRegexOnRowAndColumn() throws Exception {
+    regexRowColumnTests(mainFilter);
+  }
+  
+  /**
+   * Only return values that are not null
+   * @throws Exception
+   */
+  public void testFilterNotNull() throws Exception {
+    filterNotNullTests(mainFilter);
+  }
+  
+  /**
+   * Test serialization
+   * @throws Exception
+   */
+  public void testSerialization() throws Exception {
+    // Decompose mainFilter to bytes.
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(stream);
+    mainFilter.write(out);
+    out.close();
+    byte[] buffer = stream.toByteArray();
+    
+    // Recompose filter.
+    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+    RowFilterInterface newFilter = new RegExpRowFilter();
+    newFilter.readFields(in);
+    
+    // Ensure the serialization preserved the filter by running all test.
+    regexRowTests(newFilter);
+    newFilter.reset();
+    regexRowColumnTests(newFilter);
+    newFilter.reset();
+    filterNotNullTests(newFilter);
+  }
+ 
+  private void regexRowTests(RowFilterInterface filter) throws Exception {
+    for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
+      byte [] t = createRow(c);
+      assertFalse("Failed with characer " + c, filter.filterRowKey(t));
+    }
+    String yahooSite = "com.yahoo.www";
+    assertTrue("Failed with character " +
+      yahooSite, filter.filterRowKey(Bytes.toBytes(yahooSite)));
+  }
+  
+  private void regexRowColumnTests(RowFilterInterface filter)
+    throws UnsupportedEncodingException {
+    
+    for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
+      byte [] t = createRow(c);
+      for (Map.Entry<byte [], Cell> e: this.colvalues.entrySet()) {
+        assertFalse("Failed on " + c,
+          filter.filterColumn(t, e.getKey(), e.getValue().getValue()));
+      }
+    }
+    // Try a row and column I know will pass.
+    char c = 'c';
+    byte [] r = createRow(c);
+    byte [] col = Bytes.toBytes(Character.toString(c));
+    assertFalse("Failed with character " + c,
+      filter.filterColumn(r, col, GOOD_BYTES));
+    
+    // Do same but with bad bytes.
+    assertTrue("Failed with character " + c,
+      filter.filterColumn(r, col, "badbytes".getBytes(HConstants.UTF8_ENCODING)));
+    
+    // Do with good bytes but bad column name.  Should not filter out.
+    assertFalse("Failed with character " + c,
+      filter.filterColumn(r, Bytes.toBytes("badcolumn"), GOOD_BYTES));
+    
+    // Good column, good bytes but bad row.
+    assertTrue("Failed with character " + c,
+      filter.filterColumn(Bytes.toBytes("bad row"),
+        Bytes.toBytes("badcolumn"), GOOD_BYTES));
+  }
+ 
+  private void filterNotNullTests(RowFilterInterface filter) throws Exception {
+    // Modify the filter to expect certain columns to be null:
+    // Expecting a row WITH columnKeys: a-d, WITHOUT columnKey: e
+    ((RegExpRowFilter)filter).setColumnFilter(new byte [] {LAST_CHAR}, null);
+    
+    char secondToLast = (char)(LAST_CHAR - 1);
+    char thirdToLast = (char)(LAST_CHAR - 2);
+    
+    // Modify the row to be missing an expected columnKey (d)
+    colvalues.remove(new byte [] {(byte)secondToLast});
+
+    // Try a row that is missing an expected columnKey.
+    // Testing row with columnKeys: a-c
+    assertTrue("Failed with last columnKey " + thirdToLast, filter.
+      filterRow(colvalues));
+
+    // Try a row that has all expected columnKeys, and NO null-expected
+    // columnKeys.
+    // Testing row with columnKeys: a-d
+    colvalues.put(new byte [] {(byte)secondToLast},
+        new Cell(GOOD_BYTES, HConstants.LATEST_TIMESTAMP));
+    assertFalse("Failed with last columnKey " + secondToLast, filter.
+      filterRow(colvalues));
+
+    // Try a row that has all expected columnKeys AND a null-expected columnKey.
+    // Testing row with columnKeys: a-e
+    colvalues.put(new byte [] {LAST_CHAR},
+        new Cell(GOOD_BYTES, HConstants.LATEST_TIMESTAMP));
+    assertTrue("Failed with last columnKey " + LAST_CHAR, filter.
+      filterRow(colvalues));
+    
+    // Try a row that has all expected columnKeys and a null-expected columnKey 
+    // that maps to a null value.
+    // Testing row with columnKeys: a-e, e maps to null
+    colvalues.put(new byte [] {LAST_CHAR}, 
+      new Cell(HLogEdit.DELETED_BYTES, HConstants.LATEST_TIMESTAMP));
+    assertFalse("Failed with last columnKey " + LAST_CHAR + " mapping to null.", 
+      filter.filterRow(colvalues));
+  }
+
+  private byte [] createRow(final char c) {
+    return Bytes.toBytes(HOST_PREFIX + Character.toString(c));
+  }
+}

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java?rev=764289&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java Sun Apr 12 10:39:55 2009
@@ -0,0 +1,201 @@
+/**
+ * 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.filter;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import junit.framework.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseClusterTestCase;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Scanner;
+import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/** Test regexp filters HBASE-476 */
+public class DisabledTestRowFilterAfterWrite extends HBaseClusterTestCase {
+
+  private static final Log LOG = LogFactory.getLog(DisabledTestRowFilterAfterWrite.class.getName());
+
+  static final String TABLE_NAME = "TestTable";
+  static final String FAMILY = "C:";
+  static final String COLUMN1 = FAMILY + "col1";
+  static final byte [] TEXT_COLUMN1 = Bytes.toBytes(COLUMN1);
+  static final String COLUMN2 = FAMILY + "col2";
+  static final byte [] TEXT_COLUMN2 = Bytes.toBytes(COLUMN2);
+
+  private static final byte [][] columns = {
+    TEXT_COLUMN1, TEXT_COLUMN2
+  };
+
+  private static final int NUM_ROWS = 10;
+  private static final int VALUE_SIZE = 1000;
+  private static final byte[] VALUE = new byte[VALUE_SIZE];
+  private static final int COL_2_SIZE = 5;
+  private static final int KEY_SIZE = 9;
+  private static final int NUM_REWRITES = 10;
+  static {
+    Arrays.fill(VALUE, (byte) 'a');
+  }
+
+  /** constructor */
+  public DisabledTestRowFilterAfterWrite() {
+    super();
+
+    // Make sure the cache gets flushed so we get multiple stores
+    conf.setInt("hbase.hregion.memcache.flush.size", (NUM_ROWS * (VALUE_SIZE + COL_2_SIZE + KEY_SIZE)));
+    LOG.info("memcach flush : " + conf.get("hbase.hregion.memcache.flush.size"));
+    conf.setInt("hbase.regionserver.optionalcacheflushinterval", 100000000);
+    // Avoid compaction to keep multiple stores
+    conf.setInt("hbase.hstore.compactionThreshold", 10000);
+
+    // Make lease timeout longer, lease checks less frequent
+    conf.setInt("hbase.master.lease.period", 10 * 1000);
+
+    // For debugging
+    conf.setInt("hbase.regionserver.lease.period", 20 * 60 * 1000);
+    conf.setInt("ipc.client.timeout", 20 * 60 * 1000);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+  }
+
+  /**
+   * Test hbase mapreduce jobs against single region and multi-region tables.
+   * 
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void testAfterWrite() throws IOException, InterruptedException {
+    singleTableTest();
+  }
+
+  /*
+   * Test against a single region. @throws IOException
+   */
+  private void singleTableTest() throws IOException, InterruptedException {
+    HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
+    desc.addFamily(new HColumnDescriptor(FAMILY));
+
+    // Create a table.
+    HBaseAdmin admin = new HBaseAdmin(this.conf);
+    admin.createTable(desc);
+
+    // insert some data into the test table
+    HTable table = new HTable(conf, TABLE_NAME);
+
+    for (int i = 0; i < NUM_ROWS; i++) {
+      BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i));
+
+      b.put(TEXT_COLUMN1, VALUE);
+      b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes());
+      table.commit(b);
+    }
+
+    // LOG.info("Print table contents using scanner before map/reduce for " + TABLE_NAME);
+    // scanTable(TABLE_NAME, false);
+    // LOG.info("Print table contents using scanner+filter before map/reduce for " + TABLE_NAME);
+    // scanTableWithRowFilter(TABLE_NAME, false);
+
+    // Do some identity write operations on one column of the data.
+    for (int n = 0; n < NUM_REWRITES; n++) {
+      for (int i = 0; i < NUM_ROWS; i++) {
+        BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i));
+
+        b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes());
+        table.commit(b);
+      }
+    }
+
+    // Wait for the flush to happen
+    LOG.info("Waiting, for flushes to complete");
+    Thread.sleep(5 * 1000);
+    // Wait for the flush to happen
+    LOG.info("Done. No flush should happen after this");
+
+    // Do another round so to populate the mem cache
+    for (int i = 0; i < NUM_ROWS; i++) {
+      BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i));
+      b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes());
+      table.commit(b);
+    }
+
+    LOG.info("Print table contents using scanner after map/reduce for " + TABLE_NAME);
+    scanTable(TABLE_NAME, true);
+    LOG.info("Print table contents using scanner+filter after map/reduce for " + TABLE_NAME);
+    scanTableWithRowFilter(TABLE_NAME, true);
+  }
+
+  private void scanTable(final String tableName, final boolean printValues) throws IOException {
+    HTable table = new HTable(conf, tableName);
+
+    Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW);
+    int numFound = doScan(scanner, printValues);
+    Assert.assertEquals(NUM_ROWS, numFound);
+  }
+
+  private void scanTableWithRowFilter(final String tableName, final boolean printValues) throws IOException {
+    HTable table = new HTable(conf, tableName);
+    Map<byte [], Cell> columnMap = new HashMap<byte [], Cell>();
+    columnMap.put(TEXT_COLUMN1,
+        new Cell(VALUE, HConstants.LATEST_TIMESTAMP));
+    RegExpRowFilter filter = new RegExpRowFilter(null, columnMap);
+    Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW, filter);
+    int numFound = doScan(scanner, printValues);
+    Assert.assertEquals(NUM_ROWS, numFound);
+  }
+
+  private int doScan(final Scanner scanner, final boolean printValues) throws IOException {
+    {
+      int count = 0;
+
+      try {
+        for (RowResult result : scanner) {
+          if (printValues) {
+            LOG.info("row: " + Bytes.toString(result.getRow()));
+            for (Map.Entry<byte [], Cell> e : result.entrySet()) {
+              LOG.info(" column: " + e.getKey() + " value: "
+                  + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING));
+            }
+          }
+          count++;
+        }
+
+      } finally {
+        scanner.close();
+      }
+      return count;
+    }
+  }
+}

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterOnMultipleFamilies.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterOnMultipleFamilies.java?rev=764289&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterOnMultipleFamilies.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterOnMultipleFamilies.java Sun Apr 12 10:39:55 2009
@@ -0,0 +1,128 @@
+/**
+ * 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.filter;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import junit.framework.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseClusterTestCase;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Scanner;
+import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Test for regexp filters (HBASE-527)
+ */
+public class DisabledTestRowFilterOnMultipleFamilies extends HBaseClusterTestCase {
+  private static final Log LOG = LogFactory.getLog(DisabledTestRowFilterOnMultipleFamilies.class.getName());
+
+  static final String TABLE_NAME = "TestTable";
+  static final String COLUMN1 = "A:col1";
+  static final byte [] TEXT_COLUMN1 = Bytes.toBytes(COLUMN1);
+  static final String COLUMN2 = "B:col2";
+  static final byte [] TEXT_COLUMN2 = Bytes.toBytes(COLUMN2);
+
+  private static final byte [][] columns = {TEXT_COLUMN1, TEXT_COLUMN2};
+
+  private static final int NUM_ROWS = 10;
+  private static final byte[] VALUE = "HELLO".getBytes();
+
+  /** @throws IOException */
+  public void testMultipleFamilies() throws IOException {
+    HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
+    desc.addFamily(new HColumnDescriptor("A:"));
+    desc.addFamily(new HColumnDescriptor("B:"));
+
+    // Create a table.
+    HBaseAdmin admin = new HBaseAdmin(this.conf);
+    admin.createTable(desc);
+
+    // insert some data into the test table
+    HTable table = new HTable(conf, TABLE_NAME);
+
+    for (int i = 0; i < NUM_ROWS; i++) {
+      BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i));
+      b.put(TEXT_COLUMN1, VALUE);
+      b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes());
+      table.commit(b);
+    }
+
+    LOG.info("Print table contents using scanner before map/reduce for " + TABLE_NAME);
+    scanTable(TABLE_NAME, true);
+    LOG.info("Print table contents using scanner+filter before map/reduce for " + TABLE_NAME);
+    scanTableWithRowFilter(TABLE_NAME, true);
+  }
+
+  private void scanTable(final String tableName, final boolean printValues) throws IOException {
+    HTable table = new HTable(conf, tableName);
+    Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW);
+    int numFound = doScan(scanner, printValues);
+    Assert.assertEquals(NUM_ROWS, numFound);
+  }
+
+  private void scanTableWithRowFilter(final String tableName, final boolean printValues) throws IOException {
+    HTable table = new HTable(conf, tableName);
+    Map<byte [], Cell> columnMap = new HashMap<byte [], Cell>();
+    columnMap.put(TEXT_COLUMN1,
+        new Cell(VALUE, HConstants.LATEST_TIMESTAMP));
+    RegExpRowFilter filter = new RegExpRowFilter(null, columnMap);
+    Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW, filter);
+    int numFound = doScan(scanner, printValues);
+    Assert.assertEquals(NUM_ROWS, numFound);
+  }
+
+  private int doScan(final Scanner scanner, final boolean printValues) throws IOException {
+    {
+      int count = 0;
+
+      try {
+        for (RowResult result : scanner) {
+          if (printValues) {
+            LOG.info("row: " + Bytes.toString(result.getRow()));
+
+            for (Map.Entry<byte [], Cell> e : result.entrySet()) {
+              LOG.info(" column: " + e.getKey() + " value: "
+                  + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING));
+            }
+          }
+          Assert.assertEquals(2, result.size());
+          count++;
+        }
+
+      } finally {
+        scanner.close();
+      }
+      return count;
+    }
+  }
+}

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterSet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterSet.java?rev=764289&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterSet.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterSet.java Sun Apr 12 10:39:55 2009
@@ -0,0 +1,188 @@
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+import junit.framework.TestCase;
+
+/**
+ * Tests filter sets
+ */
+public class DisabledTestRowFilterSet extends TestCase {
+
+  RowFilterInterface filterMPALL;
+  RowFilterInterface filterMPONE;
+  static final int MAX_PAGES = 5;
+  static final char FIRST_CHAR = 'a';
+  static final char LAST_CHAR = 'e';
+  TreeMap<byte [], Cell> colvalues;
+  static byte[] GOOD_BYTES = null;
+  static byte[] BAD_BYTES = null;
+
+  static {
+    try {
+      GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING);
+      BAD_BYTES = "def".getBytes(HConstants.UTF8_ENCODING);
+    } catch (UnsupportedEncodingException e) {
+      fail();
+    }
+  }
+  
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    
+    colvalues = new TreeMap<byte [], Cell>(Bytes.BYTES_COMPARATOR);
+    for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
+      colvalues.put(new byte [] {(byte)c},
+          new Cell(GOOD_BYTES, HConstants.LATEST_TIMESTAMP));
+    }
+    
+    Set<RowFilterInterface> filters = new HashSet<RowFilterInterface>();
+    filters.add(new PageRowFilter(MAX_PAGES));
+    filters.add(new RegExpRowFilter(".*regex.*", colvalues));
+    filters.add(new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("yyy"))));
+    filters.add(new WhileMatchRowFilter(new RegExpRowFilter(".*match.*")));
+    filterMPALL = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL, 
+      filters);
+    filterMPONE = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ONE, 
+      filters);
+  }
+  
+  /**
+   * Test "must pass one"
+   * @throws Exception
+   */
+  public void testMPONE() throws Exception {
+    MPONETests(filterMPONE);
+  }
+
+  /**
+   * Test "must pass all"
+   * @throws Exception
+   */
+  public void testMPALL() throws Exception {
+    MPALLTests(filterMPALL);
+  }
+  
+  /**
+   * Test serialization
+   * @throws Exception
+   */
+  public void testSerialization() throws Exception {
+    // Decompose filterMPALL to bytes.
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(stream);
+    filterMPALL.write(out);
+    out.close();
+    byte[] buffer = stream.toByteArray();
+    
+    // Recompose filterMPALL.
+    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+    RowFilterInterface newFilter = new RowFilterSet();
+    newFilter.readFields(in);
+    
+    // Ensure the serialization preserved the filter by running a full test.
+    MPALLTests(newFilter);
+  }
+  
+  private void MPONETests(RowFilterInterface filter) throws Exception {
+    // A row that shouldn't cause any filters to return true.
+    RFSAssertion(filter, "regex_match", false);
+    
+    // A row that should cause the WhileMatchRowFilter to filter all remaining.
+    RFSAssertion(filter, "regex_only", false);
+    
+    // Make sure the overall filterAllRemaining is unchanged (correct for 
+    // MUST_PASS_ONE).
+    assertFalse(filter.filterAllRemaining());
+    
+    // A row that should cause the RegExpRowFilter to fail and the 
+    // StopRowFilter to filter all remaining.
+    RFSAssertion(filter, "yyy_match", false);
+    
+    // Accept several more rows such that PageRowFilter will exceed its limit.
+    for (int i=0; i<=MAX_PAGES-3; i++)
+      filter.rowProcessed(false, Bytes.toBytes("unimportant_key"));
+    
+    // A row that should cause the RegExpRowFilter to filter this row, making 
+    // all the filters return true and thus the RowFilterSet as well.
+    RFSAssertion(filter, "bad_column", true);
+    
+    // Make sure the overall filterAllRemaining is unchanged (correct for 
+    // MUST_PASS_ONE).
+    assertFalse(filter.filterAllRemaining());
+  }
+  
+  private void MPALLTests(RowFilterInterface filter) throws Exception {
+    // A row that shouldn't cause any filters to return true.
+    RFSAssertion(filter, "regex_match", false);
+    
+    // A row that should cause WhileMatchRowFilter to filter all remaining.
+    RFSAssertion(filter, "regex_only", true);
+
+    // Make sure the overall filterAllRemaining is changed (correct for 
+    // MUST_PASS_ALL).
+    RFSAssertReset(filter);
+    
+    // A row that should cause the RegExpRowFilter to fail and the 
+    // StopRowFilter to filter all remaining.
+    RFSAssertion(filter, "yyy_match", true);
+
+    // Make sure the overall filterAllRemaining is changed (correct for 
+    // MUST_PASS_ALL).
+    RFSAssertReset(filter);
+    
+    // A row that should cause the RegExpRowFilter to fail.
+    boolean filtered = filter.filterColumn(Bytes.toBytes("regex_match"), 
+      new byte [] { FIRST_CHAR }, BAD_BYTES);
+    assertTrue("Filtering on 'regex_match' and bad column data.", filtered);
+    filterMPALL.rowProcessed(filtered, Bytes.toBytes("regex_match"));
+  }
+  
+  private void RFSAssertion(RowFilterInterface filter, String toTest, 
+    boolean assertTrue) throws Exception {
+    byte [] testText = Bytes.toBytes(toTest);
+    boolean filtered = filter.filterRowKey(testText);
+    assertTrue("Filtering on '" + toTest + "'", 
+      assertTrue? filtered : !filtered);
+    filter.rowProcessed(filtered, testText);
+  }
+  
+  private void RFSAssertReset(RowFilterInterface filter) throws Exception{
+    assertTrue(filter.filterAllRemaining());
+    // Reset for continued testing
+    filter.reset();
+    assertFalse(filter.filterAllRemaining());
+  }
+}

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestStopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestStopRowFilter.java?rev=764289&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestStopRowFilter.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestStopRowFilter.java Sun Apr 12 10:39:55 2009
@@ -0,0 +1,94 @@
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import junit.framework.TestCase;
+
+/**
+ * Tests the stop row filter
+ */
+public class DisabledTestStopRowFilter extends TestCase {
+  private final byte [] STOP_ROW = Bytes.toBytes("stop_row");
+  private final byte [] GOOD_ROW = Bytes.toBytes("good_row");
+  private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz");
+  
+  RowFilterInterface mainFilter;
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    mainFilter = new StopRowFilter(STOP_ROW);
+  }
+  
+  /**
+   * Tests identification of the stop row
+   * @throws Exception
+   */
+  public void testStopRowIdentification() throws Exception {
+    stopRowTests(mainFilter);
+  }
+  
+  /**
+   * Tests serialization
+   * @throws Exception
+   */
+  public void testSerialization() throws Exception {
+    // Decompose mainFilter to bytes.
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(stream);
+    mainFilter.write(out);
+    out.close();
+    byte[] buffer = stream.toByteArray();
+    
+    // Recompose mainFilter.
+    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+    RowFilterInterface newFilter = new StopRowFilter();
+    newFilter.readFields(in);
+    
+    // Ensure the serialization preserved the filter by running a full test.
+    stopRowTests(newFilter);
+  }
+  
+  private void stopRowTests(RowFilterInterface filter) throws Exception {
+    assertFalse("Filtering on " + Bytes.toString(GOOD_ROW), filter.filterRowKey(GOOD_ROW));
+    assertTrue("Filtering on " + Bytes.toString(STOP_ROW), filter.filterRowKey(STOP_ROW));
+    assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW), filter.filterRowKey(PAST_STOP_ROW));
+    
+    assertFalse("Filtering on " + Bytes.toString(GOOD_ROW), filter.filterColumn(GOOD_ROW, null, 
+      null));
+    assertTrue("Filtering on " + Bytes.toString(STOP_ROW), filter.filterColumn(STOP_ROW, null, null));
+    assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW), filter.filterColumn(PAST_STOP_ROW, 
+      null, null));
+
+    assertFalse("FilterAllRemaining", filter.filterAllRemaining());
+    assertFalse("FilterNotNull", filter.filterRow((List<KeyValue>)null));
+    
+    assertFalse("Filter a null", filter.filterRowKey(null));
+  }
+}
\ No newline at end of file

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestWhileMatchRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestWhileMatchRowFilter.java?rev=764289&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestWhileMatchRowFilter.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestWhileMatchRowFilter.java Sun Apr 12 10:39:55 2009
@@ -0,0 +1,151 @@
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import junit.framework.TestCase;
+
+
+/**
+ * Tests for the while-match filter
+ */
+public class DisabledTestWhileMatchRowFilter extends TestCase {
+
+  WhileMatchRowFilter wmStopRowFilter;
+  WhileMatchRowFilter wmRegExpRowFilter;
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    wmStopRowFilter = new WhileMatchRowFilter(new StopRowFilter(
+        Bytes.toBytes("s")));
+    wmRegExpRowFilter = new WhileMatchRowFilter(new RegExpRowFilter(
+    ".*regex.*"));
+  }
+  
+  /**
+   * Tests while match stop row
+   * @throws Exception
+   */
+  public void testWhileMatchStopRow() throws Exception {
+    whileMatchStopRowTests(wmStopRowFilter);
+  }
+  
+  /**
+   * Tests while match regex
+   * @throws Exception
+   */
+  public void testWhileMatchRegExp() throws Exception {
+    whileMatchRegExpTests(wmRegExpRowFilter);
+  }
+  
+  /**
+   * Tests serialization
+   * @throws Exception
+   */
+  public void testSerialization() throws Exception {
+    // Decompose wmRegExpRowFilter to bytes.
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(stream);
+    wmRegExpRowFilter.write(out);
+    out.close();
+    byte[] buffer = stream.toByteArray();
+    
+    // Recompose wmRegExpRowFilter.
+    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+    WhileMatchRowFilter newFilter = new WhileMatchRowFilter();
+    newFilter.readFields(in);
+
+    // Ensure the serialization preserved the filter by running a full test.
+    whileMatchRegExpTests(newFilter);
+  }
+  
+  private void whileMatchStopRowTests(WhileMatchRowFilter filter) throws 
+    Exception {
+    RowFilterInterface innerFilter = filter.getInternalFilter();
+    String toTest;
+    
+    // Test cases that should pass the row
+    toTest = "apples";
+    assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+        toTest)));
+    
+    // Test cases that should fail the row
+    toTest = "tuna";
+    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+        toTest)));
+    
+    // The difference in switch
+    assertTrue("filter: filterAllRemaining", filter.filterAllRemaining());
+    assertFalse("innerFilter: filterAllRemaining pre-reset", 
+      innerFilter.filterAllRemaining());
+    
+    // Test resetting
+    filter.reset();
+    assertFalse("filter: filterAllRemaining post-reset", 
+        filter.filterAllRemaining());
+    
+    // Test filterNotNull for functionality only (no switch-cases)
+    assertFalse("filter: filterNotNull", filter.filterRow((List<KeyValue>)null));
+  }
+  
+  private void whileMatchRegExpTests(WhileMatchRowFilter filter) throws 
+    Exception {
+    RowFilterInterface innerFilter = filter.getInternalFilter();
+    String toTest;
+    
+    // Test cases that should pass the row
+    toTest = "regex_match";
+    assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+        toTest)));
+    
+    // Test cases that should fail the row
+    toTest = "not_a_match";
+    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+        toTest)));
+    
+    // The difference in switch
+    assertTrue("filter: filterAllRemaining", filter.filterAllRemaining());
+    assertFalse("innerFilter: filterAllRemaining pre-reset", 
+      innerFilter.filterAllRemaining());
+    
+    // Test resetting
+    filter.reset();
+    assertFalse("filter: filterAllRemaining post-reset", 
+        filter.filterAllRemaining());
+    
+    // Test filter(Text, Text, byte[]) for functionality only (no switch-cases)
+    toTest = "asdf_regex_hjkl";
+    assertFalse("filter: '" + toTest + "'", filter.filterColumn(Bytes.toBytes(toTest), 
+      null, null));
+  }
+}

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/hfile/TestHFile.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/hfile/TestHFile.java?rev=764289&r1=764288&r2=764289&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/hfile/TestHFile.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/hfile/TestHFile.java Sun Apr 12 10:39:55 2009
@@ -35,10 +35,10 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HStoreKey;
 import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.RawComparator;
 
 /**
  * test hfile features.
@@ -243,7 +243,7 @@
     Path mFile = new Path(ROOT_DIR, "meta.tfile");
     FSDataOutputStream fout = createFSOutput(mFile);
     Writer writer = new Writer(fout, minBlockSize, null,
-      new HStoreKey.StoreKeyComparator() {
+      new RawComparator<byte []>() {
         @Override
         public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2,
             int l2) {

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java?rev=764289&r1=764288&r2=764289&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java Sun Apr 12 10:39:55 2009
@@ -129,16 +129,16 @@
     reader.loadFileInfo();
     System.out.println(reader.blockIndex.toString());
     // falls before the start of the file.
-    assertEquals(-1, reader.blockIndex.blockContainingKey(Bytes.toBytes("a")));
-    assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("c")));
-    assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("d")));
-    assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("e")));
-    assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("g")));
-    assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("h")));
-    assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("i")));
-    assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("j")));
-    assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("k")));
-    assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("l")));
+    assertEquals(-1, reader.blockIndex.blockContainingKey(Bytes.toBytes("a"), 0, 1));
+    assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("c"), 0, 1));
+    assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("d"), 0, 1));
+    assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("e"), 0, 1));
+    assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("g"), 0, 1));
+    assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("h"), 0, 1));
+    assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("i"), 0, 1));
+    assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("j"), 0, 1));
+    assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("k"), 0, 1));
+    assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("l"), 0, 1));
 
 
     

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/DisableTestCompaction.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/DisableTestCompaction.java?rev=764289&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/DisableTestCompaction.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/DisableTestCompaction.java Sun Apr 12 10:39:55 2009
@@ -0,0 +1,195 @@
+/**
+ * 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.regionserver;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestCase;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+/**
+ * Test compactions
+ */
+public class DisableTestCompaction extends HBaseTestCase {
+  static final Log LOG = LogFactory.getLog(DisableTestCompaction.class.getName());
+  private HRegion r = null;
+  private static final byte [] COLUMN_FAMILY = COLFAMILY_NAME1;
+  private final byte [] STARTROW = Bytes.toBytes(START_KEY);
+  private static final byte [] COLUMN_FAMILY_TEXT = COLUMN_FAMILY;
+  private static final byte [] COLUMN_FAMILY_TEXT_MINUS_COLON =
+    Bytes.toBytes(Bytes.toString(COLUMN_FAMILY).substring(0, COLUMN_FAMILY.length - 1));
+  private static final int COMPACTION_THRESHOLD = MAXVERSIONS;
+
+  private MiniDFSCluster cluster;
+  
+  /** constructor */
+  public DisableTestCompaction() {
+    super();
+    
+    // Set cache flush size to 1MB
+    conf.setInt("hbase.hregion.memcache.flush.size", 1024*1024);
+    conf.setInt("hbase.hregion.memcache.block.multiplier", 10);
+    this.cluster = null;
+  }
+  
+  @Override
+  public void setUp() throws Exception {
+    this.cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
+    // Make the hbase rootdir match the minidfs we just span up
+    this.conf.set(HConstants.HBASE_DIR,
+      this.cluster.getFileSystem().getHomeDirectory().toString());
+    super.setUp();
+    HTableDescriptor htd = createTableDescriptor(getName());
+    this.r = createNewHRegion(htd, null, null);
+  }
+  
+  @Override
+  public void tearDown() throws Exception {
+    HLog hlog = r.getLog();
+    this.r.close();
+    hlog.closeAndDelete();
+    if (this.cluster != null) {
+      shutdownDfs(cluster);
+    }
+    super.tearDown();
+  }
+  
+  /**
+   * Run compaction and flushing memcache
+   * Assert deletes get cleaned up.
+   * @throws Exception
+   */
+  public void testCompaction() throws Exception {
+    createStoreFile(r);
+    for (int i = 0; i < COMPACTION_THRESHOLD; i++) {
+      createStoreFile(r);
+    }
+    // Add more content.  Now there are about 5 versions of each column.
+    // Default is that there only 3 (MAXVERSIONS) versions allowed per column.
+    // Assert == 3 when we ask for versions.
+    addContent(new HRegionIncommon(r), Bytes.toString(COLUMN_FAMILY));
+    // FIX!!
+    Cell[] cellValues = 
+      Cell.createSingleCellArray(r.get(STARTROW, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/));
+    // Assert that I can get 3 versions since it is the max I should get
+    assertEquals(cellValues.length, 3);
+    r.flushcache();
+    r.compactStores();
+    // Always 3 versions if that is what max versions is.
+    byte [] secondRowBytes = START_KEY.getBytes(HConstants.UTF8_ENCODING);
+    // Increment the least significant character so we get to next row.
+    secondRowBytes[START_KEY_BYTES.length - 1]++;
+    // FIX
+    cellValues = Cell.createSingleCellArray(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100/*Too many*/));
+    LOG.info("Count of " + Bytes.toString(secondRowBytes) + ": " +
+      cellValues.length);
+    assertTrue(cellValues.length == 3);
+
+    // Now add deletes to memcache and then flush it.  That will put us over
+    // the compaction threshold of 3 store files.  Compacting these store files
+    // should result in a compacted store file that has no references to the
+    // deleted row.
+    r.deleteAll(secondRowBytes, COLUMN_FAMILY_TEXT, System.currentTimeMillis(),
+      null);
+    // Assert deleted.
+    assertNull(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/));
+    r.flushcache();
+    assertNull(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/));
+    // Add a bit of data and flush.  Start adding at 'bbb'.
+    createSmallerStoreFile(this.r);
+    r.flushcache();
+    // Assert that the second row is still deleted.
+    // FIX
+    cellValues = Cell.createSingleCellArray(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/));
+    assertNull(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/));
+    // Force major compaction.
+    r.compactStores(true);
+    assertEquals(r.getStore(COLUMN_FAMILY_TEXT).getStorefiles().size(), 1);
+    assertNull(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/));
+    // Make sure the store files do have some 'aaa' keys in them -- exactly 3.
+    // Also, that compacted store files do not have any secondRowBytes because
+    // they were deleted.
+    int count = 0;
+    boolean containsStartRow = false;
+    for (StoreFile f: this.r.stores.
+        get(Bytes.mapKey(COLUMN_FAMILY_TEXT_MINUS_COLON)).getStorefiles().values()) {
+      HFileScanner scanner = f.getReader().getScanner();
+      scanner.seekTo();
+      do {
+        HStoreKey key = HStoreKey.create(scanner.getKey());
+        if (Bytes.equals(key.getRow(), STARTROW)) {
+          containsStartRow = true;
+          count++;
+        } else {
+          // After major compaction, should be none of these rows in compacted
+          // file.
+          assertFalse(Bytes.equals(key.getRow(), secondRowBytes));
+        }
+      } while(scanner.next());
+    }
+    assertTrue(containsStartRow);
+    assertTrue(count == 3);
+    // Do a simple TTL test.
+    final int ttlInSeconds = 1;
+    for (Store store: this.r.stores.values()) {
+      store.ttl = ttlInSeconds * 1000;
+    }
+    Thread.sleep(ttlInSeconds * 1000);
+    r.compactStores(true);
+    count = count();
+    assertTrue(count == 0);
+  }
+  
+  private int count() throws IOException {
+    int count = 0;
+    for (StoreFile f: this.r.stores.
+        get(Bytes.mapKey(COLUMN_FAMILY_TEXT_MINUS_COLON)).getStorefiles().values()) {
+      HFileScanner scanner = f.getReader().getScanner();
+      if (!scanner.seekTo()) {
+        continue;
+      }
+      do {
+        count++;
+      } while(scanner.next());
+    }
+    return count;
+  }
+
+  private void createStoreFile(final HRegion region) throws IOException {
+    HRegionIncommon loader = new HRegionIncommon(region);
+    addContent(loader, Bytes.toString(COLUMN_FAMILY));
+    loader.flushcache();
+  }
+
+  private void createSmallerStoreFile(final HRegion region) throws IOException {
+    HRegionIncommon loader = new HRegionIncommon(region); 
+    addContent(loader, Bytes.toString(COLUMN_FAMILY),
+        ("bbb").getBytes(), null);
+    loader.flushcache();
+  }
+}

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/DisabledTestAtomicIncrement.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/DisabledTestAtomicIncrement.java?rev=764289&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/DisabledTestAtomicIncrement.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/DisabledTestAtomicIncrement.java Sun Apr 12 10:39:55 2009
@@ -0,0 +1,118 @@
+/**
+ * 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.regionserver;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseClusterTestCase;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public class DisabledTestAtomicIncrement extends HBaseClusterTestCase {
+  static final Log LOG = LogFactory.getLog(DisabledTestAtomicIncrement.class);
+
+  private static final byte [] CONTENTS = Bytes.toBytes("contents:");
+
+  public void testIncrement() throws IOException {
+    try {
+      HTable table = null;
+
+      // Setup
+
+      HTableDescriptor desc = new HTableDescriptor(getName());
+      desc.addFamily(
+          new HColumnDescriptor(CONTENTS,               // Column name
+              1,                                        // Max versions
+              HColumnDescriptor.DEFAULT_COMPRESSION,   // no compression
+              HColumnDescriptor.DEFAULT_IN_MEMORY,      // not in memory
+              HColumnDescriptor.DEFAULT_BLOCKCACHE,
+              HColumnDescriptor.DEFAULT_LENGTH,
+              HColumnDescriptor.DEFAULT_TTL,
+              false
+          )
+      );
+
+      // Create the table
+
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      admin.createTable(desc);
+
+      try {
+        // Give cache flusher and log roller a chance to run
+        // Otherwise we'll never hit the bloom filter, just the memcache
+        Thread.sleep(conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000) * 10);
+        
+      } catch (InterruptedException e) {
+        // ignore
+      }
+      // Open table
+
+      table = new HTable(conf, desc.getName());
+      
+      byte [] row = Bytes.toBytes("foo");
+      byte [] column = "contents:1".getBytes(HConstants.UTF8_ENCODING);
+      // increment by 1:
+      assertEquals(1L, table.incrementColumnValue(row, column, 1));
+      
+      // set a weird value, then increment:
+      row = Bytes.toBytes("foo2");
+      byte [] value = {0,0,1};
+      BatchUpdate bu = new BatchUpdate(row);
+      bu.put(column, value);
+      table.commit(bu);
+      
+      assertEquals(2L, table.incrementColumnValue(row, column, 1));
+
+      assertEquals(-2L, table.incrementColumnValue(row, column, -4));
+
+      row = Bytes.toBytes("foo3");
+      byte[] value2 = {1,2,3,4,5,6,7,8,9};
+      bu = new BatchUpdate(row);
+      bu.put(column, value2);
+      table.commit(bu);
+      
+      try {
+        table.incrementColumnValue(row, column, 1);
+        fail();
+      } catch (IOException e) {
+        System.out.println("Expected exception: " + e);
+        // expected exception.
+      }
+      
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      if (e instanceof IOException) {
+        IOException i = (IOException) e;
+        throw i;
+      }
+      fail();
+    }
+
+  }
+
+}

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java?rev=764289&r1=764288&r2=764289&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java Sun Apr 12 10:39:55 2009
@@ -73,7 +73,6 @@
       makeSureRegexWorks(region, region_incommon, false);
       // regex test hstore
       makeSureRegexWorks(region, region_incommon, true);
-      
     } finally {
       if (region != null) {
         try {
@@ -91,30 +90,32 @@
   throws Exception{
     // insert a few versions worth of data for a row
     byte [] row = Bytes.toBytes("test_row");
-    long t0 = System.currentTimeMillis();
-    long t1 = t0 - 15000;
-    long t2 = t1 - 15000;
+    long now = System.currentTimeMillis();
+    long past = now - 100;
+    long future = now + 100;
+    Thread.sleep(100);
+    LOG.info("now=" + now + ", past=" + past + ", future=" + future);
 
     byte [] colA = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a");
     byte [] colB = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b");
     byte [] colC = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "c");
     byte [] colD = Bytes.toBytes(Bytes.toString(COLUMNS[0]));
 
-    BatchUpdate batchUpdate = new BatchUpdate(row, t0);
+    BatchUpdate batchUpdate = new BatchUpdate(row, now);
     batchUpdate.put(colA, cellData(0, flush).getBytes());
     batchUpdate.put(colB, cellData(0, flush).getBytes());
     batchUpdate.put(colC, cellData(0, flush).getBytes());      
     batchUpdate.put(colD, cellData(0, flush).getBytes());      
     region_incommon.commit(batchUpdate);
 
-    batchUpdate = new BatchUpdate(row, t1);
+    batchUpdate = new BatchUpdate(row, past);
     batchUpdate.put(colA, cellData(1, flush).getBytes());
     batchUpdate.put(colB, cellData(1, flush).getBytes());
     batchUpdate.put(colC, cellData(1, flush).getBytes());      
     batchUpdate.put(colD, cellData(1, flush).getBytes());      
     region_incommon.commit(batchUpdate);
     
-    batchUpdate = new BatchUpdate(row, t2);
+    batchUpdate = new BatchUpdate(row, future);
     batchUpdate.put(colA, cellData(2, flush).getBytes());
     batchUpdate.put(colB, cellData(2, flush).getBytes());
     batchUpdate.put(colC, cellData(2, flush).getBytes());      
@@ -124,27 +125,27 @@
     if (flush) {region_incommon.flushcache();}
 
     // call delete all at a timestamp, make sure only the most recent stuff is left behind
-    region.deleteAll(row, t1, null);
+    region.deleteAll(row, now, null);
     if (flush) {region_incommon.flushcache();}    
-    assertCellEquals(region, row, colA, t0, cellData(0, flush));
-    assertCellEquals(region, row, colA, t1, null);
-    assertCellEquals(region, row, colA, t2, null);
-    assertCellEquals(region, row, colD, t0, cellData(0, flush));
-    assertCellEquals(region, row, colD, t1, null);
-    assertCellEquals(region, row, colD, t2, null);
+    assertCellEquals(region, row, colA, future, cellData(2, flush));
+    assertCellEquals(region, row, colA, past, null);
+    assertCellEquals(region, row, colA, now, null);
+    assertCellEquals(region, row, colD, future, cellData(2, flush));
+    assertCellEquals(region, row, colD, past, null);
+    assertCellEquals(region, row, colD, now, null);
 
     // call delete all w/o a timestamp, make sure nothing is left.
     region.deleteAll(row, HConstants.LATEST_TIMESTAMP, null);
     if (flush) {region_incommon.flushcache();}    
-    assertCellEquals(region, row, colA, t0, null);
-    assertCellEquals(region, row, colA, t1, null);
-    assertCellEquals(region, row, colA, t2, null);
-    assertCellEquals(region, row, colD, t0, null);
-    assertCellEquals(region, row, colD, t1, null);
-    assertCellEquals(region, row, colD, t2, null);
+    assertCellEquals(region, row, colA, now, null);
+    assertCellEquals(region, row, colA, past, null);
+    assertCellEquals(region, row, colA, future, null);
+    assertCellEquals(region, row, colD, now, null);
+    assertCellEquals(region, row, colD, past, null);
+    assertCellEquals(region, row, colD, future, null);
     
   }
-  
+
   private void makeSureRegexWorks(HRegion region, HRegionIncommon region_incommon, 
       boolean flush)
     throws Exception{

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java?rev=764289&r1=764288&r2=764289&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java Sun Apr 12 10:39:55 2009
@@ -20,22 +20,23 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
+import java.util.NavigableSet;
 import java.util.TreeSet;
 
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HStoreKey;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.filter.StopRowFilter;
 import org.apache.hadoop.hbase.filter.WhileMatchRowFilter;
 import org.apache.hadoop.hbase.io.BatchUpdate;
 import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 
 /**
  * {@link TestGet} is a medley of tests of get all done up as a single test.
@@ -62,6 +63,56 @@
   }
 
 
+  /**
+   * Test for HBASE-808 and HBASE-809.
+   * @throws Exception
+   */
+  public void testMaxVersionsAndDeleting() throws Exception {
+    HRegion region = null;
+    try {
+      HTableDescriptor htd = createTableDescriptor(getName());
+      region = createNewHRegion(htd, null, null);
+      
+      byte [] column = COLUMNS[0];
+      for (int i = 0; i < 100; i++) {
+        addToRow(region, T00, column, i, T00.getBytes());
+      }
+      checkVersions(region, T00, column);
+      // Flush and retry.
+      region.flushcache();
+      checkVersions(region, T00, column);
+      
+      // Now delete all then retry
+      region.deleteAll(Bytes.toBytes(T00), System.currentTimeMillis(), null);
+      Cell [] cells = Cell.createSingleCellArray(region.get(Bytes.toBytes(T00), column, -1,
+        HColumnDescriptor.DEFAULT_VERSIONS));
+      assertTrue(cells == null);
+      region.flushcache();
+      cells = Cell.createSingleCellArray(region.get(Bytes.toBytes(T00), column, -1,
+          HColumnDescriptor.DEFAULT_VERSIONS));
+      assertTrue(cells == null);
+      
+      // Now add back the rows
+      for (int i = 0; i < 100; i++) {
+        addToRow(region, T00, column, i, T00.getBytes());
+      }
+      // Run same verifications.
+      checkVersions(region, T00, column);
+      // Flush and retry.
+      region.flushcache();
+      checkVersions(region, T00, column);
+    } finally {
+      if (region != null) {
+        try {
+          region.close();
+        } catch (Exception e) {
+          e.printStackTrace();
+        }
+        region.getLog().closeAndDelete();
+      }
+    }
+  }
+
   public void testGetFullMultiMapfile() throws IOException {
     HRegion region = null;
     BatchUpdate batchUpdate = null;
@@ -84,7 +135,7 @@
       region.flushcache();
 
       // assert that getFull gives us the older value
-      results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
+      results = region.getFull(row, (NavigableSet<byte []>)null, LATEST_TIMESTAMP, 1, null);
       assertEquals("olderValue", new String(results.get(COLUMNS[0]).getValue()));
       
       // write a new value for the cell
@@ -96,7 +147,7 @@
       region.flushcache();
       
       // assert that getFull gives us the later value
-      results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
+      results = region.getFull(row, (NavigableSet<byte []>)null, LATEST_TIMESTAMP, 1, null);
       assertEquals("newerValue", new String(results.get(COLUMNS[0]).getValue()));
      
       //
@@ -117,7 +168,7 @@
       region.flushcache();
       
       // assert i get both columns
-      results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
+      results = region.getFull(row2, (NavigableSet<byte []>)null, LATEST_TIMESTAMP, 1, null);
       assertEquals("Should have two columns in the results map", 2, results.size());
       assertEquals("column0 value", new String(results.get(cell1).getValue()));
       assertEquals("column1 value", new String(results.get(cell2).getValue()));
@@ -132,7 +183,7 @@
       region.flushcache(); 
       
       // assert i get the second column only
-      results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
+      results = region.getFull(row2, (NavigableSet<byte []>)null, LATEST_TIMESTAMP, 1, null);
       System.out.println(Bytes.toString(results.keySet().iterator().next()));
       assertEquals("Should have one column in the results map", 1, results.size());
       assertNull("column0 value", results.get(cell1));
@@ -147,7 +198,7 @@
       region.batchUpdate(batchUpdate, null);
       
       // assert i get the third column only
-      results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
+      results = region.getFull(row2, (NavigableSet<byte []>)null, LATEST_TIMESTAMP, 1, null);
       assertEquals("Should have one column in the results map", 1, results.size());
       assertNull("column0 value", results.get(cell1));
       assertNull("column1 value", results.get(cell2));
@@ -232,56 +283,6 @@
     }
   }
 
-  /**
-   * Test for HBASE-808 and HBASE-809.
-   * @throws Exception
-   */
-  public void testMaxVersionsAndDeleting() throws Exception {
-    HRegion region = null;
-    try {
-      HTableDescriptor htd = createTableDescriptor(getName());
-      region = createNewHRegion(htd, null, null);
-      
-      byte [] column = COLUMNS[0];
-      for (int i = 0; i < 100; i++) {
-        addToRow(region, T00, column, i, T00.getBytes());
-      }
-      checkVersions(region, T00, column);
-      // Flush and retry.
-      region.flushcache();
-      checkVersions(region, T00, column);
-      
-      // Now delete all then retry
-      region.deleteAll(Bytes.toBytes(T00), System.currentTimeMillis(), null);
-      Cell [] cells = region.get(Bytes.toBytes(T00), column, -1,
-        HColumnDescriptor.DEFAULT_VERSIONS);
-      assertTrue(cells == null);
-      region.flushcache();
-      cells = region.get(Bytes.toBytes(T00), column, -1,
-          HColumnDescriptor.DEFAULT_VERSIONS);
-      assertTrue(cells == null);
-      
-      // Now add back the rows
-      for (int i = 0; i < 100; i++) {
-        addToRow(region, T00, column, i, T00.getBytes());
-      }
-      // Run same verifications.
-      checkVersions(region, T00, column);
-      // Flush and retry.
-      region.flushcache();
-      checkVersions(region, T00, column);
-    } finally {
-      if (region != null) {
-        try {
-          region.close();
-        } catch (Exception e) {
-          e.printStackTrace();
-        }
-        region.getLog().closeAndDelete();
-      }
-    }
-  }
-  
   private void addToRow(final HRegion r, final String row, final byte [] column,
       final long ts, final byte [] bytes)
   throws IOException {
@@ -294,11 +295,11 @@
       final byte [] column)
   throws IOException {
     byte [] r = Bytes.toBytes(row);
-    Cell [] cells = region.get(r, column, -1, 100);
+    Cell [] cells = Cell.createSingleCellArray(region.get(r, column, -1, 100));
     assertTrue(cells.length == HColumnDescriptor.DEFAULT_VERSIONS);
-    cells = region.get(r, column, -1, 1);
+    cells = Cell.createSingleCellArray(region.get(r, column, -1, 1));
     assertTrue(cells.length == 1);
-    cells = region.get(r, column, -1, HConstants.ALL_VERSIONS);
+    cells = Cell.createSingleCellArray(region.get(r, column, -1, 10000));
     assertTrue(cells.length == HColumnDescriptor.DEFAULT_VERSIONS);
   }
   
@@ -435,14 +436,12 @@
       scanner = region.getScanner(columns,
           arbitraryStartRow, HConstants.LATEST_TIMESTAMP,
           new WhileMatchRowFilter(new StopRowFilter(arbitraryStopRow)));
-      HStoreKey key = new HStoreKey();
-      TreeMap<byte [], Cell> value =
-        new TreeMap<byte [], Cell>(Bytes.BYTES_COMPARATOR);
-      while (scanner.next(key, value)) { 
+      List<KeyValue> value = new ArrayList<KeyValue>();
+      while (scanner.next(value)) { 
         if (actualStartRow == null) {
-          actualStartRow = key.getRow();
+          actualStartRow = value.get(0).getRow();
         } else {
-          actualStopRow = key.getRow();
+          actualStopRow = value.get(0).getRow();
         }
       }
       // Assert I got all out.

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java?rev=764289&r1=764288&r2=764289&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java Sun Apr 12 10:39:55 2009
@@ -20,17 +20,17 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
-import java.util.TreeMap;
+import java.util.ArrayList;
+import java.util.List;
 
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.Reader;
-
 import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
 
 /** JUnit test case for HLog */
 public class TestHLog extends HBaseTestCase implements HConstants {
@@ -73,10 +73,10 @@
       for (int ii = 0; ii < 3; ii++) {
         for (int i = 0; i < 3; i++) {
           for (int j = 0; j < 3; j++) {
-            TreeMap<HStoreKey, byte[]> edit = new TreeMap<HStoreKey, byte[]>();
-            byte [] column = Bytes.toBytes(Integer.toString(j));
-            edit.put(new HStoreKey(rowName, column, System.currentTimeMillis()),
-              column);
+            List<KeyValue> edit = new ArrayList<KeyValue>();
+            byte [] column = Bytes.toBytes("column:" + Integer.toString(j));
+            edit.add(new KeyValue(rowName, column, System.currentTimeMillis(),
+              column));
             log.append(Bytes.toBytes(Integer.toString(i)), tableName, edit, false);
           }
         }
@@ -105,10 +105,10 @@
       // Write columns named 1, 2, 3, etc. and then values of single byte
       // 1, 2, 3...
       long timestamp = System.currentTimeMillis();
-      TreeMap<HStoreKey, byte []> cols = new TreeMap<HStoreKey, byte []>();
+      List<KeyValue> cols = new ArrayList<KeyValue>();
       for (int i = 0; i < COL_COUNT; i++) {
-        cols.put(new HStoreKey(row, Bytes.toBytes(Integer.toString(i)), timestamp),
-            new byte[] { (byte)(i + '0') });
+        cols.add(new KeyValue(row, Bytes.toBytes("column:" + Integer.toString(i)),
+          timestamp, new byte[] { (byte)(i + '0') }));
       }
       log.append(regionName, tableName, cols, false);
       long logSeqId = log.startCacheFlush();
@@ -124,18 +124,18 @@
         reader.next(key, val);
         assertTrue(Bytes.equals(regionName, key.getRegionName()));
         assertTrue(Bytes.equals(tableName, key.getTablename()));
-        assertTrue(Bytes.equals(row, key.getRow()));
-        assertEquals((byte)(i + '0'), val.getVal()[0]);
+        assertTrue(Bytes.equals(row, val.getKeyValue().getRow()));
+        assertEquals((byte)(i + '0'), val.getKeyValue().getValue()[0]);
         System.out.println(key + " " + val);
       }
       while (reader.next(key, val)) {
         // Assert only one more row... the meta flushed row.
         assertTrue(Bytes.equals(regionName, key.getRegionName()));
         assertTrue(Bytes.equals(tableName, key.getTablename()));
-        assertTrue(Bytes.equals(HLog.METAROW, key.getRow()));
-        assertTrue(Bytes.equals(HLog.METACOLUMN, val.getColumn()));
+        assertTrue(Bytes.equals(HLog.METAROW, val.getKeyValue().getRow()));
+        assertTrue(Bytes.equals(HLog.METACOLUMN, val.getKeyValue().getColumn()));
         assertEquals(0, Bytes.compareTo(HLogEdit.COMPLETE_CACHE_FLUSH,
-          val.getVal()));
+          val.getKeyValue().getValue()));
         System.out.println(key + " " + val);
       }
     } finally {