HBASE-1320 hbase-1234 broke filter tests
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@769257 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
4a05b0693a
commit
40245ba36e
|
@ -89,6 +89,7 @@ Release 0.20.0 - Unreleased
|
|||
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
|
||||
|
|
|
@ -512,7 +512,7 @@ public class KeyValue {
|
|||
* @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);
|
||||
|
|
|
@ -37,11 +37,10 @@ import org.apache.hadoop.io.ObjectWritable;
|
|||
* 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 @@ public class ColumnValueFilter implements RowFilterInterface {
|
|||
return false;
|
||||
}
|
||||
|
||||
|
||||
public boolean filterColumn(final byte[] rowKey,
|
||||
final byte[] colKey, final byte[] data) {
|
||||
if (!filterIfColumnMissing) {
|
||||
|
@ -140,18 +140,25 @@ public class ColumnValueFilter implements RowFilterInterface {
|
|||
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 @@ public class ColumnValueFilter implements RowFilterInterface {
|
|||
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) {
|
||||
|
|
|
@ -44,16 +44,16 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
*
|
||||
* 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 class RegExpRowFilter implements RowFilterInterface {
|
|||
|
||||
|
||||
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 @@ public class RegExpRowFilter implements RowFilterInterface {
|
|||
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;
|
||||
}
|
||||
|
|
|
@ -243,8 +243,26 @@ public class RowFilterSet implements RowFilterInterface {
|
|||
}
|
||||
|
||||
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 {
|
||||
|
|
|
@ -97,7 +97,8 @@ public class StopRowFilter implements RowFilterInterface {
|
|||
}
|
||||
return false;
|
||||
}
|
||||
return Bytes.compareTo(stopRowKey, rowKey) <= 0;
|
||||
return Bytes.compareTo(stopRowKey, 0, stopRowKey.length, rowKey, offset,
|
||||
length) <= 0;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.hadoop.mapred.JobConfigurable;
|
|||
*/
|
||||
public class TableInputFormat extends TableInputFormatBase implements
|
||||
JobConfigurable {
|
||||
@SuppressWarnings("hiding")
|
||||
private final Log LOG = LogFactory.getLog(TableInputFormat.class);
|
||||
|
||||
/**
|
||||
|
|
|
@ -28,6 +28,7 @@ import java.util.regex.Pattern;
|
|||
* 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:";
|
||||
|
|
|
@ -101,4 +101,4 @@ public class TestGetRowVersions extends HBaseClusterTestCase {
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -33,7 +33,7 @@ import junit.framework.TestCase;
|
|||
/**
|
||||
* Tests the stop row filter
|
||||
*/
|
||||
public class DisabledTestColumnValueFilter extends TestCase {
|
||||
public class TestColumnValueFilter extends TestCase {
|
||||
|
||||
private static final byte[] ROW = Bytes.toBytes("test");
|
||||
private static final byte[] COLUMN = Bytes.toBytes("test:foo");
|
||||
|
@ -65,26 +65,34 @@ public class DisabledTestColumnValueFilter extends TestCase {
|
|||
|
||||
private void basicFilterTests(RowFilterInterface filter)
|
||||
throws Exception {
|
||||
assertTrue("basicFilter1", filter.filterColumn(ROW, COLUMN, VAL_1));
|
||||
assertFalse("basicFilter2", filter.filterColumn(ROW, COLUMN, VAL_2));
|
||||
assertFalse("basicFilter3", filter.filterColumn(ROW, COLUMN, VAL_3));
|
||||
assertFalse("basicFilter4", filter.filterColumn(ROW, COLUMN, VAL_4));
|
||||
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, COLUMN, FULLSTRING_1));
|
||||
assertFalse("substrFalse", filter.filterColumn(ROW, COLUMN, FULLSTRING_2));
|
||||
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, COLUMN, FULLSTRING_1));
|
||||
assertFalse("regexFalse", filter.filterColumn(ROW, COLUMN, FULLSTRING_2));
|
||||
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));
|
||||
}
|
|
@ -33,7 +33,7 @@ import junit.framework.TestCase;
|
|||
/**
|
||||
* Tests the stop row filter
|
||||
*/
|
||||
public class DisabledTestStopRowFilter extends TestCase {
|
||||
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");
|
||||
|
@ -74,21 +74,25 @@ public class DisabledTestStopRowFilter extends TestCase {
|
|||
// 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));
|
||||
|
||||
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));
|
||||
assertFalse("Filter a null", filter.filterRowKey(null, 0, 0));
|
||||
}
|
||||
}
|
|
@ -34,7 +34,7 @@ import junit.framework.TestCase;
|
|||
/**
|
||||
* Tests for the while-match filter
|
||||
*/
|
||||
public class DisabledTestWhileMatchRowFilter extends TestCase {
|
||||
public class TestWhileMatchRowFilter extends TestCase {
|
||||
|
||||
WhileMatchRowFilter wmStopRowFilter;
|
||||
WhileMatchRowFilter wmRegExpRowFilter;
|
||||
|
@ -93,14 +93,16 @@ public class DisabledTestWhileMatchRowFilter extends TestCase {
|
|||
// 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)));
|
||||
|
||||
byte [] toTestBytes = Bytes.toBytes(toTest);
|
||||
assertFalse("innerFilter: '" + toTest + "'",
|
||||
innerFilter.filterRowKey(toTestBytes, 0, toTestBytes.length));
|
||||
|
||||
// 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)));
|
||||
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());
|
||||
|
@ -123,15 +125,17 @@ public class DisabledTestWhileMatchRowFilter extends TestCase {
|
|||
|
||||
// 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(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(Bytes.toBytes(
|
||||
toTest)));
|
||||
assertTrue("innerFilter: '" + toTest + "'",
|
||||
innerFilter.filterRowKey(toTestBytes, 0, toTestBytes.length));
|
||||
|
||||
// The difference in switch
|
||||
assertTrue("filter: filterAllRemaining", filter.filterAllRemaining());
|
||||
|
@ -145,7 +149,9 @@ public class DisabledTestWhileMatchRowFilter extends TestCase {
|
|||
|
||||
// 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));
|
||||
toTestBytes = Bytes.toBytes(toTest);
|
||||
assertFalse("filter: '" + toTest + "'",
|
||||
filter.filterColumn(toTestBytes, 0, toTestBytes.length,
|
||||
null, 0, 0, null, 0, 0));
|
||||
}
|
||||
}
|
|
@ -90,6 +90,53 @@ public class TestScanner extends HBaseTestCase {
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 @@ public class TestScanner extends HBaseTestCase {
|
|||
throws IOException {
|
||||
InternalScanner scanner = null;
|
||||
List<KeyValue> results = new ArrayList<KeyValue>();
|
||||
|
||||
byte [][][] scanColumns = {
|
||||
COLS,
|
||||
EXPLICIT_COLS
|
||||
|
@ -238,27 +284,26 @@ public class TestScanner extends HBaseTestCase {
|
|||
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,47 +317,30 @@ public class TestScanner extends HBaseTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private boolean hasColumn(final List<KeyValue> kvs, final byte [] column) {
|
||||
for (KeyValue kv: kvs) {
|
||||
if (kv.matchingColumn(column)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private KeyValue getColumn(final List<KeyValue> kvs, final byte [] column) {
|
||||
for (KeyValue kv: kvs) {
|
||||
if (kv.matchingColumn(column)) {
|
||||
return kv;
|
||||
}
|
||||
}
|
||||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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++;
|
||||
}
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* HBase-910.
|
||||
* @throws Exception
|
||||
|
|
|
@ -54,7 +54,7 @@ public class TestBytes extends TestCase {
|
|||
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));
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue