HBASE-2840 Remove the final remnants of the old Get code - the query matchers and other helper classes

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@964981 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Ryan Rawson 2010-07-16 23:29:19 +00:00
parent 8164552e35
commit 7557b73e90
21 changed files with 226 additions and 2417 deletions

View File

@ -774,6 +774,8 @@ Release 0.21.0 - Unreleased
next column (Pranav via jgray)
HBASE-2835 Update hadoop jar to head of branch-0.20-append to catch three
added patches
HBASE-2840 Remove the final remnants of the old Get code - the query matchers
and other helper classes
NEW FEATURES
HBASE-1961 HBase EC2 scripts

View File

@ -19,8 +19,6 @@
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
/**
* Implementing classes of this interface will be used for the tracking
* and enforcement of columns and numbers of versions during the course of a
@ -29,12 +27,10 @@ import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
* Currently there are two different types of Store/Family-level queries.
* <ul><li>{@link ExplicitColumnTracker} is used when the query specifies
* one or more column qualifiers to return in the family.
* <li>{@link WildcardColumnTracker} is used when the query asks for all
* qualifiers within the family.
* <p>
* This class is utilized by {@link QueryMatcher} through two methods:
* This class is utilized by {@link ScanQueryMatcher} through two methods:
* <ul><li>{@link #checkColumn} is called when a Put satisfies all other
* conditions of the query. This method returns a {@link MatchCode} to define
* conditions of the query. This method returns a {@link org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode} to define
* what action should be taken.
* <li>{@link #update} is called at the end of every StoreFile or memstore.
* <p>
@ -48,7 +44,7 @@ public interface ColumnTracker {
* @param length
* @return The match code instance.
*/
public MatchCode checkColumn(byte [] bytes, int offset, int length);
public ScanQueryMatcher.MatchCode checkColumn(byte [] bytes, int offset, int length);
/**
* Updates internal variables in between files

View File

@ -1,138 +0,0 @@
/**
* Copyright 2010 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Class that provides static method needed when putting deletes into memstore
*/
public class DeleteCompare {
/**
* Return codes from deleteCompare.
*/
enum DeleteCode {
/**
* Do nothing. Move to next KV in memstore
*/
SKIP,
/**
* Add to the list of deletes.
*/
DELETE,
/**
* Stop looking at KVs in memstore. Finalize.
*/
DONE
}
/**
* Method used when putting deletes into memstore to remove all the previous
* entries that are affected by this Delete
* @param mem
* @param deleteBuffer
* @param deleteRowOffset
* @param deleteRowLength
* @param deleteQualifierOffset
* @param deleteQualifierLength
* @param deleteTimeOffset
* @param deleteType
* @param comparator
* @return SKIP if current KeyValue should not be deleted, DELETE if
* current KeyValue should be deleted and DONE when the current KeyValue is
* out of the Deletes range
*/
public static DeleteCode deleteCompare(KeyValue mem, byte [] deleteBuffer,
int deleteRowOffset, short deleteRowLength, int deleteQualifierOffset,
int deleteQualifierLength, int deleteTimeOffset, byte deleteType,
KeyValue.KeyComparator comparator) {
//Parsing new KeyValue
byte [] memBuffer = mem.getBuffer();
int memOffset = mem.getOffset();
//Getting key lengths
int memKeyLen = Bytes.toInt(memBuffer, memOffset);
memOffset += Bytes.SIZEOF_INT;
//Skipping value lengths
memOffset += Bytes.SIZEOF_INT;
//Getting row lengths
short memRowLen = Bytes.toShort(memBuffer, memOffset);
memOffset += Bytes.SIZEOF_SHORT;
int res = comparator.compareRows(memBuffer, memOffset, memRowLen,
deleteBuffer, deleteRowOffset, deleteRowLength);
if(res > 0) {
return DeleteCode.DONE;
} else if(res < 0){
return DeleteCode.SKIP;
}
memOffset += memRowLen;
//Getting family lengths
byte memFamLen = memBuffer[memOffset];
memOffset += Bytes.SIZEOF_BYTE + memFamLen;
//Get column lengths
int memQualifierLen = memKeyLen - memRowLen - memFamLen -
Bytes.SIZEOF_SHORT - Bytes.SIZEOF_BYTE - Bytes.SIZEOF_LONG -
Bytes.SIZEOF_BYTE;
//Compare timestamp
int tsOffset = memOffset + memQualifierLen;
int timeRes = Bytes.compareTo(memBuffer, tsOffset, Bytes.SIZEOF_LONG,
deleteBuffer, deleteTimeOffset, Bytes.SIZEOF_LONG);
if (deleteType == KeyValue.Type.DeleteFamily.getCode()) {
if (timeRes <= 0) {
return DeleteCode.DELETE;
}
return DeleteCode.SKIP;
}
//Compare columns
res = Bytes.compareTo(memBuffer, memOffset, memQualifierLen,
deleteBuffer, deleteQualifierOffset, deleteQualifierLength);
if (res < 0) {
return DeleteCode.SKIP;
} else if(res > 0) {
return DeleteCode.DONE;
}
// same column, compare the time.
if (timeRes == 0) {
return DeleteCode.DELETE;
} else if (timeRes < 0) {
if (deleteType == KeyValue.Type.DeleteColumn.getCode()) {
return DeleteCode.DELETE;
}
return DeleteCode.DONE;
} else {
return DeleteCode.SKIP;
}
}
}

View File

