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/28 08:05:55 UTC
svn commit: r769257 - in /hadoop/hbase/trunk: ./
src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/filter/
src/java/org/apache/hadoop/hbase/mapred/
src/java/org/apache/hadoop/hbase/util/
src/test/org/apache/hadoop/hbase/client/ src/tes...
Author: stack
Date: Tue Apr 28 06:05:53 2009
New Revision: 769257
URL: http://svn.apache.org/viewvc?rev=769257&view=rev
Log:
HBASE-1320 hbase-1234 broke filter tests
Added:
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java
Removed:
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestColumnValueFilter.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestStopRowFilter.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestWhileMatchRowFilter.java
Modified:
hadoop/hbase/trunk/CHANGES.txt
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/KeyValue.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Keying.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestBytes.java
Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Tue Apr 28 06:05:53 2009
@@ -89,6 +89,7 @@
HBASE-1340 Fix new javadoc warnings (Evgeny Ryabitskiy via Stack)
HBASE-1287 Partitioner class not used in TableMapReduceUtil.initTableReduceJob()
(Lars George and Billy Pearson via Stack)
+ HBASE-1320 hbase-1234 broke filter tests
IMPROVEMENTS
HBASE-1089 Add count of regions on filesystem to master UI; add percentage
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/KeyValue.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/KeyValue.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/KeyValue.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/KeyValue.java Tue Apr 28 06:05:53 2009
@@ -512,7 +512,7 @@
* @return Clone of this key with type set to <code>newtype</code>
*/
private KeyValue createKey(final Type newtype) {
- int keylength= getKeyLength();
+ int keylength = getKeyLength();
int l = keylength + ROW_OFFSET;
byte [] other = new byte[l];
System.arraycopy(getBuffer(), getOffset(), other, 0, l);
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=769257&r1=769256&r2=769257&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 Tue Apr 28 06:05:53 2009
@@ -37,11 +37,10 @@
* 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
+ * a long and then compare it to a fixed long value), then you can pass in your
* own comparator instead.
*/
public class ColumnValueFilter implements RowFilterInterface {
-
/** Comparison operators. */
public enum CompareOp {
/** less than */
@@ -132,6 +131,7 @@
return false;
}
+
public boolean filterColumn(final byte[] rowKey,
final byte[] colKey, final byte[] data) {
if (!filterIfColumnMissing) {
@@ -140,18 +140,25 @@
if (!Arrays.equals(colKey, columnName)) {
return false;
}
- return filterColumnValue(data);
+ return filterColumnValue(data, 0, data.length);
}
public boolean filterColumn(byte[] rowKey, int roffset, int rlength,
- byte[] colunmName, int coffset, int clength, byte[] columnValue,
+ byte[] cn, int coffset, int clength, byte[] columnValue,
int voffset, int vlength) {
- if (true) throw new RuntimeException("Not yet implemented");
- return false;
+ if (!filterIfColumnMissing) {
+ return false; // Must filter on the whole row
+ }
+ if (Bytes.compareTo(cn, coffset, clength,
+ this.columnName, 0, this.columnName.length) != 0) {
+ return false;
+ }
+ return filterColumnValue(columnValue, voffset, vlength);
}
- private boolean filterColumnValue(final byte [] data) {
+ private boolean filterColumnValue(final byte [] data, final int offset,
+ final int length) {
int compareResult;
if (comparator != null) {
compareResult = comparator.compareTo(data);
@@ -192,13 +199,37 @@
if (colCell == null) {
return false;
}
- return this.filterColumnValue(colCell.getValue());
+ byte [] v = colCell.getValue();
+ return this.filterColumnValue(v, 0, v.length);
}
-
public boolean filterRow(List<KeyValue> results) {
- if (true) throw new RuntimeException("Not yet implemented");
- return false;
+ if (results == null) return false;
+ KeyValue found = null;
+ if (filterIfColumnMissing) {
+ boolean doesntHaveIt = true;
+ for (KeyValue kv: results) {
+ if (kv.matchingColumn(columnName)) {
+ doesntHaveIt = false;
+ found = kv;
+ break;
+ }
+ }
+ if (doesntHaveIt) return doesntHaveIt;
+ }
+ if (found == null) {
+ for (KeyValue kv: results) {
+ if (kv.matchingColumn(columnName)) {
+ found = kv;
+ break;
+ }
+ }
+ }
+ if (found == null) {
+ return false;
+ }
+ return this.filterColumnValue(found.getValue(), found.getValueOffset(),
+ found.getValueLength());
}
private int compare(final byte[] b1, final byte[] b2) {
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java Tue Apr 28 06:05:53 2009
@@ -44,16 +44,16 @@
*
* Note that column value filtering in this interface has been replaced by
* {@link ColumnValueFilter}.
+ * @deprecated This interface doesn't really work well in new KeyValue world.
+ * Needs to be refactored/removed. Marking it as deprecated till it gets
+ * cleaned up. Its also inefficient as written.
*/
public class RegExpRowFilter implements RowFilterInterface {
private Pattern rowKeyPattern = null;
private String rowKeyRegExp = null;
-
- @Deprecated
private Map<byte [], byte[]> equalsMap =
new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
- @Deprecated
private Set<byte []> nullColumns =
new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
@@ -177,12 +177,42 @@
public boolean filterColumn(byte[] rowKey, int roffset, int rlength,
- byte[] colunmName, int coffset, int clength, byte[] columnValue,
+ byte [] colunmName, int coffset, int clength, byte[] columnValue,
int voffset, int vlength) {
- if (true) throw new RuntimeException("Not implemented yet");
+ if (filterRowKey(rowKey, roffset, rlength)) {
+ return true;
+ }
+ byte [] colkey = null;
+ if (filtersByColumnValue()) {
+ colkey = getColKey(colunmName, coffset, clength);
+ byte [] filterValue = equalsMap.get(colkey);
+ if (null != filterValue) {
+ return Bytes.compareTo(filterValue, 0, filterValue.length, columnValue,
+ voffset, vlength) != 0;
+ }
+ }
+ if (colkey == null) {
+ colkey = getColKey(colunmName, coffset, clength);
+ }
+ if (nullColumns.contains(colkey)) {
+ if (columnValue != null /* TODO: FIX!!! && !HLogEdit.isDeleted(data)*/) {
+ return true;
+ }
+ }
return false;
}
+ private byte [] getColKey(final byte [] c, final int offset, final int length) {
+ byte [] colkey = null;
+ if (offset == 0) {
+ colkey = c;
+ } else {
+ colkey = new byte [length];
+ System.arraycopy(c, offset, colkey, 0, length);
+ }
+ return colkey;
+ }
+
public boolean filterRow(final SortedMap<byte [], Cell> columns) {
for (Entry<byte [], Cell> col : columns.entrySet()) {
if (nullColumns.contains(col.getKey())
@@ -198,12 +228,20 @@
return false;
}
- public boolean filterRow(List<KeyValue> results) {
- if (true) throw new RuntimeException("NOT YET IMPLEMENTED");
+ // THIS METHOD IS HORRIDLY EXPENSIVE TO RUN. NEEDS FIXUP.
+ public boolean filterRow(List<KeyValue> kvs) {
+ for (KeyValue kv: kvs) {
+ byte [] column = kv.getColumn();
+ if (nullColumns.contains(column) && !kv.isDeleteType()) {
+ return true;
+ }
+ if (!equalsMap.containsKey(column)) {
+ return true;
+ }
+ }
return false;
}
- @Deprecated
private boolean filtersByColumnValue() {
return equalsMap != null && equalsMap.size() > 0;
}
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=769257&r1=769256&r2=769257&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 Tue Apr 28 06:05:53 2009
@@ -243,8 +243,26 @@
}
public boolean filterRow(List<KeyValue> results) {
- if (true) throw new RuntimeException("Not Yet Implemented");
- return false;
+ boolean resultFound = false;
+ boolean result = operator == Operator.MUST_PASS_ONE;
+ for (RowFilterInterface filter : filters) {
+ if (!resultFound) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterAllRemaining() || filter.filterRow(results)) {
+ result = true;
+ resultFound = true;
+ }
+ } else if (operator == Operator.MUST_PASS_ONE) {
+ if (!filter.filterAllRemaining() && !filter.filterRow(results)) {
+ result = false;
+ resultFound = true;
+ }
+ }
+ } else if (filter.processAlways()) {
+ filter.filterRow(results);
+ }
+ }
+ return result;
}
public void readFields(final DataInput in) throws IOException {
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=769257&r1=769256&r2=769257&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 Tue Apr 28 06:05:53 2009
@@ -97,7 +97,8 @@
}
return false;
}
- return Bytes.compareTo(stopRowKey, rowKey) <= 0;
+ return Bytes.compareTo(stopRowKey, 0, stopRowKey.length, rowKey, offset,
+ length) <= 0;
}
/**
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java Tue Apr 28 06:05:53 2009
@@ -36,7 +36,6 @@
*/
public class TableInputFormat extends TableInputFormatBase implements
JobConfigurable {
- @SuppressWarnings("hiding")
private final Log LOG = LogFactory.getLog(TableInputFormat.class);
/**
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Keying.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Keying.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Keying.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Keying.java Tue Apr 28 06:05:53 2009
@@ -28,6 +28,7 @@
* Use fabricating row names or column qualifiers.
* <p>TODO: Add createSchemeless key, a key that doesn't care if scheme is
* http or https.
+ * @see Bytes#split(byte[], byte[], int)
*/
public class Keying {
private static final String SCHEME = "r:";
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java Tue Apr 28 06:05:53 2009
@@ -101,4 +101,4 @@
}
}
}
-}
+}
\ No newline at end of file
Modified: 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=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java Tue Apr 28 06:05:53 2009
@@ -1,201 +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.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/TestColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java?rev=769257&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java Tue Apr 28 06:05:53 2009
@@ -0,0 +1,153 @@
+/**
+ * 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 TestColumnValueFilter extends TestCase {
+
+ private static final byte[] ROW = Bytes.toBytes("test");
+ private static final byte[] COLUMN = Bytes.toBytes("test:foo");
+ private static final byte[] VAL_1 = Bytes.toBytes("a");
+ private static final byte[] VAL_2 = Bytes.toBytes("ab");
+ private static final byte[] VAL_3 = Bytes.toBytes("abc");
+ private static final byte[] VAL_4 = Bytes.toBytes("abcd");
+ private static final byte[] FULLSTRING_1 =
+ Bytes.toBytes("The quick brown fox jumps over the lazy dog.");
+ private static final byte[] FULLSTRING_2 =
+ Bytes.toBytes("The slow grey fox trips over the lazy dog.");
+ private static final String QUICK_SUBSTR = "quick";
+ private static final String QUICK_REGEX = "[q][u][i][c][k]";
+
+ private RowFilterInterface basicFilterNew() {
+ return new ColumnValueFilter(COLUMN,
+ ColumnValueFilter.CompareOp.GREATER_OR_EQUAL, VAL_2);
+ }
+
+ private RowFilterInterface substrFilterNew() {
+ return new ColumnValueFilter(COLUMN, ColumnValueFilter.CompareOp.EQUAL,
+ new SubstringComparator(QUICK_SUBSTR));
+ }
+
+ private RowFilterInterface regexFilterNew() {
+ return new ColumnValueFilter(COLUMN, ColumnValueFilter.CompareOp.EQUAL,
+ new RegexStringComparator(QUICK_REGEX));
+ }
+
+ private void basicFilterTests(RowFilterInterface filter)
+ throws Exception {
+ assertTrue("basicFilter1", filter.filterColumn(ROW, 0, ROW.length,
+ COLUMN, 0, COLUMN.length, VAL_1, 0, VAL_1.length));
+ assertFalse("basicFilter2", filter.filterColumn(ROW, 0, ROW.length,
+ COLUMN, 0, COLUMN.length, VAL_2, 0, VAL_2.length));
+ assertFalse("basicFilter3", filter.filterColumn(ROW, 0, ROW.length,
+ COLUMN, 0, COLUMN.length, VAL_3, 0, VAL_3.length));
+ assertFalse("basicFilter4", filter.filterColumn(ROW, 0, ROW.length,
+ COLUMN, 0, COLUMN.length, VAL_4, 0, VAL_4.length));
+ assertFalse("basicFilterAllRemaining", filter.filterAllRemaining());
+ assertFalse("basicFilterNotNull", filter.filterRow((List<KeyValue>)null));
+ }
+
+ private void substrFilterTests(RowFilterInterface filter)
+ throws Exception {
+ assertTrue("substrTrue", filter.filterColumn(ROW, 0, ROW.length,
+ COLUMN, 0, COLUMN.length, FULLSTRING_1, 0, FULLSTRING_1.length));
+ assertFalse("substrFalse", filter.filterColumn(ROW, 0, ROW.length,
+ COLUMN, 0, COLUMN.length, FULLSTRING_2, 0, FULLSTRING_2.length));
+ assertFalse("substrFilterAllRemaining", filter.filterAllRemaining());
+ assertFalse("substrFilterNotNull", filter.filterRow((List<KeyValue>)null));
+ }
+
+ private void regexFilterTests(RowFilterInterface filter)
+ throws Exception {
+ assertTrue("regexTrue", filter.filterColumn(ROW, 0, ROW.length,
+ COLUMN, 0, COLUMN.length, FULLSTRING_1, 0, FULLSTRING_1.length));
+ assertFalse("regexFalse", filter.filterColumn(ROW, 0, ROW.length,
+ COLUMN, 0, COLUMN.length, FULLSTRING_2, 0, FULLSTRING_2.length));
+ assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
+ assertFalse("regexFilterNotNull", filter.filterRow((List<KeyValue>)null));
+ }
+
+ private RowFilterInterface serializationTest(RowFilterInterface filter)
+ throws Exception {
+ // Decompose filter to bytes.
+ ByteArrayOutputStream stream = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputStream(stream);
+ filter.write(out);
+ out.close();
+ byte[] buffer = stream.toByteArray();
+
+ // Recompose filter.
+ DataInputStream in =
+ new DataInputStream(new ByteArrayInputStream(buffer));
+ RowFilterInterface newFilter = new ColumnValueFilter();
+ newFilter.readFields(in);
+
+ return newFilter;
+ }
+
+ RowFilterInterface basicFilter;
+ RowFilterInterface substrFilter;
+ RowFilterInterface regexFilter;
+
+ @Override
+ protected void setUp() throws Exception {
+ super.setUp();
+ basicFilter = basicFilterNew();
+ substrFilter = substrFilterNew();
+ regexFilter = regexFilterNew();
+ }
+
+ /**
+ * Tests identification of the stop row
+ * @throws Exception
+ */
+ public void testStop() throws Exception {
+ basicFilterTests(basicFilter);
+ substrFilterTests(substrFilter);
+ regexFilterTests(regexFilter);
+ }
+
+ /**
+ * Tests serialization
+ * @throws Exception
+ */
+ public void testSerialization() throws Exception {
+ RowFilterInterface newFilter = serializationTest(basicFilter);
+ basicFilterTests(newFilter);
+ newFilter = serializationTest(substrFilter);
+ substrFilterTests(newFilter);
+ newFilter = serializationTest(regexFilter);
+ regexFilterTests(newFilter);
+ }
+
+}
Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java?rev=769257&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java Tue Apr 28 06:05:53 2009
@@ -0,0 +1,98 @@
+/**
+ * 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 TestStopRowFilter 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, 0, GOOD_ROW.length));
+ assertTrue("Filtering on " + Bytes.toString(STOP_ROW),
+ filter.filterRowKey(STOP_ROW, 0, STOP_ROW.length));
+ assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW),
+ filter.filterRowKey(PAST_STOP_ROW, 0, PAST_STOP_ROW.length));
+ assertFalse("Filtering on " + Bytes.toString(GOOD_ROW),
+ filter.filterColumn(GOOD_ROW, 0, GOOD_ROW.length, null, 0, 0,
+ null, 0, 0));
+ assertTrue("Filtering on " + Bytes.toString(STOP_ROW),
+ filter.filterColumn(STOP_ROW, 0, STOP_ROW.length, null, 0, 0, null, 0, 0));
+ assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW),
+ filter.filterColumn(PAST_STOP_ROW, 0, PAST_STOP_ROW.length, null, 0, 0,
+ null, 0, 0));
+ assertFalse("FilterAllRemaining", filter.filterAllRemaining());
+ assertFalse("FilterNotNull", filter.filterRow((List<KeyValue>)null));
+
+ assertFalse("Filter a null", filter.filterRowKey(null, 0, 0));
+ }
+}
\ No newline at end of file
Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java?rev=769257&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java Tue Apr 28 06:05:53 2009
@@ -0,0 +1,157 @@
+/**
+ * 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 TestWhileMatchRowFilter 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)));
+ byte [] toTestBytes = Bytes.toBytes(toTest);
+ assertFalse("innerFilter: '" + toTest + "'",
+ innerFilter.filterRowKey(toTestBytes, 0, toTestBytes.length));
+
+ // Test cases that should fail the row
+ toTest = "tuna";
+ toTestBytes = Bytes.toBytes(toTest);
+ assertTrue("filter: '" + toTest + "'", filter.filterRowKey(toTestBytes));
+ assertTrue("innerFilter: '" + toTest + "'",
+ innerFilter.filterRowKey(toTestBytes, 0, toTestBytes.length));
+
+ // 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";
+ byte [] toTestBytes = Bytes.toBytes(toTest);
+ assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+ assertFalse("innerFilter: '" + toTest + "'",
+ innerFilter.filterRowKey(toTestBytes, 0, toTestBytes.length));
+
+ // Test cases that should fail the row
+ toTest = "not_a_match";
+ toTestBytes = Bytes.toBytes(toTest);
+ assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+ assertTrue("innerFilter: '" + toTest + "'",
+ innerFilter.filterRowKey(toTestBytes, 0, toTestBytes.length));
+
+ // 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";
+ toTestBytes = Bytes.toBytes(toTest);
+ assertFalse("filter: '" + toTest + "'",
+ filter.filterColumn(toTestBytes, 0, toTestBytes.length,
+ null, 0, 0, null, 0, 0));
+ }
+}
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java Tue Apr 28 06:05:53 2009
@@ -90,6 +90,53 @@
}
+ /**
+ * Test basic stop row filter works.
+ * @throws Exception
+ */
+ public void testStopRow() throws Exception {
+ byte [] startrow = Bytes.toBytes("bbb");
+ byte [] stoprow = Bytes.toBytes("ccc");
+ try {
+ this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
+ addContent(this.r, HConstants.COLUMN_FAMILY);
+ List<KeyValue> results = new ArrayList<KeyValue>();
+ // Do simple test of getting one row only first.
+ InternalScanner s = r.getScanner(HConstants.COLUMN_FAMILY_ARRAY,
+ Bytes.toBytes("abc"), HConstants.LATEST_TIMESTAMP,
+ new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("abd"))));
+ int count = 0;
+ while (s.next(results)) {
+ count++;
+ }
+ s.close();
+ assertEquals(1, count);
+ // Now do something a bit more imvolved.
+ s = r.getScanner(HConstants.COLUMN_FAMILY_ARRAY,
+ startrow, HConstants.LATEST_TIMESTAMP,
+ new WhileMatchRowFilter(new StopRowFilter(stoprow)));
+ count = 0;
+ KeyValue kv = null;
+ results = new ArrayList<KeyValue>();
+ for (boolean first = true; s.next(results);) {
+ kv = results.get(0);
+ if (first) {
+ assertTrue(Bytes.BYTES_COMPARATOR.compare(startrow, kv.getRow()) == 0);
+ first = false;
+ }
+ count++;
+ }
+ assertTrue(Bytes.BYTES_COMPARATOR.compare(stoprow, kv.getRow()) > 0);
+ // We got something back.
+ assertTrue(count > 10);
+ s.close();
+ } finally {
+ this.r.close();
+ this.r.getLog().closeAndDelete();
+ shutdownDfs(this.cluster);
+ }
+ }
+
/** The test!
* @throws IOException
*/
@@ -227,7 +274,6 @@
throws IOException {
InternalScanner scanner = null;
List<KeyValue> results = new ArrayList<KeyValue>();
-
byte [][][] scanColumns = {
COLS,
EXPLICIT_COLS
@@ -238,27 +284,26 @@
scanner = r.getScanner(scanColumns[i], FIRST_ROW,
System.currentTimeMillis(), null);
while (scanner.next(results)) {
- // FIX!!!
-// assertTrue(results.containsKey(HConstants.COL_REGIONINFO));
-// byte [] val = results.get(HConstants.COL_REGIONINFO).getValue();
-// validateRegionInfo(val);
-// if(validateStartcode) {
-// assertTrue(results.containsKey(HConstants.COL_STARTCODE));
-// val = results.get(HConstants.COL_STARTCODE).getValue();
-// assertNotNull(val);
-// assertFalse(val.length == 0);
-// long startCode = Bytes.toLong(val);
-// assertEquals(START_CODE, startCode);
-// }
-//
-// if(serverName != null) {
-// assertTrue(results.containsKey(HConstants.COL_SERVER));
-// val = results.get(HConstants.COL_SERVER).getValue();
-// assertNotNull(val);
-// assertFalse(val.length == 0);
-// String server = Bytes.toString(val);
-// assertEquals(0, server.compareTo(serverName));
-// }
+ assertTrue(hasColumn(results, HConstants.COL_REGIONINFO));
+ byte [] val = getColumn(results, HConstants.COL_REGIONINFO).getValue();
+ validateRegionInfo(val);
+ if(validateStartcode) {
+ assertTrue(hasColumn(results, HConstants.COL_STARTCODE));
+ val = getColumn(results, HConstants.COL_STARTCODE).getValue();
+ assertNotNull(val);
+ assertFalse(val.length == 0);
+ long startCode = Bytes.toLong(val);
+ assertEquals(START_CODE, startCode);
+ }
+
+ if(serverName != null) {
+ assertTrue(hasColumn(results, HConstants.COL_SERVER));
+ val = getColumn(results, HConstants.COL_SERVER).getValue();
+ assertNotNull(val);
+ assertFalse(val.length == 0);
+ String server = Bytes.toString(val);
+ assertEquals(0, server.compareTo(serverName));
+ }
results.clear();
}
@@ -272,45 +317,28 @@
}
}
- /** Use get to retrieve the HRegionInfo and validate it */
- private void getRegionInfo() throws IOException {
- byte [] bytes = region.get(ROW_KEY, HConstants.COL_REGIONINFO).getValue();
- validateRegionInfo(bytes);
+ private boolean hasColumn(final List<KeyValue> kvs, final byte [] column) {
+ for (KeyValue kv: kvs) {
+ if (kv.matchingColumn(column)) {
+ return true;
+ }
+ }
+ return false;
}
- /**
- * Test basic stop row filter works.
- * @throws Exception
- */
- public void testStopRow() throws Exception {
- byte [] startrow = Bytes.toBytes("bbb");
- byte [] stoprow = Bytes.toBytes("ccc");
- try {
- this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
- addContent(this.r, HConstants.COLUMN_FAMILY);
- InternalScanner s = r.getScanner(HConstants.COLUMN_FAMILY_ARRAY,
- startrow, HConstants.LATEST_TIMESTAMP,
- new WhileMatchRowFilter(new StopRowFilter(stoprow)));
- List<KeyValue> results = new ArrayList<KeyValue>();
- int count = 0;
- KeyValue kv = null;
- for (boolean first = true; s.next(results);) {
- kv = results.get(0);
- if (first) {
- assertTrue(Bytes.BYTES_COMPARATOR.compare(startrow, kv.getRow()) == 0);
- first = false;
- }
- count++;
+ private KeyValue getColumn(final List<KeyValue> kvs, final byte [] column) {
+ for (KeyValue kv: kvs) {
+ if (kv.matchingColumn(column)) {
+ return kv;
}
- assertTrue(Bytes.BYTES_COMPARATOR.compare(stoprow, kv.getRow()) > 0);
- // We got something back.
- assertTrue(count > 10);
- s.close();
- } finally {
- this.r.close();
- this.r.getLog().closeAndDelete();
- shutdownDfs(this.cluster);
}
+ return null;
+ }
+
+ /** Use get to retrieve the HRegionInfo and validate it */
+ private void getRegionInfo() throws IOException {
+ byte [] bytes = region.get(ROW_KEY, HConstants.COL_REGIONINFO).getValue();
+ validateRegionInfo(bytes);
}
/**
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestBytes.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestBytes.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestBytes.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestBytes.java Tue Apr 28 06:05:53 2009
@@ -54,7 +54,7 @@
for (int i = 0; i < parts.length; i++) {
System.out.println(Bytes.toString(parts[i]));
}
- assertEquals(2, parts.length);
+ assertEquals(3, parts.length);
assertTrue(Bytes.equals(parts[1], middle));
}