@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.regionserver;
import java.util.ArrayList;
import java.util.List;
import java.util.NavigableSet;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.util.Bytes;
* between rows.
*
* <p>
* This class is utilized by {@link QueryMatcher} through two methods:
* This class is utilized by {@link ScanQueryMatcher} through two methods:
* <ul><li>{@link #checkColumn} is called when a Put satisfies all other
* conditions of the query. This method returns a {@link MatchCode} to define
* conditions of the query. This method returns a {@link org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode} to define
* what action should be taken.
* <li>{@link #update} is called at the end of every StoreFile or memstore.
* <p>
@ -84,18 +84,18 @@ public class ExplicitColumnTracker implements ColumnTracker {
* @param bytes KeyValue buffer
* @param offset offset to the start of the qualifier
* @param length length of the qualifier
* @return MatchCode telling QueryMatcher what action to take
* @return MatchCode telling ScanQueryMatcher what action to take
*/
public MatchCode checkColumn(byte [] bytes, int offset, int length) {
public ScanQueryMatcher.MatchCode checkColumn(byte [] bytes, int offset, int length) {
do {
// No more columns left, we are done with this query
if(this.columns.size() == 0) {
return MatchCode.DONE; // done_row
return ScanQueryMatcher.MatchCode.DONE; // done_row
}
// No more columns to match against, done with storefile
if(this.column == null) {
return MatchCode.NEXT; // done_row
return ScanQueryMatcher.MatchCode.NEXT; // done_row
}
// Compare specific column to current column
@ -114,13 +114,13 @@ public class ExplicitColumnTracker implements ColumnTracker {
this.column = this.columns.get(this.index);
}
}
return MatchCode.INCLUDE;
return ScanQueryMatcher.MatchCode.INCLUDE;
}
if (ret > 0) {
// Specified column is smaller than the current, skip to next column.
return MatchCode.SKIP;
return ScanQueryMatcher.MatchCode.SKIP;
}
// Specified column is bigger than current column
@ -128,7 +128,7 @@ public class ExplicitColumnTracker implements ColumnTracker {
if(ret <= -1) {
if(++this.index == this.columns.size()) {
// No more to match, do not include, done with storefile
return MatchCode.NEXT; // done_row
return ScanQueryMatcher.MatchCode.NEXT; // done_row
}
// This is the recursive case.
this.column = this.columns.get(this.index);

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* State and utility processing {@link HRegion#getClosestRowBefore(byte[], byte[])}.
* Like {@link GetDeleteTracker} and {@link ScanDeleteTracker} but does not
* Like {@link ScanDeleteTracker} and {@link ScanDeleteTracker} but does not
* implement the {@link DeleteTracker} interface since state spans rows (There
* is no update nor reset method).
*/

View File

@ -1,404 +0,0 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
/**
* This class is responsible for the tracking and enforcement of Deletes
* during the course of a Get operation.
* <p>
* This class is utilized through three methods:
* <ul><li>{@link #add} when encountering a Delete
* <li>{@link #isDeleted} when checking if a Put KeyValue has been deleted
* <li>{@link #update} when reaching the end of a StoreFile
* <p>
* This class is NOT thread-safe as queries are never multi-threaded
*/
public class GetDeleteTracker implements DeleteTracker {
private static long UNSET = -1L;
private long familyStamp = UNSET;
protected List<Delete> deletes = null;
private List<Delete> newDeletes = new ArrayList<Delete>();
private Iterator<Delete> iterator;
private Delete delete = null;
/**
* Constructor
*/
public GetDeleteTracker() {}
/**
* Add the specified KeyValue to the list of deletes to check against for
* this row operation.
* <p>
* This is called when a Delete is encountered in a StoreFile.
* @param buffer
* @param qualifierOffset
* @param qualifierLength
* @param timestamp
* @param type
*/
@Override
public void add(byte [] buffer, int qualifierOffset, int qualifierLength,
long timestamp, byte type) {
if (type == KeyValue.Type.DeleteFamily.getCode()) {
if(timestamp > familyStamp) {
familyStamp = timestamp;
}
return;
}
if(timestamp > familyStamp) {
this.newDeletes.add(new Delete(buffer, qualifierOffset, qualifierLength,
type, timestamp));
}
}
/**
* Check if the specified KeyValue buffer has been deleted by a previously
* seen delete.
* @param buffer KeyValue buffer
* @param qualifierOffset column qualifier offset
* @param qualifierLength column qualifier length
* @param timestamp timestamp
* @return true is the specified KeyValue is deleted, false if not
*/
@Override
public boolean isDeleted(byte [] buffer, int qualifierOffset,
int qualifierLength, long timestamp) {
// Check against DeleteFamily
if (timestamp <= familyStamp) {
return true;
}
// Check if there are other deletes
if (this.delete == null) {
return false;
}
// Check column
int ret = Bytes.compareTo(buffer, qualifierOffset, qualifierLength,
this.delete.buffer, this.delete.qualifierOffset,
this.delete.qualifierLength);
while (ret != 0) {
if (ret <= -1) {
// Have not reached the next delete yet
return false;
} else if (ret >= 1) {
// Deletes an earlier column, need to move down deletes
if (this.iterator.hasNext()) {
this.delete = this.iterator.next();
} else {
this.delete = null;
return false;
}
ret = Bytes.compareTo(buffer, qualifierOffset, qualifierLength,
this.delete.buffer, this.delete.qualifierOffset,
this.delete.qualifierLength);
}
}
// Check Timestamp
if(timestamp > this.delete.timestamp) {
return false;
}
// Check Type
switch(KeyValue.Type.codeToType(this.delete.type)) {
case Delete:
boolean equal = timestamp == this.delete.timestamp;
if(this.iterator.hasNext()) {
this.delete = this.iterator.next();
} else {
this.delete = null;
}
if(equal){
return true;
}
// timestamp < this.delete.timestamp
// Delete of an explicit column newer than current
return isDeleted(buffer, qualifierOffset, qualifierLength, timestamp);
case DeleteColumn:
return true;
}
// should never reach this
return false;
}
@Override
public boolean isEmpty() {
return this.familyStamp == UNSET && this.delete == null &&
this.newDeletes.isEmpty();
}
@Override
public void reset() {
this.deletes = null;
this.delete = null;
this.newDeletes = new ArrayList<Delete>();
this.familyStamp = UNSET;
this.iterator = null;
}
/**
* Called at the end of every StoreFile.
* <p>
* Many optimized implementations of Trackers will require an update at
* when the end of each StoreFile is reached.
*/
@Override
public void update() {
// If no previous deletes, use new deletes and return
if (this.deletes == null || this.deletes.size() == 0) {
finalize(this.newDeletes);
return;
}
// If no new delete, retain previous deletes and return
if(this.newDeletes.size() == 0) {
return;
}
// Merge previous deletes with new deletes
List<Delete> mergeDeletes =
new ArrayList<Delete>(this.newDeletes.size());
int oldIndex = 0;
int newIndex = 0;
Delete newDelete = newDeletes.get(oldIndex);
Delete oldDelete = deletes.get(oldIndex);
while(true) {
switch(compareDeletes(oldDelete,newDelete)) {
case NEXT_NEW: {
if(++newIndex == newDeletes.size()) {
// Done with new, add the rest of old to merged and return
mergeDown(mergeDeletes, deletes, oldIndex);
finalize(mergeDeletes);
return;
}
newDelete = this.newDeletes.get(newIndex);
break;
}
case INCLUDE_NEW_NEXT_NEW: {
mergeDeletes.add(newDelete);
if(++newIndex == newDeletes.size()) {
// Done with new, add the rest of old to merged and return
mergeDown(mergeDeletes, deletes, oldIndex);
finalize(mergeDeletes);
return;
}
newDelete = this.newDeletes.get(newIndex);
break;
}
case INCLUDE_NEW_NEXT_BOTH: {
mergeDeletes.add(newDelete);
++oldIndex;
++newIndex;
if(oldIndex == deletes.size()) {
if(newIndex == newDeletes.size()) {
finalize(mergeDeletes);
return;
}
mergeDown(mergeDeletes, newDeletes, newIndex);
finalize(mergeDeletes);
return;
} else if(newIndex == newDeletes.size()) {
mergeDown(mergeDeletes, deletes, oldIndex);
finalize(mergeDeletes);
return;
}
oldDelete = this.deletes.get(oldIndex);
newDelete = this.newDeletes.get(newIndex);
break;
}
case INCLUDE_OLD_NEXT_BOTH: {
mergeDeletes.add(oldDelete);
++oldIndex;
++newIndex;
if(oldIndex == deletes.size()) {
if(newIndex == newDeletes.size()) {
finalize(mergeDeletes);
return;
}
mergeDown(mergeDeletes, newDeletes, newIndex);
finalize(mergeDeletes);
return;
} else if(newIndex == newDeletes.size()) {
mergeDown(mergeDeletes, deletes, oldIndex);
finalize(mergeDeletes);
return;
}
oldDelete = this.deletes.get(oldIndex);
newDelete = this.newDeletes.get(newIndex);
break;
}
case INCLUDE_OLD_NEXT_OLD: {
mergeDeletes.add(oldDelete);
if(++oldIndex == deletes.size()) {
mergeDown(mergeDeletes, newDeletes, newIndex);
finalize(mergeDeletes);
return;
}
oldDelete = this.deletes.get(oldIndex);
break;
}
case NEXT_OLD: {
if(++oldIndex == deletes.size()) {
// Done with old, add the rest of new to merged and return
mergeDown(mergeDeletes, newDeletes, newIndex);
finalize(mergeDeletes);
return;
}
oldDelete = this.deletes.get(oldIndex);
}
}
}
}
private void finalize(List<Delete> mergeDeletes) {
this.deletes = mergeDeletes;
this.newDeletes = new ArrayList<Delete>();
if(this.deletes.size() > 0){
this.iterator = deletes.iterator();
this.delete = iterator.next();
}
}
private void mergeDown(List<Delete> mergeDeletes, List<Delete> srcDeletes,
int srcIndex) {
int index = srcIndex;
while(index < srcDeletes.size()) {
mergeDeletes.add(srcDeletes.get(index++));
}
}
protected DeleteCompare compareDeletes(Delete oldDelete, Delete newDelete) {
// Compare columns
// Just compairing qualifier portion, can keep on using Bytes.compareTo().
int ret = Bytes.compareTo(oldDelete.buffer, oldDelete.qualifierOffset,
oldDelete.qualifierLength, newDelete.buffer, newDelete.qualifierOffset,
newDelete.qualifierLength);
if(ret <= -1) {
return DeleteCompare.INCLUDE_OLD_NEXT_OLD;
} else if(ret >= 1) {
return DeleteCompare.INCLUDE_NEW_NEXT_NEW;
}
// Same column
// Branches below can be optimized. Keeping like this until testing
// is complete.
if(oldDelete.type == newDelete.type) {
// the one case where we can merge 2 deletes -> 1 delete.
if(oldDelete.type == KeyValue.Type.Delete.getCode()){
if(oldDelete.timestamp > newDelete.timestamp) {
return DeleteCompare.INCLUDE_OLD_NEXT_OLD;
} else if(oldDelete.timestamp < newDelete.timestamp) {
return DeleteCompare.INCLUDE_NEW_NEXT_NEW;
} else {
return DeleteCompare.INCLUDE_OLD_NEXT_BOTH;
}
}
if(oldDelete.timestamp < newDelete.timestamp) {
return DeleteCompare.INCLUDE_NEW_NEXT_BOTH;
}
return DeleteCompare.INCLUDE_OLD_NEXT_BOTH;
}
// old delete is more specific than the new delete.
// if the olddelete is newer then the newdelete, we have to
// keep it
if(oldDelete.type < newDelete.type) {
if(oldDelete.timestamp > newDelete.timestamp) {
return DeleteCompare.INCLUDE_OLD_NEXT_OLD;
} else if(oldDelete.timestamp < newDelete.timestamp) {
return DeleteCompare.NEXT_OLD;
} else {
return DeleteCompare.NEXT_OLD;
}
}
// new delete is more specific than the old delete.
if(oldDelete.type > newDelete.type) {
if(oldDelete.timestamp > newDelete.timestamp) {
return DeleteCompare.NEXT_NEW;
} else if(oldDelete.timestamp < newDelete.timestamp) {
return DeleteCompare.INCLUDE_NEW_NEXT_NEW;
} else {
return DeleteCompare.NEXT_NEW;
}
}
// Should never reach,
// throw exception for assertion?
throw new RuntimeException("GetDeleteTracker:compareDelete reached terminal state");
}
/**
* Internal class used to store the necessary information for a Delete.
* <p>
* Rather than reparsing the KeyValue, or copying fields, this class points
* to the underlying KeyValue buffer with pointers to the qualifier and fields
* for type and timestamp. No parsing work is done in DeleteTracker now.
* <p>
* Fields are public because they are accessed often, directly, and only
* within this class.
*/
protected static class Delete {
byte [] buffer;
int qualifierOffset;
int qualifierLength;
byte type;
long timestamp;
/**
* Constructor
* @param buffer
* @param qualifierOffset
* @param qualifierLength
* @param type
* @param timestamp
*/
public Delete(byte [] buffer, int qualifierOffset, int qualifierLength,
byte type, long timestamp) {
this.buffer = buffer;
this.qualifierOffset = qualifierOffset;
this.qualifierLength = qualifierLength;
this.type = type;
this.timestamp = timestamp;
}
}
}

View File

@ -24,14 +24,11 @@ import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.lang.management.RuntimeMXBean;
import java.rmi.UnexpectedException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.NavigableSet;
import java.util.Set;
import java.util.SortedSet;
import java.util.concurrent.CopyOnWriteArraySet;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@ -41,7 +38,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.regionserver.DeleteCompare.DeleteCode;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;

View File

@ -1,390 +0,0 @@
/**
* Copyright 2010 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KeyComparator;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.util.Bytes;
import java.util.NavigableSet;
/**
* This is the primary class used to process KeyValues during a Get or Scan
* operation.
* <p>
* It encapsulates the handling of the column and version input parameters to
* the query through a {@link ColumnTracker}.
* <p>
* Deletes are handled using the {@link DeleteTracker}.
* <p>
* All other query parameters are accessed from the client-specified Get.
* <p>
* The primary method used is {@link #match} with the current KeyValue. It will
* return a {@link QueryMatcher.MatchCode}
*
* , deletes,
* versions,
*/
public class QueryMatcher {
/**
* {@link #match} return codes. These instruct the scanner moving through
* memstores and StoreFiles what to do with the current KeyValue.
* <p>
* Additionally, this contains "early-out" language to tell the scanner to
* move on to the next File (memstore or Storefile), or to return immediately.
*/
public static enum MatchCode {
/**
* Include KeyValue in the returned result
*/
INCLUDE,
/**
* Do not include KeyValue in the returned result
*/
SKIP,
/**
* Do not include, jump to next StoreFile or memstore (in time order)
*/
NEXT,
/**
* Do not include, return current result
*/
DONE,
/**
* These codes are used by the ScanQueryMatcher
*/
/**
* Done with the row, seek there.
*/
SEEK_NEXT_ROW,
/**
* Done with column, seek to next.
*/
SEEK_NEXT_COL,
/**
* Done with scan, thanks to the row filter.
*/
DONE_SCAN,
}
/** Keeps track of deletes */
protected DeleteTracker deletes;
/** Keeps track of columns and versions */
protected ColumnTracker columns;
/** Key to seek to in memstore and StoreFiles */
protected KeyValue startKey;
/** Row comparator for the region this query is for */
KeyComparator rowComparator;
/** Row the query is on */
protected byte [] row;
/** TimeRange the query is for */
protected TimeRange tr;
/** Oldest allowed version stamp for TTL enforcement */
protected long oldestStamp;
protected Filter filter;
/**
* Constructs a QueryMatcher for a Get.
* @param get
* @param family
* @param columns
* @param ttl
* @param rowComparator
*/
public QueryMatcher(Get get, byte [] family,
NavigableSet<byte[]> columns, long ttl, KeyComparator rowComparator,
int maxVersions) {
this.row = get.getRow();
this.filter = get.getFilter();
this.tr = get.getTimeRange();
this.oldestStamp = System.currentTimeMillis() - ttl;
this.rowComparator = rowComparator;
this.deletes = new GetDeleteTracker();
this.startKey = KeyValue.createFirstOnRow(row);
// Single branch to deal with two types of Gets (columns vs all in family)
if (columns == null || columns.size() == 0) {
this.columns = new WildcardColumnTracker(maxVersions);
} else {
this.columns = new ExplicitColumnTracker(columns, maxVersions);
}
}
// For the subclasses.
protected QueryMatcher() {
super();
}
/**
* Constructs a copy of an existing QueryMatcher with a new row.
* @param matcher
* @param row
*/
public QueryMatcher(QueryMatcher matcher, byte [] row) {
this.row = row;
this.filter = matcher.filter;
this.tr = matcher.getTimeRange();
this.oldestStamp = matcher.getOldestStamp();
this.rowComparator = matcher.getRowComparator();
this.columns = matcher.getColumnTracker();
this.deletes = matcher.getDeleteTracker();
this.startKey = matcher.getStartKey();
reset();
}
/**
* Main method for ColumnMatcher.
* <p>
* Determines whether the specified KeyValue should be included in the
* result or not.
* <p>
* Contains additional language to early-out of the current file or to
* return immediately.
* <p>
* Things to be checked:<ul>
* <li>Row
* <li>TTL
* <li>Type
* <li>TimeRange
* <li>Deletes
* <li>Column
* <li>Versions
* @param kv KeyValue to check
* @return MatchCode: include, skip, next, done
*/
public MatchCode match(KeyValue kv) {
if (this.columns.done()) {
return MatchCode.DONE; // done_row
}
if (this.filter != null && this.filter.filterAllRemaining()) {
return MatchCode.DONE;
}
// Directly act on KV buffer
byte [] bytes = kv.getBuffer();
int offset = kv.getOffset();
int keyLength = Bytes.toInt(bytes, offset);
offset += KeyValue.ROW_OFFSET;
short rowLength = Bytes.toShort(bytes, offset);
offset += Bytes.SIZEOF_SHORT;
// scanners are relying on us to check the row first, and return
// "NEXT" when we are there.
/* Check ROW
* If past query's row, go to next StoreFile
* If not reached query's row, go to next KeyValue
*/
int ret = this.rowComparator.compareRows(row, 0, row.length,
bytes, offset, rowLength);
if (ret <= -1) {
// Have reached the next row
return MatchCode.NEXT; // got_to_next_row (end)
} else if (ret >= 1) {
// At a previous row
return MatchCode.SKIP; // skip_to_cur_row
}
offset += rowLength;
byte familyLength = bytes[offset];
offset += Bytes.SIZEOF_BYTE + familyLength;
int columnLength = keyLength + KeyValue.ROW_OFFSET -
(offset - kv.getOffset()) - KeyValue.TIMESTAMP_TYPE_SIZE;
int columnOffset = offset;
offset += columnLength;
/* Check TTL
* If expired, go to next KeyValue
*/
long timestamp = Bytes.toLong(bytes, offset);
if(isExpired(timestamp)) {
/* KeyValue is expired, skip but don't early out since a non-expired
* kv could come next.
*/
return MatchCode.SKIP; // go to next kv
}
offset += Bytes.SIZEOF_LONG;
/* Check TYPE
* If a delete within (or after) time range, add to deletes
* Move to next KeyValue
*/
byte type = bytes[offset];
// if delete type == delete family, return done_row
if (isDelete(type)) {
if (tr.withinOrAfterTimeRange(timestamp)) {
this.deletes.add(bytes, columnOffset, columnLength, timestamp, type);
}
return MatchCode.SKIP; // skip the delete cell.
}
/* Check TimeRange
* If outside of range, move to next KeyValue
*/
if (!tr.withinTimeRange(timestamp)) {
return MatchCode.SKIP; // optimization chances here.
}
/* Check Deletes
* If deleted, move to next KeyValue
*/
if (!deletes.isEmpty() && deletes.isDeleted(bytes, columnOffset,
columnLength, timestamp)) {
// 2 types of deletes:
// affects 1 cell or 1 column, so just skip the keyvalues.
// - delete family, so just skip to the next row.
return MatchCode.SKIP;
}
/* Check Column and Versions
* Returns a MatchCode directly, identical language
* If matched column without enough versions, include
* If enough versions of this column or does not match, skip
* If have moved past
* If enough versions of everything,
* TODO: No mapping from Filter.ReturnCode to MatchCode.
*/
MatchCode mc = columns.checkColumn(bytes, columnOffset, columnLength);
if (mc == MatchCode.INCLUDE && this.filter != null) {
switch(this.filter.filterKeyValue(kv)) {
case INCLUDE: return MatchCode.INCLUDE;
case SKIP: return MatchCode.SKIP;
default: return MatchCode.DONE;
}
}
return mc;
}
// should be in KeyValue.
protected boolean isDelete(byte type) {
return (type != KeyValue.Type.Put.getCode());
}
protected boolean isExpired(long timestamp) {
return (timestamp < oldestStamp);
}
/**
* If matcher returns SEEK_NEXT_COL you may be able
* to get a hint of the next column to seek to - call this.
* If it returns null, there is no hint.
*
* @return immediately after match returns SEEK_NEXT_COL - null if no hint,
* else the next column we want
*/
public ColumnCount getSeekColumn() {
return this.columns.getColumnHint();
}
/**
* Called after reading each section (memstore, snapshot, storefiles).
* <p>
* This method will update the internal structures to be accurate for
* the next section.
*/
public void update() {
this.deletes.update();
this.columns.update();
}
/**
* Resets the current columns and deletes
*/
public void reset() {
this.deletes.reset();
this.columns.reset();
}
/**
* Set current row
* @param row
*/
public void setRow(byte [] row) {
this.row = row;
}
/**
*
* @return the start key
*/
public KeyValue getStartKey() {
return this.startKey;
}
/**
* @return the TimeRange
*/
public TimeRange getTimeRange() {
return this.tr;
}
/**
* @return the oldest stamp
*/
public long getOldestStamp() {
return this.oldestStamp;
}
/**
* @return current KeyComparator
*/
public KeyComparator getRowComparator() {
return this.rowComparator;
}
/**
* @return ColumnTracker
*/
public ColumnTracker getColumnTracker() {
return this.columns;
}
/**
* @return DeleteTracker
*/
public DeleteTracker getDeleteTracker() {
return this.deletes;
}
/**
*
* @return <code>true</code> when done.
*/
public boolean isDone() {
return this.columns.done();
}
}

View File

@ -23,7 +23,9 @@ package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.util.Bytes;
import java.util.NavigableSet;
@ -31,14 +33,36 @@ import java.util.NavigableSet;
/**
* A query matcher that is specifically designed for the scan case.
*/
public class ScanQueryMatcher extends QueryMatcher {
public class ScanQueryMatcher {
// Optimization so we can skip lots of compares when we decide to skip
// to the next row.
private boolean stickyNextRow;
private byte[] stopRow;
protected TimeRange tr;
protected Filter filter;
/** Keeps track of deletes */
protected DeleteTracker deletes;
/** Keeps track of columns and versions */
protected ColumnTracker columns;
/** Key to seek to in memstore and StoreFiles */
protected KeyValue startKey;
/** Oldest allowed version stamp for TTL enforcement */
protected long oldestStamp;
/** Row comparator for the region this query is for */
KeyValue.KeyComparator rowComparator;
/** Row the query is on */
protected byte [] row;
/**
* Constructs a QueryMatcher for a Scan.
* Constructs a ScanQueryMatcher for a Scan.
* @param scan
* @param family
* @param columns
@ -219,15 +243,79 @@ public class ScanQueryMatcher extends QueryMatcher {
* Set current row
* @param row
*/
@Override
public void setRow(byte [] row) {
this.row = row;
reset();
}
@Override
public void reset() {
super.reset();
this.deletes.reset();
this.columns.reset();
stickyNextRow = false;
}
// should be in KeyValue.
protected boolean isDelete(byte type) {
return (type != KeyValue.Type.Put.getCode());
}
protected boolean isExpired(long timestamp) {
return (timestamp < oldestStamp);
}
/**
*
* @return the start key
*/
public KeyValue getStartKey() {
return this.startKey;
}
/**
* {@link #match} return codes. These instruct the scanner moving through
* memstores and StoreFiles what to do with the current KeyValue.
* <p>
* Additionally, this contains "early-out" language to tell the scanner to
* move on to the next File (memstore or Storefile), or to return immediately.
*/
public static enum MatchCode {
/**
* Include KeyValue in the returned result
*/
INCLUDE,
/**
* Do not include KeyValue in the returned result
*/
SKIP,
/**
* Do not include, jump to next StoreFile or memstore (in time order)
*/
NEXT,
/**
* Do not include, return current result
*/
DONE,
/**
* These codes are used by the ScanQueryMatcher
*/
/**
* Done with the row, seek there.
*/
SEEK_NEXT_ROW,
/**
* Done with column, seek to next.
*/
SEEK_NEXT_COL,
/**
* Done with scan, thanks to the row filter.
*/
DONE_SCAN,
}
}

View File

@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.regionserver;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -65,15 +65,15 @@ public class ScanWildcardColumnTracker implements ColumnTracker {
currentCount = 0;
if (++currentCount > maxVersions)
return MatchCode.SKIP;
return MatchCode.INCLUDE;
return ScanQueryMatcher.MatchCode.SKIP;
return ScanQueryMatcher.MatchCode.INCLUDE;
}
int cmp = Bytes.compareTo(bytes, offset, length,
columnBuffer, columnOffset, columnLength);
if (cmp == 0) {
if (++currentCount > maxVersions)
return MatchCode.SKIP; // skip to next col
return MatchCode.INCLUDE;
return ScanQueryMatcher.MatchCode.SKIP; // skip to next col
return ScanQueryMatcher.MatchCode.INCLUDE;
}
// new col > old col
@ -84,8 +84,8 @@ public class ScanWildcardColumnTracker implements ColumnTracker {
columnLength = length;
currentCount = 0;
if (++currentCount > maxVersions)
return MatchCode.SKIP;
return MatchCode.INCLUDE;
return ScanQueryMatcher.MatchCode.SKIP;
return ScanQueryMatcher.MatchCode.INCLUDE;
}
// new col < oldcol
@ -102,8 +102,8 @@ public class ScanWildcardColumnTracker implements ColumnTracker {
columnLength = length;
currentCount = 0;
if (++currentCount > maxVersions)
return MatchCode.SKIP;
return MatchCode.INCLUDE;
return ScanQueryMatcher.MatchCode.SKIP;
return ScanQueryMatcher.MatchCode.INCLUDE;
}
@Override

View File

@ -1,112 +0,0 @@
/**
* Copyright 2010 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.IOException;
import java.util.List;
/**
* Use to execute a get by scanning all the store files in order.
*/
public class StoreFileGetScan {
private List<HFileScanner> scanners;
private QueryMatcher matcher;
private KeyValue startKey;
/**
* Constructor
* @param scanners
* @param matcher
*/
public StoreFileGetScan(List<HFileScanner> scanners, QueryMatcher matcher) {
this.scanners = scanners;
this.matcher = matcher;
this.startKey = matcher.getStartKey();
}
/**
* Performs a GET operation across multiple StoreFiles.
* <p>
* This style of StoreFile scanning goes through each
* StoreFile in its entirety, most recent first, before
* proceeding to the next StoreFile.
* <p>
* This strategy allows for optimal, stateless (no persisted Scanners)
* early-out scenarios.
* @param result List to add results to
* @throws IOException
*/
public void get(List<KeyValue> result) throws IOException {
for(HFileScanner scanner : this.scanners) {
this.matcher.update();
if (getStoreFile(scanner, result) || matcher.isDone()) {
return;
}
}
}
/**
* Performs a GET operation on a single StoreFile.
* @param scanner
* @param result
* @return true if done with this store, false if must continue to next
* @throws IOException
*/
public boolean getStoreFile(HFileScanner scanner, List<KeyValue> result)
throws IOException {
if (scanner.seekTo(startKey.getBuffer(), startKey.getKeyOffset(),
startKey.getKeyLength()) == -1) {
// No keys in StoreFile at or after specified startKey
// First row may be = our row, so we have to check anyways.
byte [] firstKey = scanner.getReader().getFirstKey();
// Key may be null if storefile is empty.
if (firstKey == null) return false;
short rowLen = Bytes.toShort(firstKey, 0, Bytes.SIZEOF_SHORT);
int rowOffset = Bytes.SIZEOF_SHORT;
if (this.matcher.rowComparator.compareRows(firstKey, rowOffset, rowLen,
startKey.getBuffer(), startKey.getRowOffset(), startKey.getRowLength())
!= 0)
return false;
scanner.seekTo();
}
do {
KeyValue kv = scanner.getKeyValue();
switch(matcher.match(kv)) {
case INCLUDE:
result.add(kv);
break;
case SKIP:
break;
case NEXT:
return false;
case DONE:
return true;
}
} while(scanner.next());
return false;
}
}

View File

@ -231,7 +231,7 @@ class StoreScanner implements KeyValueScanner, InternalScanner, ChangedReadersOb
KeyValue kv;
List<KeyValue> results = new ArrayList<KeyValue>();
LOOP: while((kv = this.heap.peek()) != null) {
QueryMatcher.MatchCode qcode = matcher.match(kv);
ScanQueryMatcher.MatchCode qcode = matcher.match(kv);
//DebugPrint.println("SS peek kv = " + kv + " with qcode = " + qcode);
switch(qcode) {
case INCLUDE:

View File

@ -1,321 +0,0 @@
/**
* Copyright 2010 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
import java.util.ArrayList;
import java.util.List;
/**
* This class is used for the tracking and enforcement of columns and numbers
* of versions during the course of a Get or Scan operation, when all available
* column qualifiers have been asked for in the query.
* <p>
* This class is utilized by {@link QueryMatcher} through two methods:
* <ul><li>{@link #checkColumn} is called when a Put satisfies all other
* conditions of the query. This method returns a {@link MatchCode} to define
* what action should be taken.
* <li>{@link #update} is called at the end of every StoreFile or memstore.
* <p>
* This class is NOT thread-safe as queries are never multi-threaded
*/
public class WildcardColumnTracker implements ColumnTracker {
private int maxVersions;
protected List<ColumnCount> columns;
private int index;
private ColumnCount column;
private List<ColumnCount> newColumns;
private int newIndex;
private ColumnCount newColumn;
/**
* Default constructor.
* @param maxVersions maximum versions to return per columns
*/
public WildcardColumnTracker(int maxVersions) {
this.maxVersions = maxVersions;
reset();
}
public void reset() {
this.index = 0;
this.column = null;
this.columns = null;
this.newColumns = new ArrayList<ColumnCount>();
this.newIndex = 0;
this.newColumn = null;
}
/**
* Can never early-out from reading more storefiles in Wildcard case.
*/
public boolean done() {
return false;
}
// wildcard scanners never have column hints.
public ColumnCount getColumnHint() {
return null;
}
/**
* Checks against the parameters of the query and the columns which have
* already been processed by this query.
* @param bytes KeyValue buffer
* @param offset offset to the start of the qualifier
* @param length length of the qualifier
* @return MatchCode telling QueryMatcher what action to take
*/
public MatchCode checkColumn(byte [] bytes, int offset, int length) {
do {
// Nothing to match against, add to new and include
if(this.column == null && this.newColumn == null) {
newColumns.add(new ColumnCount(bytes, offset, length, 1));
this.newColumn = newColumns.get(newIndex);
return MatchCode.INCLUDE;
}
// Nothing old, compare against new
if(this.column == null && this.newColumn != null) {
int ret = Bytes.compareTo(newColumn.getBuffer(), newColumn.getOffset(),
newColumn.getLength(), bytes, offset, length);
// Same column
if(ret == 0) {
if(newColumn.increment() > this.maxVersions) {
return MatchCode.SKIP;
}
return MatchCode.INCLUDE;
}
// Specified column is bigger than current column
// Move down current column and check again
if(ret <= -1) {
if(++newIndex == newColumns.size()) {
// No more, add to end and include
newColumns.add(new ColumnCount(bytes, offset, length, 1));
this.newColumn = newColumns.get(newIndex);
return MatchCode.INCLUDE;
}
// recursive case
this.newColumn = newColumns.get(newIndex);
continue;
}
// ret >= 1
// Specified column is smaller than current column
// Nothing to match against, add to new and include
newColumns.add(new ColumnCount(bytes, offset, length, 1));
this.newColumn = newColumns.get(++newIndex);
return MatchCode.INCLUDE;
}
// Nothing new, compare against old
if(this.newColumn == null && this.column != null) {
int ret = Bytes.compareTo(column.getBuffer(), column.getOffset(),
column.getLength(), bytes, offset, length);
// Same column
if(ret == 0) {
if(column.increment() > this.maxVersions) {
return MatchCode.SKIP;
}
return MatchCode.INCLUDE;
}
// Specified column is bigger than current column
// Move down current column and check again
if(ret <= -1) {
if(++index == columns.size()) {
// No more, add to new and include (new was empty prior to this)
newColumns.add(new ColumnCount(bytes, offset, length, 1));
this.newColumn = newColumns.get(newIndex);
this.column = null;
return MatchCode.INCLUDE;
}
// recursive case
this.column = columns.get(index);
continue;
}
// ret >= 1
// Specified column is smaller than current column
// Nothing to match against, add to new and include
newColumns.add(new ColumnCount(bytes, offset, length, 1));
this.newColumn = newColumns.get(newIndex);
return MatchCode.INCLUDE;
}
if (column != null && newColumn != null) {
// There are new and old, figure which to check first
int ret = Bytes.compareTo(column.getBuffer(), column.getOffset(),
column.getLength(), newColumn.getBuffer(), newColumn.getOffset(),
newColumn.getLength());
// Old is smaller than new, compare against old
if(ret <= -1) {
ret = Bytes.compareTo(column.getBuffer(), column.getOffset(),
column.getLength(), bytes, offset, length);
// Same column
if(ret == 0) {
if(column.increment() > this.maxVersions) {
return MatchCode.SKIP;
}
return MatchCode.INCLUDE;
}
// Specified column is bigger than current column
// Move down current column and check again
if(ret <= -1) {
if(++index == columns.size()) {
this.column = null;
} else {
this.column = columns.get(index);
}
// Recursive case
continue;
}
// ret >= 1
// Specified column is smaller than current column
// Nothing to match against, add to new and include
newColumns.add(new ColumnCount(bytes, offset, length, 1));
return MatchCode.INCLUDE;
}
}
if (newColumn != null) {
// Cannot be equal, so ret >= 1
// New is smaller than old, compare against new
int ret = Bytes.compareTo(newColumn.getBuffer(), newColumn.getOffset(),
newColumn.getLength(), bytes, offset, length);
// Same column
if(ret == 0) {
if(newColumn.increment() > this.maxVersions) {
return MatchCode.SKIP;
}
return MatchCode.INCLUDE;
}
// Specified column is bigger than current column
// Move down current column and check again
if(ret <= -1) {
if(++newIndex == newColumns.size()) {
this.newColumn = null;
} else {
this.newColumn = newColumns.get(newIndex);
}
// Recursive case
continue;
}
// ret >= 1
// Specified column is smaller than current column
// Nothing to match against, add to new and include
newColumns.add(new ColumnCount(bytes, offset, length, 1));
return MatchCode.INCLUDE;
}
} while(true);
}
/**
* Called at the end of every StoreFile or memstore.
*/
public void update() {
// If no previous columns, use new columns and return
if(this.columns == null || this.columns.size() == 0) {
if(this.newColumns.size() > 0){
finish(newColumns);
}
return;
}
// If no new columns, retain previous columns and return
if(this.newColumns.size() == 0) {
this.index = 0;
this.column = this.columns.get(index);
return;
}
// Merge previous columns with new columns
// There will be no overlapping
List<ColumnCount> mergeColumns = new ArrayList<ColumnCount>(
columns.size() + newColumns.size());
index = 0;
newIndex = 0;
column = columns.get(0);
newColumn = newColumns.get(0);
while(true) {
int ret = Bytes.compareTo(
column.getBuffer(), column.getOffset(),column.getLength(),
newColumn.getBuffer(), newColumn.getOffset(), newColumn.getLength());
// Existing is smaller than new, add existing and iterate it
if(ret <= -1) {
mergeColumns.add(column);
if(++index == columns.size()) {
// No more existing left, merge down rest of new and return
mergeDown(mergeColumns, newColumns, newIndex);
finish(mergeColumns);
return;
}
column = columns.get(index);
continue;
}
// New is smaller than existing, add new and iterate it
mergeColumns.add(newColumn);
if(++newIndex == newColumns.size()) {
// No more new left, merge down rest of existing and return
mergeDown(mergeColumns, columns, index);
finish(mergeColumns);
return;
}
newColumn = newColumns.get(newIndex);
continue;
}
}
private void mergeDown(List<ColumnCount> mergeColumns,
List<ColumnCount> srcColumns, int srcIndex) {
int index = srcIndex;
while(index < srcColumns.size()) {
mergeColumns.add(srcColumns.get(index++));
}
}
private void finish(List<ColumnCount> mergeColumns) {
this.columns = mergeColumns;
this.index = 0;
this.column = this.columns.size() > 0? columns.get(index) : null;
this.newColumns = new ArrayList<ColumnCount>();
this.newIndex = 0;
this.newColumn = null;
}
}

View File

@ -528,7 +528,7 @@ public class HBaseTestingUtility {
results.close();
return digest.toString();
}
/**
* Creates many regions names "aaa" to "zzz".
*
@ -566,7 +566,7 @@ public class HBaseTestingUtility {
};
return createMultiRegions(c, table, columnFamily, KEYS);
}
public int createMultiRegions(final Configuration c, final HTable table,
final byte[] columnFamily, byte [][] startKeys)
throws IOException {
@ -624,7 +624,7 @@ public class HBaseTestingUtility {
s.close();
return rows;
}
/**
* Returns all rows from the .META. table for a given user table
*
@ -828,7 +828,7 @@ public class HBaseTestingUtility {
}
public void cleanupTestDir() throws IOException {
getTestDir().getFileSystem(conf).delete(getTestDir(), true);
getTestDir().getFileSystem(conf).delete(getTestDir(), true);
}
public void waitTableAvailable(byte[] table, long timeoutMillis)
@ -888,16 +888,16 @@ public class HBaseTestingUtility {
* You'll get a NPE if you call before you've started a minidfscluster.
* @param soft Soft limit
* @param hard Hard limit
* @throws NoSuchFieldException
* @throws SecurityException
* @throws IllegalAccessException
* @throws IllegalArgumentException
* @throws NoSuchFieldException
* @throws SecurityException
* @throws IllegalAccessException
* @throws IllegalArgumentException
*/
public void setNameNodeNameSystemLeasePeriod(final int soft, final int hard)
throws SecurityException, NoSuchFieldException, IllegalArgumentException, IllegalAccessException {
// TODO: If 0.20 hadoop do one thing, if 0.21 hadoop do another.
// Not available in 0.20 hdfs. Use reflection to make it happen.
// private NameNode nameNode;
Field field = this.dfsCluster.getClass().getDeclaredField("nameNode");
field.setAccessible(true);
@ -913,10 +913,10 @@ public class HBaseTestingUtility {
* </pre>
* @param stream A DFSClient.DFSOutputStream.
* @param max
* @throws NoSuchFieldException
* @throws SecurityException
* @throws IllegalAccessException
* @throws IllegalArgumentException
* @throws NoSuchFieldException
* @throws SecurityException
* @throws IllegalAccessException
* @throws IllegalArgumentException
*/
public static void setMaxRecoveryErrorCount(final OutputStream stream,
final int max) {
@ -966,7 +966,7 @@ public class HBaseTestingUtility {
// If I get to here and all rows have a Server, then all have been assigned.
if (rows == countOfRegions) break;
LOG.info("Found=" + rows);
Threads.sleep(1000);
Threads.sleep(1000);
}
}

View File

@ -1,211 +0,0 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.regionserver.DeleteCompare.DeleteCode;
import org.apache.hadoop.hbase.util.Bytes;
import junit.framework.TestCase;
public class TestDeleteCompare extends TestCase {
//Cases to compare:
//1. DeleteFamily and whatever of the same row
//2. DeleteColumn and whatever of the same row + qualifier
//3. Delete and the matching put
//4. Big test that include starting on the wrong row and qualifier
public void testDeleteCompare_DeleteFamily() {
//Creating memstore
Set<KeyValue> memstore = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col2", 1, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col3", 3, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col3", 2, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col3", 1, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 1, "d-c"));
//Creating expected result
List<DeleteCode> expected = new ArrayList<DeleteCode>();
expected.add(DeleteCode.SKIP);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.SKIP);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.DONE);
KeyValue delete = KeyValueTestUtil.create("row11",
"fam", "", 2, KeyValue.Type.DeleteFamily, "dont-care");
byte [] deleteBuffer = delete.getBuffer();
int deleteRowOffset = delete.getRowOffset();
short deleteRowLen = delete.getRowLength();
int deleteQualifierOffset = delete.getQualifierOffset();
int deleteQualifierLen = delete.getQualifierLength();
int deleteTimestampOffset = deleteQualifierOffset + deleteQualifierLen;
byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG];
List<DeleteCode> actual = new ArrayList<DeleteCode>();
for(KeyValue mem : memstore){
actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset,
deleteRowLen, deleteQualifierOffset, deleteQualifierLen,
deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR));
}
assertEquals(expected.size(), actual.size());
for(int i=0; i<expected.size(); i++){
assertEquals(expected.get(i), actual.get(i));
}
}
public void testDeleteCompare_DeleteColumn() {
//Creating memstore
Set<KeyValue> memstore = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 1, "d-c"));
//Creating expected result
List<DeleteCode> expected = new ArrayList<DeleteCode>();
expected.add(DeleteCode.SKIP);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.DONE);
KeyValue delete = KeyValueTestUtil.create("row11", "fam", "col1", 2,
KeyValue.Type.DeleteColumn, "dont-care");
byte [] deleteBuffer = delete.getBuffer();
int deleteRowOffset = delete.getRowOffset();
short deleteRowLen = delete.getRowLength();
int deleteQualifierOffset = delete.getQualifierOffset();
int deleteQualifierLen = delete.getQualifierLength();
int deleteTimestampOffset = deleteQualifierOffset + deleteQualifierLen;
byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG];
List<DeleteCode> actual = new ArrayList<DeleteCode>();
for(KeyValue mem : memstore){
actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset,
deleteRowLen, deleteQualifierOffset, deleteQualifierLen,
deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR));
}
assertEquals(expected.size(), actual.size());
for(int i=0; i<expected.size(); i++){
assertEquals(expected.get(i), actual.get(i));
}
}
public void testDeleteCompare_Delete() {
//Creating memstore
Set<KeyValue> memstore = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
//Creating expected result
List<DeleteCode> expected = new ArrayList<DeleteCode>();
expected.add(DeleteCode.SKIP);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.DONE);
KeyValue delete = KeyValueTestUtil.create("row11", "fam", "col1", 2,
KeyValue.Type.Delete, "dont-care");
byte [] deleteBuffer = delete.getBuffer();
int deleteRowOffset = delete.getRowOffset();
short deleteRowLen = delete.getRowLength();
int deleteQualifierOffset = delete.getQualifierOffset();
int deleteQualifierLen = delete.getQualifierLength();
int deleteTimestampOffset = deleteQualifierOffset + deleteQualifierLen;
byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG];
List<DeleteCode> actual = new ArrayList<DeleteCode>();
for(KeyValue mem : memstore){
actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset,
deleteRowLen, deleteQualifierOffset, deleteQualifierLen,
deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR));
}
assertEquals(expected.size(), actual.size());
for(int i=0; i<expected.size(); i++){
assertEquals(expected.get(i), actual.get(i));
}
}
public void testDeleteCompare_Multiple() {
//Creating memstore
Set<KeyValue> memstore = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 4, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 3, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 2, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 1,
KeyValue.Type.Delete, "dont-care"));
memstore.add(KeyValueTestUtil.create("row31", "fam", "col1", 1, "dont-care"));
//Creating expected result
List<DeleteCode> expected = new ArrayList<DeleteCode>();
expected.add(DeleteCode.SKIP);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.DELETE);
expected.add(DeleteCode.DONE);
KeyValue delete = KeyValueTestUtil.create("row21", "fam", "col1", 5,
KeyValue.Type.DeleteColumn, "dont-care");
byte [] deleteBuffer = delete.getBuffer();
int deleteRowOffset = delete.getRowOffset();
short deleteRowLen = delete.getRowLength();
int deleteQualifierOffset = delete.getQualifierOffset();
int deleteQualifierLen = delete.getQualifierLength();
int deleteTimestampOffset = deleteQualifierOffset + deleteQualifierLen;
byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG];
List<DeleteCode> actual = new ArrayList<DeleteCode>();
for(KeyValue mem : memstore){
actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset,
deleteRowLen, deleteQualifierOffset, deleteQualifierLen,
deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR));
}
assertEquals(expected.size(), actual.size());
for(int i=0; i<expected.size(); i++){
assertEquals(expected.get(i), actual.get(i));
}
}
}

View File

@ -26,8 +26,7 @@ import java.util.TreeSet;
import java.util.Arrays;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
@ -49,7 +48,7 @@ public class TestExplicitColumnTracker extends HBaseTestCase {
//Initialize result
List<MatchCode> result = new ArrayList<MatchCode>();
List<ScanQueryMatcher.MatchCode> result = new ArrayList<ScanQueryMatcher.MatchCode>();
//"Match"
for(byte [] col : scannerColumns){
@ -76,12 +75,12 @@ public class TestExplicitColumnTracker extends HBaseTestCase {
//Looking for every other
columns.add(col2);
columns.add(col4);
List<MatchCode> expected = new ArrayList<MatchCode>();
expected.add(MatchCode.SKIP);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.SKIP);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.DONE);
List<MatchCode> expected = new ArrayList<ScanQueryMatcher.MatchCode>();
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.DONE);
int maxVersions = 1;
//Create "Scanner"
@ -106,26 +105,26 @@ public class TestExplicitColumnTracker extends HBaseTestCase {
columns.add(col2);
columns.add(col4);
List<MatchCode> expected = new ArrayList<MatchCode>();
expected.add(MatchCode.SKIP);
expected.add(MatchCode.SKIP);
expected.add(MatchCode.SKIP);
List<ScanQueryMatcher.MatchCode> expected = new ArrayList<ScanQueryMatcher.MatchCode>();
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.SKIP);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(MatchCode.SKIP);
expected.add(MatchCode.SKIP);
expected.add(MatchCode.SKIP);
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.DONE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.DONE);
expected.add(MatchCode.DONE);
expected.add(MatchCode.DONE);
expected.add(MatchCode.DONE);
expected.add(ScanQueryMatcher.MatchCode.DONE);
expected.add(ScanQueryMatcher.MatchCode.DONE);
expected.add(ScanQueryMatcher.MatchCode.DONE);
int maxVersions = 2;
//Create "Scanner"
@ -183,10 +182,10 @@ public class TestExplicitColumnTracker extends HBaseTestCase {
col2, col3, col5 }));
List<byte[]> scanner = Arrays.<byte[]>asList(
new byte[][] { col1, col4 });
List<MatchCode> expected = Arrays.<MatchCode>asList(
new MatchCode[] {
MatchCode.SKIP,
MatchCode.SKIP });
List<ScanQueryMatcher.MatchCode> expected = Arrays.<ScanQueryMatcher.MatchCode>asList(
new ScanQueryMatcher.MatchCode[] {
ScanQueryMatcher.MatchCode.SKIP,
ScanQueryMatcher.MatchCode.SKIP });
runTest(1, columns, scanner, expected);
}
}

View File

@ -1,327 +0,0 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.regionserver.GetDeleteTracker.Delete;
import org.apache.hadoop.hbase.util.Bytes;
public class TestGetDeleteTracker extends HBaseTestCase {
private static final boolean PRINT = true;
private byte[] col1 = null;
private byte[] col2 = null;
private int col1Len = 0;
private int col2Len = 0;
private byte[] empty = null;
private long ts1 = 0L;
private long ts2 = 0L;
private long ts3 = 0L;
private Delete del10 = null;
private Delete del11 = null;
private Delete delQf10 = null;
private Delete delQf11 = null;
private Delete delFam10 = null;
private Delete del20 = null;
private Delete del21 = null;
private Delete delQf20 = null;
private Delete delQf21 = null;
private Delete delFam20 = null;
private Delete del30 = null;
GetDeleteTracker dt = null;
private byte del = KeyValue.Type.Delete.getCode();
private byte delCol = KeyValue.Type.DeleteColumn.getCode();
private byte delFam = KeyValue.Type.DeleteFamily.getCode();
protected void setUp() throws Exception {
super.setUp();
dt = new GetDeleteTracker();
col1 = "col".getBytes();
col2 = "col2".getBytes();
col1Len = col1.length;
col2Len = col2.length;
empty = new byte[0];
//ts1
ts1 = System.nanoTime();
del10 = new Delete(col1, 0, col1Len, del, ts1);
del11 = new Delete(col2, 0, col2Len, del, ts1);
delQf10 = new Delete(col1, 0, col1Len, delCol, ts1);
delQf11 = new Delete(col2, 0, col2Len, delCol, ts1);
delFam10 = new Delete(empty, 0, 0, delFam, ts1);
//ts2
ts2 = System.nanoTime();
del20 = new Delete(col1, 0, col1Len, del, ts2);
del21 = new Delete(col2, 0, col2Len, del, ts2);
delQf20 = new Delete(col1, 0, col1Len, delCol, ts2);
delQf21 = new Delete(col2, 0, col2Len, delCol, ts2);
delFam20 = new Delete(empty, 0, 0, delFam, ts1);
//ts3
ts3 = System.nanoTime();
del30 = new Delete(col1, 0, col1Len, del, ts3);
}
public void testUpdate_CompareDeletes() {
GetDeleteTracker.DeleteCompare res = null;
//Testing Delete and Delete
res = dt.compareDeletes(del10, del10);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_BOTH, res);
//Testing Delete qf1 and Delete qf2 and <==>
res = dt.compareDeletes(del10, del11);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res);
res = dt.compareDeletes(del11, del10);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res);
//Testing Delete ts1 and Delete ts2 and <==>
res = dt.compareDeletes(del10, del20);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res);
res = dt.compareDeletes(del20, del10);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res);
//Testing DeleteColumn and DeleteColumn
res = dt.compareDeletes(delQf10, delQf10);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_BOTH, res);
//Testing DeleteColumn qf1 and DeleteColumn qf2 and <==>
res = dt.compareDeletes(delQf10, delQf11);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res);
res = dt.compareDeletes(delQf11, delQf10);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res);
//Testing DeleteColumn ts1 and DeleteColumn ts2 and <==>
res = dt.compareDeletes(delQf10, delQf20);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_BOTH, res);
res = dt.compareDeletes(delQf20, delQf10);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_BOTH, res);
//Testing Delete and DeleteColumn and <==>
res = dt.compareDeletes(del10, delQf10);
assertEquals(DeleteTracker.DeleteCompare.NEXT_OLD, res);
res = dt.compareDeletes(delQf10, del10);
assertEquals(DeleteTracker.DeleteCompare.NEXT_NEW, res);
//Testing Delete qf1 and DeleteColumn qf2 and <==>
res = dt.compareDeletes(del10, delQf11);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res);
res = dt.compareDeletes(delQf11, del10);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res);
//Testing Delete qf2 and DeleteColumn qf1 and <==>
res = dt.compareDeletes(del11, delQf10);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res);
res = dt.compareDeletes(delQf10, del11);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res);
//Testing Delete ts2 and DeleteColumn ts1 and <==>
res = dt.compareDeletes(del20, delQf10);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res);
res = dt.compareDeletes(delQf10, del20);
assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res);
//Testing Delete ts1 and DeleteColumn ts2 and <==>
res = dt.compareDeletes(del10, delQf20);
assertEquals(DeleteTracker.DeleteCompare.NEXT_OLD, res);
res = dt.compareDeletes(delQf20, del10);
assertEquals(DeleteTracker.DeleteCompare.NEXT_NEW, res);
}
public void testUpdate(){
//Building lists
List<Delete> dels1 = new ArrayList<Delete>();
dels1.add(delQf10);
dels1.add(del21);
List<Delete> dels2 = new ArrayList<Delete>();
dels2.add(delFam10);
dels2.add(del30);
dels2.add(delQf20);
List<Delete> res = new ArrayList<Delete>();
res.add(del30);
res.add(delQf20);
res.add(del21);
//Adding entries
for(Delete del : dels1){
dt.add(del.buffer, del.qualifierOffset, del.qualifierLength,
del.timestamp, del.type);
}
//update()
dt.update();
//Check deleteList
List<Delete> delList = dt.deletes;
assertEquals(dels1.size(), delList.size());
for(int i=0; i<dels1.size(); i++){
assertEquals(0, Bytes.compareTo(dels1.get(i).buffer,
delList.get(i).buffer));
assertEquals(dels1.get(i).qualifierOffset, delList.get(i).qualifierOffset);
assertEquals(dels1.get(i).qualifierLength, delList.get(i).qualifierLength);
assertEquals(dels1.get(i).timestamp, delList.get(i).timestamp);
assertEquals(dels1.get(i).type, delList.get(i).type);
}
//Add more entries
for(Delete del : dels2){
dt.add(del.buffer, del.qualifierOffset, del.qualifierLength,
del.timestamp, del.type);
}
//Update()
dt.update();
//Check deleteList
delList = dt.deletes;
for(int i=0; i<res.size(); i++){
assertEquals(0, Bytes.compareTo(res.get(i).buffer,
delList.get(i).buffer));
assertEquals(res.get(i).qualifierOffset, delList.get(i).qualifierOffset);
assertEquals(res.get(i).qualifierLength, delList.get(i).qualifierLength);
assertEquals(res.get(i).timestamp, delList.get(i).timestamp);
assertEquals(res.get(i).type, delList.get(i).type);
if(PRINT){
System.out.println("Qf " +new String(delList.get(i).buffer) +
", timestamp, " +delList.get(i).timestamp+
", type " +KeyValue.Type.codeToType(delList.get(i).type));
}
}
}
/**
* Test if a KeyValue is in the lists of deletes already. Cases that needs to
* be tested are:
* Not deleted
* Deleted by a Delete
* Deleted by a DeleteColumn
* Deleted by a DeleteFamily
*/
public void testIsDeleted_NotDeleted(){
//Building lists
List<Delete> dels = new ArrayList<Delete>();
dels.add(delQf10);
dels.add(del21);
//Adding entries
for(Delete del : dels){
dt.add(del.buffer, del.qualifierOffset, del.qualifierLength,
del.timestamp, del.type);
}
//update()
dt.update();
assertFalse(dt.isDeleted(col2, 0, col2Len, ts3));
assertFalse(dt.isDeleted(col2, 0, col2Len, ts1));
}
public void testIsDeleted_Delete(){
//Building lists
List<Delete> dels = new ArrayList<Delete>();
dels.add(del21);
//Adding entries
for(Delete del : dels){
dt.add(del.buffer, del.qualifierOffset, del.qualifierLength,
del.timestamp, del.type);
}
//update()
dt.update();
assertEquals(true, dt.isDeleted(col2, 0, col2Len, ts2));
}
public void testIsDeleted_DeleteColumn(){
//Building lists
List<Delete> dels = new ArrayList<Delete>();
dels.add(delQf21);
//Adding entries
for(Delete del : dels){
dt.add(del.buffer, del.qualifierOffset, del.qualifierLength,
del.timestamp, del.type);
}
//update()
dt.update();
assertEquals(true, dt.isDeleted(col2, 0, col2Len, ts1));
}
public void testIsDeleted_DeleteFamily(){
//Building lists
List<Delete> dels = new ArrayList<Delete>();
dels.add(delFam20);
//Adding entries
for(Delete del : dels){
dt.add(del.buffer, del.qualifierOffset, del.qualifierLength,
del.timestamp, del.type);
}
//update()
dt.update();
assertEquals(true, dt.isDeleted(col2, 0, col2Len, ts1));
}
// HBASE-1951
public void testStackOverflow() {
List<Delete> dels = new ArrayList<Delete>();
Delete adel = new Delete(col1, 0, col1Len, del, 0L);
for(long i = 0; i < 9000; i++) {
dt.add(adel.buffer, adel.qualifierOffset, adel.qualifierLength,
i, adel.type);
}
//update()
dt.update();
assertEquals(false, dt.isDeleted(col2, 0, col2Len, 7000000));
}
}

View File

@ -361,7 +361,7 @@ public class TestHRegion extends HBaseTestCase {
assertEquals(OperationStatusCode.SUCCESS, codes[i]);
}
assertEquals(1, HLog.getSyncOps());
LOG.info("Next a batch put with one invalid family");
puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
codes = this.region.put(puts);
@ -371,7 +371,7 @@ public class TestHRegion extends HBaseTestCase {
OperationStatusCode.SUCCESS, codes[i]);
}
assertEquals(1, HLog.getSyncOps());
LOG.info("Next a batch put that has to break into two batches to avoid a lock");
Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
@ -396,7 +396,7 @@ public class TestHRegion extends HBaseTestCase {
if (System.currentTimeMillis() - startWait > 10000) {
fail("Timed out waiting for thread to sync first minibatch");
}
}
}
LOG.info("...releasing row lock, which should let put thread continue");
region.releaseRowLock(lockedRow);
LOG.info("...joining on thread");
@ -408,7 +408,7 @@ public class TestHRegion extends HBaseTestCase {
assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
OperationStatusCode.SUCCESS, codes[i]);
}
LOG.info("Nexta, a batch put which uses an already-held lock");
lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
LOG.info("...obtained row lock");
@ -427,13 +427,13 @@ public class TestHRegion extends HBaseTestCase {
}
// Make sure we didn't do an extra batch
assertEquals(1, HLog.getSyncOps());
// Make sure we still hold lock
assertTrue(region.isRowLocked(lockedRow));
LOG.info("...releasing lock");
region.releaseRowLock(lockedRow);
}
//////////////////////////////////////////////////////////////////////////////
// checkAndMutate tests
//////////////////////////////////////////////////////////////////////////////
@ -834,7 +834,7 @@ public class TestHRegion extends HBaseTestCase {
result = region.get(get, null);
assertEquals(0, result.size());
}
/**
* Tests that the special LATEST_TIMESTAMP option for puts gets
* replaced by the actual timestamp
@ -863,7 +863,7 @@ public class TestHRegion extends HBaseTestCase {
LOG.info("Got: " + kv);
assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
// Check same with WAL enabled (historically these took different
// code paths, so check both)
row = Bytes.toBytes("row2");

View File

@ -25,12 +25,11 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.KeyValue.KeyComparator;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
@ -53,6 +52,7 @@ public class TestQueryMatcher extends HBaseTestCase {
long ttl = Long.MAX_VALUE;
KeyComparator rowComparator;
private Scan scan;
public void setUp() throws Exception {
super.setUp();
@ -74,6 +74,7 @@ public class TestQueryMatcher extends HBaseTestCase {
get.addColumn(fam2, col2);
get.addColumn(fam2, col4);
get.addColumn(fam2, col5);
this.scan = new Scan(get);
rowComparator = KeyValue.KEY_COMPARATOR;
@ -85,15 +86,15 @@ public class TestQueryMatcher extends HBaseTestCase {
//of just byte []
//Expected result
List<MatchCode> expected = new ArrayList<MatchCode>();
expected.add(MatchCode.SKIP);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.SKIP);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.DONE);
List<MatchCode> expected = new ArrayList<ScanQueryMatcher.MatchCode>();
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.DONE);
QueryMatcher qm = new QueryMatcher(get, fam2,
ScanQueryMatcher qm = new ScanQueryMatcher(scan, fam2,
get.getFamilyMap().get(fam2), ttl, rowComparator, 1);
List<KeyValue> memstore = new ArrayList<KeyValue>();
@ -105,9 +106,10 @@ public class TestQueryMatcher extends HBaseTestCase {
memstore.add(new KeyValue(row2, fam1, col1, data));
List<MatchCode> actual = new ArrayList<MatchCode>();
List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
for(KeyValue kv : memstore){
qm.setRow(kv.getRow());
actual.add(qm.match(kv));
}
@ -128,15 +130,15 @@ public class TestQueryMatcher extends HBaseTestCase {
//of just byte []
//Expected result
List<MatchCode> expected = new ArrayList<MatchCode>();
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.NEXT);
List<MatchCode> expected = new ArrayList<ScanQueryMatcher.MatchCode>();
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.NEXT);
QueryMatcher qm = new QueryMatcher(get, fam2, null, ttl, rowComparator, 1);
ScanQueryMatcher qm = new ScanQueryMatcher(scan, fam2, null, ttl, rowComparator, 1);
List<KeyValue> memstore = new ArrayList<KeyValue>();
memstore.add(new KeyValue(row1, fam2, col1, data));
@ -146,9 +148,10 @@ public class TestQueryMatcher extends HBaseTestCase {
memstore.add(new KeyValue(row1, fam2, col5, data));
memstore.add(new KeyValue(row2, fam1, col1, data));
List<MatchCode> actual = new ArrayList<MatchCode>();
List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
for(KeyValue kv : memstore){
for(KeyValue kv : memstore) {
qm.setRow(kv.getRow());
actual.add(qm.match(kv));
}
@ -164,7 +167,7 @@ public class TestQueryMatcher extends HBaseTestCase {
/**
* Verify that {@link QueryMatcher} only skips expired KeyValue
* Verify that {@link ScanQueryMatcher} only skips expired KeyValue
* instances and does not exit early from the row (skipping
* later non-expired KeyValues). This version mimics a Get with
* explicitly specified column qualifiers.
@ -176,15 +179,15 @@ public class TestQueryMatcher extends HBaseTestCase {
long testTTL = 1000;
MatchCode [] expected = new MatchCode[] {
MatchCode.SKIP,
MatchCode.INCLUDE,
MatchCode.SKIP,
MatchCode.INCLUDE,
MatchCode.SKIP,
MatchCode.NEXT
ScanQueryMatcher.MatchCode.SKIP,
ScanQueryMatcher.MatchCode.INCLUDE,
ScanQueryMatcher.MatchCode.SKIP,
ScanQueryMatcher.MatchCode.INCLUDE,
ScanQueryMatcher.MatchCode.SKIP,
ScanQueryMatcher.MatchCode.NEXT
};
QueryMatcher qm = new QueryMatcher(get, fam2,
ScanQueryMatcher qm = new ScanQueryMatcher(scan, fam2,
get.getFamilyMap().get(fam2), testTTL, rowComparator, 1);
long now = System.currentTimeMillis();
@ -199,6 +202,7 @@ public class TestQueryMatcher extends HBaseTestCase {
List<MatchCode> actual = new ArrayList<MatchCode>(kvs.length);
for (KeyValue kv : kvs) {
qm.setRow(kv.getRow());
actual.add( qm.match(kv) );
}
@ -214,7 +218,7 @@ public class TestQueryMatcher extends HBaseTestCase {
/**
* Verify that {@link QueryMatcher} only skips expired KeyValue
* Verify that {@link ScanQueryMatcher} only skips expired KeyValue
* instances and does not exit early from the row (skipping
* later non-expired KeyValues). This version mimics a Get with
* wildcard-inferred column qualifiers.
@ -226,15 +230,15 @@ public class TestQueryMatcher extends HBaseTestCase {
long testTTL = 1000;
MatchCode [] expected = new MatchCode[] {
MatchCode.INCLUDE,
MatchCode.INCLUDE,
MatchCode.SKIP,
MatchCode.INCLUDE,
MatchCode.SKIP,
MatchCode.NEXT
ScanQueryMatcher.MatchCode.INCLUDE,
ScanQueryMatcher.MatchCode.INCLUDE,
ScanQueryMatcher.MatchCode.SKIP,
ScanQueryMatcher.MatchCode.INCLUDE,
ScanQueryMatcher.MatchCode.SKIP,
ScanQueryMatcher.MatchCode.NEXT
};
QueryMatcher qm = new QueryMatcher(get, fam2,
ScanQueryMatcher qm = new ScanQueryMatcher(scan, fam2,
null, testTTL, rowComparator, 1);
long now = System.currentTimeMillis();
@ -247,8 +251,9 @@ public class TestQueryMatcher extends HBaseTestCase {
new KeyValue(row2, fam1, col1, now-10, data)
};
List<MatchCode> actual = new ArrayList<MatchCode>(kvs.length);
List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>(kvs.length);
for (KeyValue kv : kvs) {
qm.setRow(kv.getRow());
actual.add( qm.match(kv) );
}

View File

@ -24,7 +24,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
public class TestScanWildcardColumnTracker extends HBaseTestCase {
@ -32,7 +32,6 @@ public class TestScanWildcardColumnTracker extends HBaseTestCase {
final static int VERSIONS = 2;
public void testCheckColumn_Ok() {
//Create a WildcardColumnTracker
ScanWildcardColumnTracker tracker =
new ScanWildcardColumnTracker(VERSIONS);
@ -45,15 +44,15 @@ public class TestScanWildcardColumnTracker extends HBaseTestCase {
//Setting up expected result
List<MatchCode> expected = new ArrayList<MatchCode>();
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
List<MatchCode> actual = new ArrayList<MatchCode>();
List<ScanQueryMatcher.MatchCode> actual = new ArrayList<MatchCode>();
for(byte [] qualifier : qualifiers) {
MatchCode mc = tracker.checkColumn(qualifier, 0, qualifier.length);
ScanQueryMatcher.MatchCode mc = tracker.checkColumn(qualifier, 0, qualifier.length);
actual.add(mc);
}
@ -64,7 +63,6 @@ public class TestScanWildcardColumnTracker extends HBaseTestCase {
}
public void testCheckColumn_EnforceVersions() {
//Create a WildcardColumnTracker
ScanWildcardColumnTracker tracker =
new ScanWildcardColumnTracker(VERSIONS);
@ -76,13 +74,13 @@ public class TestScanWildcardColumnTracker extends HBaseTestCase {
qualifiers.add(Bytes.toBytes("qualifer2"));
//Setting up expected result
List<MatchCode> expected = new ArrayList<MatchCode>();
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.SKIP);
expected.add(MatchCode.INCLUDE);
List<ScanQueryMatcher.MatchCode> expected = new ArrayList<MatchCode>();
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
expected.add(ScanQueryMatcher.MatchCode.SKIP);
expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
List<MatchCode> actual = new ArrayList<MatchCode>();
List<MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
for(byte [] qualifier : qualifiers) {
MatchCode mc = tracker.checkColumn(qualifier, 0, qualifier.length);
@ -96,7 +94,6 @@ public class TestScanWildcardColumnTracker extends HBaseTestCase {
}
public void DisabledTestCheckColumn_WrongOrder() {
//Create a WildcardColumnTracker
ScanWildcardColumnTracker tracker =
new ScanWildcardColumnTracker(VERSIONS);

View File

@ -1,371 +0,0 @@
/*
* Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
public class TestWildcardColumnTracker extends HBaseTestCase {
private boolean PRINT = false;
public void testGet_SingleVersion() {
if (PRINT) {
System.out.println("SingleVersion");
}
byte[] col1 = Bytes.toBytes("col1");
byte[] col2 = Bytes.toBytes("col2");
byte[] col3 = Bytes.toBytes("col3");
byte[] col4 = Bytes.toBytes("col4");
byte[] col5 = Bytes.toBytes("col5");
//Create tracker
List<MatchCode> expected = new ArrayList<MatchCode>();
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
int maxVersions = 1;
ColumnTracker exp = new WildcardColumnTracker(maxVersions);
//Create "Scanner"
List<byte[]> scanner = new ArrayList<byte[]>();
scanner.add(col1);
scanner.add(col2);
scanner.add(col3);
scanner.add(col4);
scanner.add(col5);
//Initialize result
List<MatchCode> result = new ArrayList<MatchCode>();
//"Match"
for(byte [] col : scanner){
result.add(exp.checkColumn(col, 0, col.length));
}
assertEquals(expected.size(), result.size());
for(int i=0; i< expected.size(); i++){
assertEquals(expected.get(i), result.get(i));
if(PRINT){
System.out.println("Expected " +expected.get(i) + ", actual " +
result.get(i));
}
}
}
public void testGet_MultiVersion() {
if(PRINT) {
System.out.println("\nMultiVersion");
}
byte [] col1 = Bytes.toBytes("col1");
byte [] col2 = Bytes.toBytes("col2");
byte [] col3 = Bytes.toBytes("col3");
byte [] col4 = Bytes.toBytes("col4");
byte [] col5 = Bytes.toBytes("col5");
//Create tracker
List<MatchCode> expected = new ArrayList<MatchCode>();
int size = 5;
for(int i=0; i<size; i++){
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.INCLUDE);
expected.add(MatchCode.SKIP);
}
int maxVersions = 2;
ColumnTracker exp = new WildcardColumnTracker(maxVersions);
//Create "Scanner"
List<byte[]> scanner = new ArrayList<byte[]>();
scanner.add(col1);
scanner.add(col1);
scanner.add(col1);
scanner.add(col2);
scanner.add(col2);
scanner.add(col2);
scanner.add(col3);
scanner.add(col3);
scanner.add(col3);
scanner.add(col4);
scanner.add(col4);
scanner.add(col4);
scanner.add(col5);
scanner.add(col5);
scanner.add(col5);
//Initialize result
List<MatchCode> result = new ArrayList<MatchCode>();
//"Match"
for(byte [] col : scanner){
result.add(exp.checkColumn(col, 0, col.length));
}
assertEquals(expected.size(), result.size());
for(int i=0; i< expected.size(); i++){
assertEquals(expected.get(i), result.get(i));
if(PRINT){
System.out.println("Expected " +expected.get(i) + ", actual " +
result.get(i));
}
}
}
public void testUpdate_SameColumns(){
if(PRINT) {
System.out.println("\nUpdate_SameColumns");
}
byte [] col1 = Bytes.toBytes("col1");
byte [] col2 = Bytes.toBytes("col2");
byte [] col3 = Bytes.toBytes("col3");
byte [] col4 = Bytes.toBytes("col4");
byte [] col5 = Bytes.toBytes("col5");
//Create tracker
List<MatchCode> expected = new ArrayList<MatchCode>();
int size = 10;
for(int i=0; i<size; i++){
expected.add(MatchCode.INCLUDE);
}
for(int i=0; i<5; i++){
expected.add(MatchCode.SKIP);
}
int maxVersions = 2;
ColumnTracker wild = new WildcardColumnTracker(maxVersions);
//Create "Scanner"
List<byte[]> scanner = new ArrayList<byte[]>();
scanner.add(col1);
scanner.add(col2);
scanner.add(col3);
scanner.add(col4);
scanner.add(col5);
//Initialize result
List<MatchCode> result = new ArrayList<MatchCode>();
//"Match"
for(int i=0; i<3; i++){
for(byte [] col : scanner){
result.add(wild.checkColumn(col, 0, col.length));
}
wild.update();
}
assertEquals(expected.size(), result.size());
for(int i=0; i<expected.size(); i++){
assertEquals(expected.get(i), result.get(i));
if(PRINT){
System.out.println("Expected " +expected.get(i) + ", actual " +
result.get(i));
}
}
}
public void testUpdate_NewColumns(){
if(PRINT) {
System.out.println("\nUpdate_NewColumns");
}
byte [] col1 = Bytes.toBytes("col1");
byte [] col2 = Bytes.toBytes("col2");
byte [] col3 = Bytes.toBytes("col3");
byte [] col4 = Bytes.toBytes("col4");
byte [] col5 = Bytes.toBytes("col5");
byte [] col6 = Bytes.toBytes("col6");
byte [] col7 = Bytes.toBytes("col7");
byte [] col8 = Bytes.toBytes("col8");
byte [] col9 = Bytes.toBytes("col9");
byte [] col0 = Bytes.toBytes("col0");
//Create tracker
List<MatchCode> expected = new ArrayList<MatchCode>();
int size = 10;
for(int i=0; i<size; i++){
expected.add(MatchCode.INCLUDE);
}
for(int i=0; i<5; i++){
expected.add(MatchCode.SKIP);
}
int maxVersions = 1;
ColumnTracker wild = new WildcardColumnTracker(maxVersions);
//Create "Scanner"
List<byte[]> scanner = new ArrayList<byte[]>();
scanner.add(col0);
scanner.add(col1);
scanner.add(col2);
scanner.add(col3);
scanner.add(col4);
//Initialize result
List<MatchCode> result = new ArrayList<MatchCode>();
for(byte [] col : scanner){
result.add(wild.checkColumn(col, 0, col.length));
}
wild.update();
//Create "Scanner1"
List<byte[]> scanner1 = new ArrayList<byte[]>();
scanner1.add(col5);
scanner1.add(col6);
scanner1.add(col7);
scanner1.add(col8);
scanner1.add(col9);
for(byte [] col : scanner1){
result.add(wild.checkColumn(col, 0, col.length));
}
wild.update();
//Scanner again
for(byte [] col : scanner){
result.add(wild.checkColumn(col, 0, col.length));
}
//"Match"
assertEquals(expected.size(), result.size());
for(int i=0; i<expected.size(); i++){
assertEquals(expected.get(i), result.get(i));
if(PRINT){
System.out.println("Expected " +expected.get(i) + ", actual " +
result.get(i));
}
}
}
public void testUpdate_MixedColumns(){
if(PRINT) {
System.out.println("\nUpdate_NewColumns");
}
byte [] col0 = Bytes.toBytes("col0");
byte [] col1 = Bytes.toBytes("col1");
byte [] col2 = Bytes.toBytes("col2");
byte [] col3 = Bytes.toBytes("col3");
byte [] col4 = Bytes.toBytes("col4");
byte [] col5 = Bytes.toBytes("col5");
byte [] col6 = Bytes.toBytes("col6");
byte [] col7 = Bytes.toBytes("col7");
byte [] col8 = Bytes.toBytes("col8");
byte [] col9 = Bytes.toBytes("col9");
//Create tracker
List<MatchCode> expected = new ArrayList<MatchCode>();
int size = 5;
for(int i=0; i<size; i++){
expected.add(MatchCode.INCLUDE);
}
for(int i=0; i<size; i++){
expected.add(MatchCode.SKIP);
}
for(int i=0; i<size; i++){
expected.add(MatchCode.INCLUDE);
}
for(int i=0; i<size; i++){
expected.add(MatchCode.SKIP);
}
int maxVersions = 1;
ColumnTracker wild = new WildcardColumnTracker(maxVersions);
//Create "Scanner"
List<byte[]> scanner = new ArrayList<byte[]>();
scanner.add(col0);
scanner.add(col2);
scanner.add(col4);
scanner.add(col6);
scanner.add(col8);
//Initialize result
List<MatchCode> result = new ArrayList<MatchCode>();
for(int i=0; i<2; i++){
for(byte [] col : scanner){
result.add(wild.checkColumn(col, 0, col.length));
}
wild.update();
}
//Create "Scanner1"
List<byte[]> scanner1 = new ArrayList<byte[]>();
scanner1.add(col1);
scanner1.add(col3);
scanner1.add(col5);
scanner1.add(col7);
scanner1.add(col9);
for(byte [] col : scanner1){
result.add(wild.checkColumn(col, 0, col.length));
}
wild.update();
//Scanner again
for(byte [] col : scanner){
result.add(wild.checkColumn(col, 0, col.length));
}
//"Match"
assertEquals(expected.size(), result.size());
for(int i=0; i<expected.size(); i++){
assertEquals(expected.get(i), result.get(i));
if(PRINT){
System.out.println("Expected " +expected.get(i) + ", actual " +
result.get(i));
}
}
}
// HBASE-1781
public void testStackOverflow(){
int maxVersions = 1;
ColumnTracker wild = new WildcardColumnTracker(maxVersions);
for(int i = 0; i < 100000; i+=2) {
byte [] col = Bytes.toBytes("col"+i);
wild.checkColumn(col, 0, col.length);
}
wild.update();
for(int i = 1; i < 100000; i+=2) {
byte [] col = Bytes.toBytes("col"+i);
wild.checkColumn(col, 0, col.length);
}
}
}