diff --git a/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java index 4e8c391..fe9de26 100644 --- a/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java @@ -77,6 +77,7 @@ public class HColumnDescriptor implements ISerializable, WritableComparable values = new HashMap(); + /** * Default constructor. Must be present for Writable. */ @@ -242,8 +250,39 @@ public class HColumnDescriptor implements ISerializable, WritableComparable: + * @param maxVersions Maximum number of versions to keep + * @param compression Compression type + * @param inMemory If true, column data should be kept in an HRegionServer's + * cache + * @param blockCacheEnabled If true, MapFile blocks should be cached + * @param blocksize + * @param maxValueLength Restrict values to <= this value + * @param timeToLive Time-to-live of cell contents, in seconds + * (use HConstants.FOREVER for unlimited TTL) + * @param bloomFilter Enable the specified bloom filter for this column + * @param multifamily If the description includes multiple families + * + * @throws IllegalArgumentException if passed a family name that is made of + * other than 'word' characters: i.e. [a-zA-Z_0-9] and does not + * end in a : + * @throws IllegalArgumentException if the number of versions is <= 0 + */ + public HColumnDescriptor(final byte [] familyName, final int maxVersions, + final String compression, final boolean inMemory, + final boolean blockCacheEnabled, final int blocksize, + final int maxValueLength, final int timeToLive, final boolean bloomFilter, + final boolean multifamily) { isLegalFamilyName(familyName); this.name = stripColon(familyName); if (maxVersions <= 0) { @@ -260,7 +299,9 @@ public class HColumnDescriptor implements ISerializable, WritableComparable>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/HColumnDescriptor.java * @return Name of this column family with colon as required by client API */ @TOJSON(fieldName = "name", base64=true) @@ -598,6 +668,7 @@ public class HColumnDescriptor implements ISerializable, WritableComparable, I public Collection getFamilies() { return Collections.unmodifiableCollection(this.families.values()); } + + /** + * @return Immutable sorted set of the keys of the families. + */ + public Set getFamiliesKeys() { + return Collections.unmodifiableSet(this.families.keySet()); + } @TOJSON(fieldName = "columns") public HColumnDescriptor[] getColumnFamilies() { diff --git a/src/java/org/apache/hadoop/hbase/KeyValue.java b/src/java/org/apache/hadoop/hbase/KeyValue.java index e5711a8..4c44d7e 100644 --- a/src/java/org/apache/hadoop/hbase/KeyValue.java +++ b/src/java/org/apache/hadoop/hbase/KeyValue.java @@ -19,13 +19,18 @@ */ package org.apache.hadoop.hbase; +import java.io.DataInput; +import java.io.DataOutput; import java.io.IOException; +import org.apache.hadoop.io.RawComparator; +import org.apache.hadoop.io.Writable; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.RawComparator; + /** * An HBase Key/Value. Instances of this class are immutable. They are not @@ -48,7 +53,7 @@ import org.apache.hadoop.io.RawComparator; *

TODO: Group Key-only compartors and operations into a Key class, just * for neatness sake, if can figure what to call it. */ -public class KeyValue { +public class KeyValue implements Writable{ static final Log LOG = LogFactory.getLog(KeyValue.class); /** @@ -62,8 +67,12 @@ public class KeyValue { public static KVComparator COMPARATOR = new KVComparator(); /** +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/KeyValue.java + * Comparator for plain key; i.e. non-catalog table key. +======= * Comparator for plain key; i.e. non-catalog table key. Works on Key portion * of KeyValue only. +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/KeyValue.java */ public static KeyComparator KEY_COMPARATOR = new KeyComparator(); @@ -93,8 +102,11 @@ public class KeyValue { /** * Comparator that compares the family portion of columns only. +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/KeyValue.java +======= * Use this making NavigableMaps of Stores or when you need to compare * column family portion only of two column names. +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/KeyValue.java */ public static final RawComparator FAMILY_COMPARATOR = new RawComparator () { @@ -137,8 +149,13 @@ public class KeyValue { /** * Key type. +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/KeyValue.java + * Has space to allow other key types to be added later. Cannot rely on + * enum ordinals.. they change if item is removed or moved. Do our own codes. +======= * Has space for other key types to be added later. Cannot rely on * enum ordinals . They change if item is removed or moved. Do our own codes. +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/KeyValue.java */ public static enum Type { Put((byte)4), @@ -184,15 +201,30 @@ public class KeyValue { /** * Lowest possible key. +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/KeyValue.java +======= * Makes a Key with highest possible Timestamp, empty row and column. No * key can be equal or lower than this one in memcache or in store file. +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/KeyValue.java */ public static final KeyValue LOWESTKEY = new KeyValue(HConstants.EMPTY_BYTE_ARRAY, HConstants.LATEST_TIMESTAMP); - private final byte [] bytes; - private final int offset; - private final int length; + + private byte [] bytes; + private int offset; + private int length; + + /** + * Creates a KeyValue from another KeyValue + * @param kv + */ + public KeyValue(final KeyValue kv){ + this.bytes = kv.getBuffer(); + this.offset = kv.getOffset(); + this.length = kv.getLength(); + } + /** * Creates a KeyValue from the start of the specified byte array. @@ -466,6 +498,95 @@ public class KeyValue { return bytes; } + + public KeyValue(final byte[] row, final byte[] family, + final byte[] qualifier, final long timestamp, Type type, + final byte[] value) { + this(row, family, qualifier, 0, qualifier==null ? 0 : qualifier.length, + timestamp, type, value, 0, value==null ? 0 : value.length); + } + + public KeyValue(final byte[] row, final byte[] family, + final byte[] qualifier, final int qoffset, int qlength, + final long timestamp, Type type, + final byte[] value, final int voffset, int vlength) { + this.bytes = createByteArray(row, 0, row==null ? 0 : row.length, family, 0, + family==null ? 0 : family.length, qualifier, qoffset, qlength, + Bytes.toBytes(timestamp), type, value, voffset, vlength); + } + + static byte [] createByteArray(final byte [] row, final int roffset, + int rlength, final byte [] family, final int foffset, int flength, + final byte [] qualifier, final int qoffset, int qlength, + final byte [] timestamp, final Type type, final byte [] value, + final int voffset, int vlength) { + + //Checking for null values + if(row == null) { + throw new IllegalArgumentException("Row is null"); + } + if(family == null) { + throw new IllegalArgumentException("family is null"); + } + if(qualifier == null) { + throw new IllegalArgumentException("qualifier is null"); + } + if(value == null) { + throw new IllegalArgumentException("value is null"); + } + + //Setting up and checking lengths + if (rlength > Short.MAX_VALUE) { + throw new IllegalArgumentException("Row > " + Short.MAX_VALUE); + } + + if (flength > Byte.MAX_VALUE) { + throw new IllegalArgumentException("family > " + Byte.MAX_VALUE); + } + + long longkeylength = KEY_INFRASTRUCTURE_SIZE + rlength + flength + qlength; + if (longkeylength > Integer.MAX_VALUE) { + throw new IllegalArgumentException("keylength " + longkeylength + " > " + Integer.MAX_VALUE); + } + int keylength = (int)longkeylength; + + // Allocate right-sized byte array. + byte [] bytes = new byte[KEYVALUE_INFRASTRUCTURE_SIZE + keylength + vlength]; + + // Write key, value and key row length. + int pos = 0; + pos = Bytes.putInt(bytes, pos, keylength); + pos = Bytes.putInt(bytes, pos, vlength); + + pos = Bytes.putShort(bytes, pos, (short)(rlength & 0x0000ffff)); + pos = Bytes.putBytes(bytes, pos, row, roffset, rlength); + + pos = Bytes.putByte(bytes, pos, (byte)(flength & 0x0000ff)); + pos = Bytes.putBytes(bytes, pos, family, foffset, flength); + + pos = Bytes.putBytes(bytes, pos, qualifier, qoffset, qlength); + + pos = Bytes.putBytes(bytes, pos, timestamp, 0, Bytes.SIZEOF_LONG); + + pos = Bytes.putByte(bytes, pos, type.getCode()); + + pos = Bytes.putBytes(bytes, pos, value, voffset, vlength); + return bytes; + } + + + /** + * Sets the current variables to the values in the passed KeyValue + * @param kv + */ + public void set(final KeyValue kv){ + this.bytes = kv.getBuffer(); + this.offset = kv.getOffset(); + this.length = kv.getLength(); + } + + + // Needed doing 'contains' on List. public boolean equals(Object other) { KeyValue kv = (KeyValue)other; @@ -529,7 +650,15 @@ public class KeyValue { } /** +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/KeyValue.java +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/KeyValue.java + * @param b +======= + * @param b Key portion of a KeyValue. +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/KeyValue.java +======= * @param k Key portion of a KeyValue. +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/KeyValue.java * @return Key as a String. */ public static String keyToString(final byte [] k) { @@ -537,7 +666,11 @@ public class KeyValue { } /** +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/KeyValue.java + * @param b +======= * @param b Key portion of a KeyValue. +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/KeyValue.java * @param o Offset to start of key * @param l Length of key. * @return Key as a String. @@ -676,7 +809,7 @@ public class KeyValue { /** * @return Type of this KeyValue. */ - byte getType() { + public byte getType() { return getType(getKeyLength()); } @@ -772,7 +905,10 @@ public class KeyValue { /** * @param column Column minus its delimiter * @return True if column matches. +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/KeyValue.java +======= * @see #matchingColumn(byte[]) +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/KeyValue.java */ public boolean matchingColumnNoDelimiter(final byte [] column) { int o = getColumnOffset(); @@ -1323,52 +1459,118 @@ public class KeyValue { volatile boolean ignoreType = false; public int compare(byte[] left, int loffset, int llength, byte[] right, - int roffset, int rlength) { + int roffset, int rlength){ + int initloffset = loffset; + int initroffset = roffset; + // Compare row short lrowlength = Bytes.toShort(left, loffset); + loffset += Bytes.SIZEOF_SHORT; short rrowlength = Bytes.toShort(right, roffset); - int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT, - lrowlength, - right, roffset + Bytes.SIZEOF_SHORT, rrowlength); + roffset += Bytes.SIZEOF_SHORT; + int compare = compareRows(left, loffset,lrowlength, right, roffset, + rrowlength); if (compare != 0) { return compare; } - - // Compare column family. Start compare past row and family length. - int lcolumnoffset = Bytes.SIZEOF_SHORT + lrowlength + 1 + loffset; - int rcolumnoffset = Bytes.SIZEOF_SHORT + rrowlength + 1 + roffset; - int lcolumnlength = llength - TIMESTAMP_TYPE_SIZE - - (lcolumnoffset - loffset); - int rcolumnlength = rlength - TIMESTAMP_TYPE_SIZE - - (rcolumnoffset - roffset); - compare = Bytes.compareTo(left, lcolumnoffset, lcolumnlength, right, - rcolumnoffset, rcolumnlength); + loffset += lrowlength; + roffset += rrowlength; + + // Compare family. + byte lfamilylength = left[loffset]; + loffset += Bytes.SIZEOF_BYTE; + byte rfamilylength = right[roffset]; + roffset += Bytes.SIZEOF_BYTE; + + compare = Bytes.compareTo(left, loffset, lfamilylength, right, + roffset, rfamilylength); if (compare != 0) { return compare; } - + loffset += lfamilylength; + roffset += rfamilylength; + + // Compare column. + int lcolumnlength = initloffset + llength - TIMESTAMP_TYPE_SIZE - loffset; + int rcolumnlength = initroffset + rlength - TIMESTAMP_TYPE_SIZE - roffset; + compare = Bytes.compareTo(left, loffset, lcolumnlength, right, + roffset, rcolumnlength); + if (compare != 0) { + return compare; + } + loffset += lcolumnlength; + roffset += rcolumnlength; + if (!this.ignoreTimestamp) { - // Get timestamps. - long ltimestamp = Bytes.toLong(left, - loffset + (llength - TIMESTAMP_TYPE_SIZE)); - long rtimestamp = Bytes.toLong(right, - roffset + (rlength - TIMESTAMP_TYPE_SIZE)); - compare = compareTimestamps(ltimestamp, rtimestamp); + //Switching order for timestamp so that new are sorted first + compare = Bytes.compareTo(right, roffset, Bytes.SIZEOF_LONG, left, + loffset, Bytes.SIZEOF_LONG); if (compare != 0) { return compare; } } - + loffset += Bytes.SIZEOF_LONG; + roffset += Bytes.SIZEOF_LONG; + if (!this.ignoreType) { - // Compare types. Let the delete types sort ahead of puts; i.e. types - // of higher numbers sort before those of lesser numbers - byte ltype = left[loffset + (llength - 1)]; - byte rtype = right[roffset + (rlength - 1)]; - return (0xff & rtype) - (0xff & ltype); + // Compare types. Let the delete types sort ahead of puts. + byte ltype = left[loffset]; + byte rtype = right[roffset]; + int ret = (0xff & rtype) - (0xff & ltype); + return ret; } return 0; } - + +// public int compare(byte[] left, int loffset, int llength, byte[] right, +// int roffset, int rlength) { +// // Compare row +// short lrowlength = Bytes.toShort(left, loffset); +// short rrowlength = Bytes.toShort(right, roffset); +// int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT, +// lrowlength, +// right, roffset + Bytes.SIZEOF_SHORT, rrowlength); +// if (compare != 0) { +// return compare; +// } +// +// // Compare column family. Start compare past row and family length. +// int lcolumnoffset = Bytes.SIZEOF_SHORT + lrowlength + 1 + loffset; +// int rcolumnoffset = Bytes.SIZEOF_SHORT + rrowlength + 1 + roffset; +// int lcolumnlength = llength - TIMESTAMP_TYPE_SIZE - +// (lcolumnoffset - loffset); +// int rcolumnlength = rlength - TIMESTAMP_TYPE_SIZE - +// (rcolumnoffset - roffset); +// compare = Bytes.compareTo(left, lcolumnoffset, lcolumnlength, right, +// rcolumnoffset, rcolumnlength); +// if (compare != 0) { +// return compare; +// } +// +// if (!this.ignoreTimestamp) { +// // Get timestamps. +// long ltimestamp = Bytes.toLong(left, +// loffset + (llength - TIMESTAMP_TYPE_SIZE)); +// long rtimestamp = Bytes.toLong(right, +// roffset + (rlength - TIMESTAMP_TYPE_SIZE)); +// compare = compareTimestamps(ltimestamp, rtimestamp); +// if (compare != 0) { +// return compare; +// } +// } +// +// if (!this.ignoreType) { +// // Compare types. Let the delete types sort ahead of puts; i.e. types +// // of higher numbers sort before those of lesser numbers +// byte ltype = left[loffset + (llength - 1)]; +// byte rtype = right[roffset + (rlength - 1)]; +// return (0xff & rtype) - (0xff & ltype); +// } +// return 0; +// } + + + public int compare(byte[] left, byte[] right) { return compare(left, 0, left.length, right, 0, right.length); } @@ -1396,4 +1598,19 @@ public class KeyValue { return 0; } } + + + //Writable + public void readFields(final DataInput in) throws IOException { + this.length = in.readInt(); + this.offset = 0; + this.bytes = new byte[this.length]; + in.readFully(this.bytes, this.offset, this.length); + } + + public void write(final DataOutput out) throws IOException { + out.writeInt(this.length); + out.write(this.bytes, this.offset, this.bytes.length); + } + } \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/client/HConnection.java b/src/java/org/apache/hadoop/hbase/client/HConnection.java index 421afe0..d6d003e 100644 --- a/src/java/org/apache/hadoop/hbase/client/HConnection.java +++ b/src/java/org/apache/hadoop/hbase/client/HConnection.java @@ -21,12 +21,14 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import java.util.ArrayList; +import java.util.List; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.RowUpdates; import org.apache.hadoop.hbase.ipc.HMasterInterface; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper; @@ -181,4 +183,14 @@ public interface HConnection { */ public void processBatchOfRows(ArrayList list, byte[] tableName) throws IOException; + + /** + * Process a batch of rows. Currently it only works for updates until + * HBASE-880 is available. Does the retries. + * @param updatesArray A batch of rows to process + * @param tableName The name of the table + * @throws IOException + */ + public void processListOfRowUpdates( byte[] tableName, List list) + throws IOException; } \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java index 1cc7386..11e9233 100644 --- a/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java +++ b/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java @@ -24,6 +24,7 @@ import java.lang.reflect.UndeclaredThrowableException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.TreeSet; @@ -47,6 +48,7 @@ import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; 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.io.RowUpdates; import org.apache.hadoop.hbase.ipc.HBaseRPC; import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion; import org.apache.hadoop.hbase.ipc.HMasterInterface; @@ -338,8 +340,7 @@ public class HConnectionManager implements HConstants { public boolean processRow(RowResult rowResult) throws IOException { HRegionInfo info = Writables.getHRegionInfo( - rowResult.get(COL_REGIONINFO)); - + rowResult.get(HConstants.COL_REGIONINFO)); // Only examine the rows where the startKey is zero length if (info.getStartKey().length == 0) { uniqueTables.add(info.getTableDesc()); @@ -1040,6 +1041,56 @@ public class HConnectionManager implements HConstants { } } + //Everything should be grouped by row and families when coming in here + public void processListOfRowUpdates(byte[] tableName, List list) + throws IOException { + if (list.isEmpty()) { + return; + } + int tries = 0; + Iterator updatesIterator = list.iterator(); + + byte [] currentRow = null; + while(updatesIterator.hasNext() && tries < numRetries){ + + System.out.println("process 1"); + final RowUpdates currentUpdates = updatesIterator.next(); + currentRow = currentUpdates.getRow(); + int index = getRegionServerWithRetries(new ServerCallable( + this, tableName, currentRow) { + public Integer call() throws IOException { + System.out.println("updating row"); + int i = server.updateRow(location.getRegionInfo() + .getRegionName(), currentUpdates); + return i; + } + }); + if (index != -1) { + HRegionLocation location = getRegionLocationForRowWithRetries( + tableName, currentRow, false); + byte [] currentRegion = location.getRegionInfo().getRegionName(); + if (tries == numRetries - 1) { + throw new RetriesExhaustedException("Some server", + currentRegion, currentRow, tries, new ArrayList()); + } + long sleepTime = getPauseTime(tries); + if (LOG.isDebugEnabled()) { + LOG.debug("Reloading region " + Bytes.toString(currentRegion) + + " location because regionserver didn't accept updates; " + + "tries=" + tries + + " of max=" + this.numRetries + ", waiting=" + sleepTime + + "ms"); + } + try { + Thread.sleep(sleepTime); + tries++; + } catch (InterruptedException e) { + // continue + } + } + } + } + void close(boolean stopProxy) { if (master != null) { if (stopProxy) { diff --git a/src/java/org/apache/hadoop/hbase/client/HTable.java b/src/java/org/apache/hadoop/hbase/client/HTable.java index ff20dcb..a58e70e 100644 --- a/src/java/org/apache/hadoop/hbase/client/HTable.java +++ b/src/java/org/apache/hadoop/hbase/client/HTable.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; import org.apache.hadoop.hbase.filter.RowFilterInterface; @@ -44,8 +45,10 @@ import org.apache.hadoop.hbase.filter.WhileMatchRowFilter; import org.apache.hadoop.hbase.io.BatchOperation; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.Get; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.HbaseMapWritable; +import org.apache.hadoop.hbase.io.RowUpdates; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; @@ -58,9 +61,12 @@ public class HTable { protected final int scannerTimeout; private volatile HBaseConfiguration configuration; private ArrayList writeBuffer; + private ArrayList newWriteBuffer; private long writeBufferSize; private boolean autoFlush; private long currentWriteBufferSize; + private long newCurrentWriteBufferSize; + protected int scannerCaching; /** @@ -113,10 +119,12 @@ public class HTable { this.configuration = conf; this.connection.locateRegion(tableName, HConstants.EMPTY_START_ROW); this.writeBuffer = new ArrayList(); + this.newWriteBuffer = new ArrayList(); this.writeBufferSize = this.configuration.getLong("hbase.client.write.buffer", 2097152); this.autoFlush = true; this.currentWriteBufferSize = 0; + this.newCurrentWriteBufferSize = 0; this.scannerCaching = conf.getInt("hbase.client.scanner.caching", 1); } @@ -281,6 +289,20 @@ public class HTable { return regionMap; } + public KeyValue [] get(final Get get) + throws IOException { + System.out.println("IN HT.get, row " +Bytes.toInt(get.getRow())); + return connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, get.getRow()) { + public KeyValue [] call() throws IOException { + System.out.println("IN HT.get.ServerCallable,"); + KeyValue [] result = server.newGet( + location.getRegionInfo().getRegionName(), get, -1L); + return (result == null)? null : result; + } + } + ); + } /** * Get a single value for the specified row and column * @@ -603,7 +625,7 @@ public class HTable { */ public RowResult getRow(final byte [] row, final byte [][] columns, final long ts, final int numVersions, final RowLock rl) - throws IOException { + throws IOException { return connection.getRegionServerWithRetries( new ServerCallable(connection, tableName, row) { public RowResult call() throws IOException { @@ -1336,6 +1358,56 @@ public class HTable { } /** + * Commit a BatchUpdate to the table. + * If autoFlush is false, the update is buffered + * @param batchUpdate + * @throws IOException + */ + public synchronized void newCommit(final RowUpdates updates) + throws IOException { + newCommit(updates, null); + } + + /** + * Commit a BatchUpdate to the table using existing row lock. + * If autoFlush is false, the update is buffered + * @param batchUpdate + * @param rl Existing row lock + * @throws IOException + */ + public synchronized void newCommit(final RowUpdates updates, final RowLock rl) + throws IOException { + if(rl != null) { + updates.setRowLock(rl.getLockId()); + } + updates.createKeyValuesFromColumns(); + newWriteBuffer.add(updates); + newCurrentWriteBufferSize += updates.heapSize(); + if (autoFlush || newCurrentWriteBufferSize > writeBufferSize) { + newFlushCommits(); + } + } + + /** + * Commit a List of BatchUpdate to the table. + * If autoFlush is false, the updates are buffered + * @param batchUpdates + * @throws IOException + */ + public synchronized void newCommit(final List updatesList) + throws IOException { + for (RowUpdates updates : updatesList) { + updates.createKeyValuesFromColumns(); + newWriteBuffer.add(updates); + newCurrentWriteBufferSize += updates.heapSize(); + } + if (autoFlush || newCurrentWriteBufferSize > writeBufferSize) { + newFlushCommits(); + } + } + + + /** * Atomically checks if a row's values match * the expectedValues. If it does, it uses the * batchUpdate to update the row. @@ -1375,7 +1447,24 @@ public class HTable { writeBuffer.clear(); } } - + + /** + * Commit to the table the buffer of BatchUpdate. + * Called automaticaly in the commit methods when autoFlush is true. + * @throws IOException + */ + public void newFlushCommits() throws IOException { + if(newWriteBuffer.isEmpty()){ + return; + } + try { + connection.processListOfRowUpdates(tableName, newWriteBuffer); + } finally { + newCurrentWriteBufferSize = 0; + newWriteBuffer.clear(); + } + } + /** * Release held resources * @@ -1386,6 +1475,18 @@ public class HTable { } /** +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/client/HTable.java + * Release held resources + * + * @throws IOException + */ + public void newClose() throws IOException{ + newFlushCommits(); + } + + /** +======= +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/client/HTable.java * Utility method that checks rows existence, length and columns well * formedness. * diff --git a/src/java/org/apache/hadoop/hbase/client/ServerCallable.java b/src/java/org/apache/hadoop/hbase/client/ServerCallable.java index a26a96a..840ae4d 100644 --- a/src/java/org/apache/hadoop/hbase/client/ServerCallable.java +++ b/src/java/org/apache/hadoop/hbase/client/ServerCallable.java @@ -25,6 +25,7 @@ import java.util.concurrent.Callable; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.util.Bytes; /** * Abstract class that implements Callable, used by retryable actions. diff --git a/src/java/org/apache/hadoop/hbase/io/AbstractGet.java b/src/java/org/apache/hadoop/hbase/io/AbstractGet.java new file mode 100644 index 0000000..68cdd2b --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/io/AbstractGet.java @@ -0,0 +1,106 @@ +package org.apache.hadoop.hbase.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.util.Bytes; + +public abstract class AbstractGet implements Get { + byte [] row = new byte [0]; +// Family [] families = new Family [0]; + List families = new ArrayList(); + //if this is changed to support more versions than this it also needs to be + //changed in the family class and in the updates of the versions in the + //serverGets + short versions = 0; + TimeRange tr = new TimeRange(); + RowFilterInterface filter = null; + + + @Override + public List getFamilies(){ + return this.families; + } + @Override + public void setFamilies(List families){ + this.families = families; + } + + @Override + public RowFilterInterface getFilter(){ + return filter; + } + @Override + public void setFilter(RowFilterInterface filter){ + this.filter = filter; + } + + @Override + public byte [] getRow(){ + return this.row; + } + + @Override + public TimeRange getTimeRange(){ + return tr; + } + + @Override + public short getVersions(){ + return this.versions; + } + @Override + public void setVersions(short versions){ + this.versions = versions; + } + + @Override + public String toString(){ + StringBuffer sb = new StringBuffer(); + + sb.append("Row "); + sb.append(new String(this.row)); + sb.append(", families ["); + int i = 0; + for(; i(famLen); + for(int i=0; i columns = new ArrayList(); + private List columns = new ArrayList(); +// protected final byte[] ZERO_BYTES = new byte[]{(byte)0}; + //This list is only used for temporary storage before the puts are converted + //into a KeyValue when sending the update + private List values = null; + + public Family(){} + public Family(byte[] family){ + this.family = family; + } + public Family(byte[] family, byte[] column) { + this.family = family; + this.columns.add(column); + } + public Family(byte[] family, List columns) { + this.family = family; + this.columns = columns; + } + + //Puts + public Family(byte[] family, byte[] column, byte[] value) { + this.family = family; + this.columns.add(column); + this.values = new ArrayList(); + this.values.add(value); + } + public Family(byte[] family, List columns, List values) { + this.family = family; + this.columns = columns; + this.values = values; + } + + + public void add(byte[] column){ + this.columns.add(column); + } + public void add(byte[] column, byte[] value){ + this.columns.add(column); + this.values.add(value); + } + + /** + * + * @return the family + */ + public byte [] getFamily() { + return family; + } + /** + * Sets the family name of this Family + * @param family + */ + public void setFamily(byte [] family){ + this.family = family; + } + + public List getColumns() { + return columns; + } + + + //TODO have to check how those sorts + public void sortColumns(){ + byte[][] cols = new byte[columns.size()][]; + cols = columns.toArray(new byte[0][]); + Arrays.sort(cols, Bytes.BYTES_COMPARATOR); + columns.clear(); + for(byte[] bytes : cols){ + columns.add(bytes); + } + } + + public void createKeyValuesFromColumns(byte[] row, final long ts){ + List updatedColumns = new ArrayList(columns.size()); + for(int i=0; i(nColumns); + for(int i=0; i { + public FamilyComparator() { + super(); + } + @Override + public int compare(byte [] left, byte [] right) { + return compareTo(left, right); + } + + private int compareTo(byte[] left, byte[] right){ + int leftLen = Bytes.toInt(left, 0, Bytes.SIZEOF_INT); + int rightLen = Bytes.toInt(right, 0, Bytes.SIZEOF_INT); + return Bytes.compareTo(left, Bytes.SIZEOF_INT, leftLen, + right, Bytes.SIZEOF_INT, rightLen); + } + } + +} diff --git a/src/java/org/apache/hadoop/hbase/io/Get.java b/src/java/org/apache/hadoop/hbase/io/Get.java new file mode 100644 index 0000000..bbd1cfc --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/io/Get.java @@ -0,0 +1,35 @@ +package org.apache.hadoop.hbase.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.io.Writable; + +import org.apache.hadoop.hbase.filter.RowFilterInterface;; + +/** + * Interface for all the different get calls + * + */ +public interface Get extends Writable{ + + public List getFamilies(); + public void setFamilies(List families); + + public RowFilterInterface getFilter(); + public void setFilter(RowFilterInterface filter); + + public byte [] getRow(); + + public TimeRange getTimeRange(); + + public short getVersions(); + public void setVersions(short versions); + + //Writable + public void readFields(final DataInput in) throws IOException; + public void write(final DataOutput out) throws IOException; + +} diff --git a/src/java/org/apache/hadoop/hbase/io/GetColumns.java b/src/java/org/apache/hadoop/hbase/io/GetColumns.java new file mode 100644 index 0000000..45443e7 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/io/GetColumns.java @@ -0,0 +1,59 @@ +package org.apache.hadoop.hbase.io; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hbase.util.Writables; + +public class GetColumns extends AbstractGet{ + + public GetColumns(byte[] row, byte[] family, byte[] column, short versions){ + this(row, family, column, versions, new TimeRange()); + } + + public GetColumns(byte[] row, byte[] family, byte[] column, short versions, + TimeRange tr){ + super.row = row; + super.versions = versions; + super.families.add(new Family(family, column)); + if (tr == null){ + tr = new TimeRange(); + } + super.tr = tr; + } + + public GetColumns(byte [] row, byte [] family, List columns, + short versions, TimeRange tr){ + super.row = row; + super.versions = versions; + super.families.add(new Family(family, columns)); + if (tr == null){ + tr = new TimeRange(); + } + super.tr = tr; + } + + + public GetColumns(byte [] row, List families, short versions, + TimeRange tr){ + super.row = row; + super.versions = versions; + super.families = families; + super.tr = tr; + } + + public GetColumns(byte [] row, List families, short versions) + throws IOException{ + this(row, families, versions, System.currentTimeMillis()); + } + + public GetColumns(byte [] row, List families, short versions, + long ts) + throws IOException{ + super.row = row; + super.versions = versions; + super.families = families; + super.tr = new TimeRange(ts, ts); + } + +} diff --git a/src/java/org/apache/hadoop/hbase/io/GetFamilies.java b/src/java/org/apache/hadoop/hbase/io/GetFamilies.java new file mode 100644 index 0000000..beabf4c --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/io/GetFamilies.java @@ -0,0 +1,17 @@ +package org.apache.hadoop.hbase.io; + +public class GetFamilies extends AbstractGet{ + public GetFamilies(byte [] row, byte [] family, short versions){ + this(row, family, versions, new TimeRange()); + } + + public GetFamilies(byte [] row, byte [] family, short versions, TimeRange tr){ + super.row = row; + super.versions = versions; + super.families.add(new Family(family)); + if(tr == null){ + super.tr = new TimeRange(); + } + } + +} diff --git a/src/java/org/apache/hadoop/hbase/io/GetRow.java b/src/java/org/apache/hadoop/hbase/io/GetRow.java new file mode 100644 index 0000000..defa83c --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/io/GetRow.java @@ -0,0 +1,34 @@ +package org.apache.hadoop.hbase.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Iterator; +import java.util.Set; + +import org.apache.hadoop.hbase.util.Bytes; + +public class GetRow extends AbstractGet { // implements Get { + + public GetRow(byte [] row, short versions){ + super.row = row; + super.versions = versions; + super.families.add(new Family()); + super.tr = new TimeRange(); + } + + public GetRow(byte [] row, short versions, TimeRange tr){ + super.row = row; + super.versions = versions; + super.families.add(new Family()); + super.tr = tr; + } + + public void setFamilies(Set families){ + int famSize = families.size(); + for(byte[] family : families){ + super.families.add(new Family(family)); + } + } + +} diff --git a/src/java/org/apache/hadoop/hbase/io/GetTop.java b/src/java/org/apache/hadoop/hbase/io/GetTop.java new file mode 100644 index 0000000..db15a98 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/io/GetTop.java @@ -0,0 +1,45 @@ +package org.apache.hadoop.hbase.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.hbase.util.Bytes; + +public class GetTop extends AbstractGet { + private int fetches = 0; + + public GetTop(byte [] row, byte [] family, int fetches){ + this(row, family, fetches, new TimeRange()); + } + public GetTop(byte [] row, byte [] family, int fetches, TimeRange tr){ + super.row = row; + this.fetches = fetches; + super.families.add(new Family(family)); + super.tr = tr; + } + + public int getFetches(){ + return fetches; + } + public void setFetches(int fetches){ + this.fetches = fetches; + } + public int incFetchesAndReturn(){ + return ++fetches; + } + + //Writable + @Override + public void readFields(final DataInput in) throws IOException { + super.readFields(in); + this.fetches = in.readInt(); + } + + @Override + public void write(final DataOutput out) throws IOException { + super.write(out); + out.writeInt(this.fetches); + } + +} diff --git a/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java b/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java index c2a3d4a..12b7a7a 100644 --- a/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java +++ b/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java @@ -24,6 +24,8 @@ import java.io.IOException; import java.lang.reflect.Array; import java.util.HashMap; import java.util.Map; +import java.util.ArrayList; +import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -36,6 +38,7 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.filter.RowFilterSet; import org.apache.hadoop.hbase.io.HbaseMapWritable; @@ -133,6 +136,20 @@ public class HbaseObjectWritable implements Writable, Configurable { e.printStackTrace(); } addToMap(BatchUpdate[].class, code++); + + //New from 0.20 + addToMap(Family.class, code++); + addToMap(Get.class, code++); + addToMap(KeyValue.class, code++); + addToMap(RowUpdates.class, code++); + addToMap(TimeRange.class, code++); + +// addToMap(AbstractGet.class, code++); +// addToMap(GetColumns.class, code++); +// addToMap(KeyValue[].class, code++); +// addToMap(ArrayList.class, code++); +// addToMap(List.class, code++); +// addToMap(Family[].class, code++); } private Class declaredClass; diff --git a/src/java/org/apache/hadoop/hbase/io/RowUpdates.java b/src/java/org/apache/hadoop/hbase/io/RowUpdates.java new file mode 100644 index 0000000..1683aaf --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/io/RowUpdates.java @@ -0,0 +1,133 @@ +package org.apache.hadoop.hbase.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.apache.hadoop.io.Writable; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + + +//TODO should be able to insert a list of KeyValue and get them sorted and +//inserted into the familyMap before sending to the server, should be done in +//commit +public class RowUpdates implements Writable, HeapSize{ + byte [] row = null; +// SortedMap> familyMap = null; + List families = new ArrayList(); + + //TODO check if you want to get the timestamp of the send and not the time + //when the Object is created + private long ts = HConstants.LATEST_TIMESTAMP; + private long rowLock = -1L; + + public RowUpdates(byte[] row){ + this.row = row; + } + + public RowUpdates(byte [] row, byte [] family, byte[] column){ + this.row = row; + this.families.add(new Family(family, column)); + } + + public RowUpdates(byte [] row, byte [] family, List columns){ + this.row = row; + this.families.add(new Family(family, columns)); + } + + public RowUpdates(byte [] row, byte [] family, List columns, long ts){ + this.row = row; + this.families.add(new Family(family, columns)); + this.ts = ts; + } + + public RowUpdates(byte [] row, Family Family, long ts){ + this.row = row; + this.families.add(Family); + this.ts = ts; + } + + public RowUpdates(byte [] row, List putFamilies, long ts){ + this.row = row; + this.families = putFamilies; + this.ts = ts; + } + + public void add(Family Family){ + this.families.add(Family); + } + public void add(List putFamilies){ + this.families.addAll(putFamilies); + } + + public List getFamilies(){ + return families; + } + + public byte [] getRow(){ + return row; + } + + /** + * Get the row lock associated with this update + * @return the row lock + */ + public long getRowLock() { + return rowLock; + } + + /** + * Set the lock to be used for this update + * @param rowLock the row lock + */ + public void setRowLock(long rowLock) { + this.rowLock = rowLock; + } + + //TODO fix heapSize + public long heapSize(){ + //Should be 16 + 24+row.length/8 + familyMap.heapSize + 8. + //familyMap.heapSize + return 0; + } + + public void createKeyValuesFromColumns(){ + for(Family family : families){ + family.createKeyValuesFromColumns(row, ts); + } + } + + + //Writable + public void readFields(final DataInput in) + throws IOException { + this.row = Bytes.readByteArray(in); + int nFamilies = in.readInt(); + this.families = new ArrayList(nFamilies); + for(int i=0; i ts >= tsMin + */ + public int withinTimeRange(byte [] bytes, int offset){ + int ret = Bytes.compareTo(tsMin, 0, Bytes.SIZEOF_LONG, bytes, + offset, Bytes.SIZEOF_LONG); + if(ret == 0){ + return 1; + } else if(ret >= 1){ + return 0; + } + + //check if ts < tsMax + ret = Bytes.compareTo(tsMax, 0, Bytes.SIZEOF_LONG, bytes, + offset, Bytes.SIZEOF_LONG); + if(ret <= 0){ + return 0; + } + return 1; + } + + public int withinTimeRange(long ts){ +// int ret = Bytes.compareTo(tsMin, 0, Bytes.SIZEOF_LONG, bytes, +// offset, Bytes.SIZEOF_LONG); + long tsmin = Bytes.toLong(tsMin); + long ret = tsmin - ts; + if(ret == 0){ + return 1; + } else if(ret >= 1){ + return 0; + } + + //check if ts < tsMax + long tsmax = Bytes.toLong(tsMax); + ret = tsmax - ts; + if(ret <= 0){ + return 0; + } + return 1; + } + + @Override + public String toString(){ + StringBuffer sb = new StringBuffer(); + sb.append("tsMax "); + sb.append(Bytes.toLong(tsMax)); + sb.append(", tsMin"); + sb.append(Bytes.toLong(tsMin)); + return sb.toString(); + } + +//Writable + public void readFields(final DataInput in) throws IOException { + this.tsMin = Bytes.readByteArray(in); + this.tsMax = Bytes.readByteArray(in); + } + + public void write(final DataOutput out) throws IOException { + Bytes.writeByteArray(out, this.tsMin); + Bytes.writeByteArray(out, this.tsMax); + } + +} diff --git a/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java b/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java index f26dee8..59b6a62 100644 --- a/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java +++ b/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java @@ -20,14 +20,18 @@ package org.apache.hadoop.hbase.ipc; import java.io.IOException; +import java.util.List; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.Get; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.HbaseMapWritable; +import org.apache.hadoop.hbase.io.RowUpdates; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.NotServingRegionException; /** @@ -222,6 +226,24 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion { public boolean exists(byte [] regionName, byte [] row, byte [] column, long timestamp, long lockID) throws IOException; + +// public List newGet(final byte [] regionName, Get get, +// final long lockId) +// throws IOException; + public KeyValue [] newGet(final byte [] regionName, Get get, + final long lockId) + throws IOException; + + /** + * Applies a batch of updates via one RPC for many rows + * + * @param regionName name of the region to update + * @param b BatchUpdate[] + * @throws IOException + * @return number of updates applied + */ + public int updateRow(final byte[] regionName, final RowUpdates rups) + throws IOException; // // remote scanner interface diff --git a/src/java/org/apache/hadoop/hbase/regionserver/AbstractServerGet.java b/src/java/org/apache/hadoop/hbase/regionserver/AbstractServerGet.java new file mode 100644 index 0000000..2e85b54 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/AbstractServerGet.java @@ -0,0 +1,745 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.hbase.filter.RowFilterInterface; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.Get; +import org.apache.hadoop.hbase.io.TimeRange; +import org.apache.hadoop.hbase.util.Bytes; + +public abstract class AbstractServerGet implements ServerGet{ + + protected static final int NEXT_KV = 0; + protected static final int ADD = 1; + protected static final int NEXT_SF = 2; + protected static final int DONE = 3; + + +// byte[] row = null; +// Family family = null; +// TimeRange tr = null; + protected Get get = null; + private byte [] family = null; + + protected static final int KEY_OFFSET = 2*Bytes.SIZEOF_INT; + private static final int KEY_SIZES = Bytes.SIZEOF_SHORT + Bytes.SIZEOF_BYTE + + Bytes.SIZEOF_LONG + Bytes.SIZEOF_BYTE; + + //This is the list of columnNames + one more byte for the count. + //For GetFamilies and GetTop values are copied into this list. My tests show + //that copying an array is faster than creating a new object, for example + //a Tuple. + + //Created a new class, SingleLinkedList to compare with the LinkedList and the + //ArrayList and it turns out the the ArrayList is faster for puts and more + //memory efficient than the other 2. Remove is slower since the whole array + //needs to be copied, but since this is only happening in one place, + //ServerGetColumns.updateVersions and probably is not going to happen to often + //leaving it as it is. + //Another interesting thing is when using Arrays.asList it casts your list to + //an AbstractList so you can't use remove and it is also like 10x slower than + //doing it yourself in a for loop. + //So using ArrayLists for everything for now. +// protected List columns = null; +// protected Iterator columnIterator = null; +// protected byte [] column = null; + protected List columns = null; +// protected Iterator columnIterator = null; + protected int columnPos = 0; + protected KeyValue column = null; + protected List versions = null; + + protected List newColumns = null; + protected List newVersions = null; + protected KeyValue newColumn = null; + protected boolean endColumns = false; + + //Same thing as above goes for this + protected List deletes = new ArrayList(); + private Iterator deleteIterator = null; + private KeyValue delete = null; + private byte [] deleteFamilyBytes = null; + protected List newDeletes = new ArrayList(); + + RowFilterInterface filter = null; + + long ttl = 0L; + long now = 0L; + + public AbstractServerGet(Get get){ + this.get = get; +// this.columns = new ArrayList(); + this.columns = new ArrayList(); + this.deletes = new ArrayList(); + this.versions = new ArrayList(); + } + + public void clear(){ + this.column = null; + this.columnPos = 0; + this.delete = null; + this.deleteIterator = null; + } + + public byte [] getFamily(){ + return family; + } + public void setFamily(byte [] family){ + this.family = family; + } + + + public byte [] getRow(){ + return get.getRow(); + } + + public TimeRange getTimeRange(){ + return get.getTimeRange(); + } + + + + public boolean isEmpty(){ + return (columns.size() == 0); + } + + @Override + public List getColumns(){ + return columns; + } +// @Override +// public void setColumns(List columns){ +// this.columns = columns; +// } +// @Override +// public void setColumnsFromBytes(List columns){ +// this.columns.clear(); +// for(byte[] column : columns){ +// this.columns.add(new KeyValue(column, 0, column.length)); +// } +// } + @Override + public void setColumns(List columns){ + this.columns.clear(); + for(byte[] column : columns){ + this.columns.add(new KeyValue(column, 0, column.length)); + } + } + @Override + public List getVersions(){ + return this.versions; + } + +// @Override +// public void setColumns(byte [][] columns){ +// for(int i=0; i getDeletes(){ + return deletes; + } + + @Override + public void setDeletes(List deletes) { + this.deletes = deletes; + } + + @Override + public List getNewDeletes(){ + return newDeletes; + } + + @Override + public void setFilter(RowFilterInterface filter){ + this.filter = filter; + } + + @Override + public long getNow(){ + return now; + } + + @Override + public void setNow(){ + this.now = System.currentTimeMillis(); + } + @Override + public void setNow(long now){ + this.now = now; + } + + @Override + public long getTTL(){ + return ttl; + } + @Override + public void setTTL(long ttl){ + this.ttl = ttl; + } + + @Override + public int compareTo(KeyValue kv) + throws IOException{ + return compareTo(kv, false); + } + + + public int getMaxVersions(){ + return get.getVersions(); + } + + protected int checkTTL(byte [] bytes, int offset){ + //Check if KeyValue is still alive + if((this.now - this.ttl) > Bytes.toLong(bytes, offset)){ + return 1; + } + return 0; + } + protected int checkTTL(long ts){ + //Check if KeyValue is still alive + if((this.now - this.ttl) > ts){ + return 1; + } + return 0; + } + + protected boolean isDelete(byte [] bytes, int initialOffset, int keyLen){ + byte type = bytes[initialOffset + KEY_OFFSET + keyLen -1]; + return (type != KeyValue.Type.Put.getCode()); + } + + /** + * This method are very different than the other compares, because it will + * loop through the columns in the column list until the current kv is found + * or the column is smaller than the kv. + * + * The structure of the column is byte[int columnSize, byte[] column, + * byte versions fetched] + */ +// protected int compareColumn(byte [] bytes, int offset, int length){ +// if(columnIterator == null){ +// columnIterator = columns.iterator(); +// if(columnIterator.hasNext()){ +// column = columnIterator.next(); +// } +// } +// +// int res = 0; +// while(true){ +// res = Bytes.compareTo(column.getBuffer(), column.getOffset(), +// column.getLength(), bytes, offset, length); +// if(res >= 0){ +// return res; +// } +// if(columnIterator.hasNext()){ +// column = columnIterator.next(); +// } else { +// return res; +// } +// } +// } + + + + + + + + + + protected int isDeleted(byte [] currBytes, int initCurrOffset, + short currRowLen, byte currFamLen, int currColLen, boolean multiFamily){ + + //Check if this is the first time isDeleted is called + if(deleteIterator == null){ + deleteIterator = deletes.iterator(); + delete = deleteIterator.next(); + } else if(!deleteIterator.hasNext()){ + return 0; + } + + int ret = 0; + int currOffset = 0; + + byte [] delBytes = null; + int delOffset = 0; + int delKeyLen = 0; + short delRowLen = 0; + int delColLen = 0; + int tsLen = Bytes.SIZEOF_LONG; + + while(true){ + //Clearing the last offset + currOffset = initCurrOffset; + + //Setting up delete KeyValue + delBytes = delete.getBuffer(); + + //Getting key length + delKeyLen = Bytes.toInt(delBytes, delOffset); + delOffset = Bytes.SIZEOF_INT; + + //Skipping value length + delOffset += Bytes.SIZEOF_INT; + + //Getting row length + delRowLen = Bytes.toShort(delBytes, delOffset); + delOffset += Bytes.SIZEOF_SHORT; + + //Skipping the key and value lengths + row length + currOffset += 2*Bytes.SIZEOF_INT + Bytes.SIZEOF_SHORT; + + //Skipping compare row since it already has been checked + currOffset += currRowLen; + delOffset += delRowLen; + + //Getting family length + byte delFamLen = delBytes[delOffset]; + delOffset += Bytes.SIZEOF_BYTE; + currOffset += Bytes.SIZEOF_BYTE; + + //CompareFamily + if(multiFamily){ + ret = Bytes.compareTo(currBytes, currOffset, currFamLen, delBytes, + delOffset, delFamLen); + if(ret <= -1){ + return -1; + } else if(ret >= 1){ + if(deleteIterator.hasNext()){ + delete.set(deleteIterator.next()); + continue; + } + return 0; + } + } + currOffset += currFamLen; + delOffset += delFamLen; + + //CompareColumn + delColLen = delKeyLen - Bytes.SIZEOF_SHORT - delRowLen - + Bytes.SIZEOF_BYTE - delFamLen - Bytes.SIZEOF_LONG - Bytes.SIZEOF_BYTE; + ret = Bytes.compareTo(currBytes, currOffset, currColLen, delBytes, + delOffset, delColLen); + if(ret <= -1){ + return -1; + } else if(ret >= 1){ + if(deleteIterator.hasNext()){ + delete.set(deleteIterator.next()); + continue; + } + return 0; + } + currOffset += currColLen; + delOffset += delColLen; + + //Compare DeleteFamily + if(deleteFamilyBytes != null){ + ret = Bytes.compareTo(currBytes, currOffset, tsLen, deleteFamilyBytes, + 0, tsLen); + if(ret >= 1){ + return 1; + } + } + + + //Compare ts and type + //compareTs + ret = Bytes.compareTo(currBytes, currOffset, tsLen, delBytes, delOffset, + tsLen); + if(ret == 0){ + return -1; + } else if(ret >= 1) { + return 0; + } else { + //Getting type + delOffset += Bytes.SIZEOF_LONG; + byte type = delBytes[delOffset]; + if(type == KeyValue.Type.DeleteColumn.getCode()){ + return -1; + } + + if(deleteIterator.hasNext()){ + delete.set(deleteIterator.next()); + continue; + } + return 0; + } + } + } + + + @Override + public Deletes mergeDeletes(List l1, List l2){ + return mergeDeletes(l1, l2, false); + } + + + //This method should not be here + @Override + public Deletes mergeDeletes(List l1, List l2, + boolean multiFamily){ + //TODO Add check for deleteFamily + long deleteFamily = 0L; + +// List mergedDeletes = new LinkedList(); + List mergedDeletes = new ArrayList(); + + if(l1.isEmpty()){ + if(l2.isEmpty()){ + return null; + } + } else if(l2.isEmpty()){ + return new Deletes(l1, 0); + } + + Iterator l1Iter = l1.iterator(); + KeyValue k1 = null; + Iterator l2Iter = l2.iterator(); + KeyValue k2 = l2Iter.next(); + + //Check for deleteFamily, only need to do on the second argument since the + //first one is already checked + byte [] k2Bytes = k2.getBuffer(); + int k2Offset = k2.getOffset(); + int k2KeyLen = Bytes.toInt(k2Bytes, k2Offset); + int k2TypeOffset = k2Offset + KEY_OFFSET + k2KeyLen - Bytes.SIZEOF_BYTE; + if(k2Bytes[k2TypeOffset] == KeyValue.Type.DeleteFamily.getCode()){ + deleteFamily = Bytes.toLong(k2Bytes, k2TypeOffset - Bytes.SIZEOF_LONG); + if(l2Iter.hasNext()){ + k2 = l2Iter.next(); + } else { + if(l1.isEmpty()){ + return new Deletes(l2, deleteFamily); + } + return new Deletes(l1, deleteFamily); + } + } + + int ret = 0; +// boolean it2 = false; + int i2break = 0; + while(l1Iter.hasNext()){ + k1 = l1Iter.next(); + while(true){ +// ret = compareDeleteKeys(k1, k2, multiFamily); + ret = compare(k1, k2, multiFamily); + if(ret == -3){ + if(l2Iter.hasNext()){ + k2 = l2Iter.next(); + continue; + } + i2break = 2; + break; + } else if(ret == -2){ + mergedDeletes.add(k1); + if(l2Iter.hasNext()){ + k2 = l2Iter.next(); + } else { + i2break = 1; + } + break; + } else if(ret == -1){ + mergedDeletes.add(k1); + break; + } else if(ret == 0){ + mergedDeletes.add(k1); + if(l2Iter.hasNext()){ + k2 = l2Iter.next(); + } else { + i2break = 1; + } + break; + } else if(ret == 1){ + mergedDeletes.add(k2); + if(l2Iter.hasNext()){ + k2 = l2Iter.next(); + continue; + } + i2break = 1; + break; + } else if(ret == 2){ + mergedDeletes.add(k2); + if(l2Iter.hasNext()){ + k2 = l2Iter.next(); + } else { + i2break = 1; + } + break; + } else { + break; + } + } + if(i2break != 0){ + break; + } + } + + + + + if(i2break != 0){ + if(i2break == 2){ + mergedDeletes.add(k1); + } + while(l1Iter.hasNext()){ + mergedDeletes.add(l1Iter.next()); + } +// } + + +// if(it2){ +// mergedDeletes.add(k1); +// while(l1Iter.hasNext()){ +// mergedDeletes.add(l1Iter.next()); +// } + } else { + mergedDeletes.add(k2); + while(l2Iter.hasNext()) { + mergedDeletes.add(l2Iter.next()); + } + } + + return new Deletes(mergedDeletes, deleteFamily); + } + + //Same as compareDelete, should fix that later + /** + * @param k1 the first Key to compare + * @param k2 the second Key to compare + * + * @return -2 if k1 was a deleteColumn, -1 if k1 should be kept, 0 if same, + * 1 if k2 should be kept and 2 if k2 was a deleteColumn + */ +// protected int compareDeleteKeys(Key k1, Key k2, boolean multiFamily){ + protected int compare(KeyValue k1, KeyValue k2, boolean multiFamily){ + + byte [] k1Bytes = k1.getBuffer(); + int k1Offset = k1.getOffset(); + + byte [] k2Bytes = k2.getBuffer(); + int k2Offset = k2.getOffset(); + + //Getting key lengths + int k1KeyLen = Bytes.toInt(k1Bytes, k1Offset); + k1Offset += Bytes.SIZEOF_INT; + int k2KeyLen = Bytes.toInt(k2Bytes, k2Offset); + k2Offset += Bytes.SIZEOF_INT; + + //Skipping value lengths + k1Offset += Bytes.SIZEOF_INT; + k2Offset += Bytes.SIZEOF_INT; + + //Getting row lengths + short k1RowLen = Bytes.toShort(k1Bytes, k1Offset); + k1Offset += Bytes.SIZEOF_SHORT; + short k2RowLen = Bytes.toShort(k2Bytes, k2Offset); + k2Offset += Bytes.SIZEOF_SHORT; + + //Skipping reading row, since it has already been checked + k1Offset += k1RowLen; + k2Offset += k2RowLen; + + int ret = 0; + //Getting family lengths + byte k1FamLen = k1Bytes[k1Offset]; + k1Offset += Bytes.SIZEOF_BYTE; + byte k2FamLen = k2Bytes[k2Offset]; + k2Offset += Bytes.SIZEOF_BYTE; + + //Compare families + if(multiFamily){ + ret = Bytes.compareTo(k1Bytes, k1Offset, k1FamLen, k2Bytes, k2Offset, + k2FamLen); + if(ret <= -1){ + return -1; + } else if(ret >=1){ + return 1; + } + } + k1Offset += k1FamLen; + k2Offset += k2FamLen; + + //Get column lengths + int k1ColLen = k1KeyLen - k1RowLen - k1FamLen - KEY_SIZES; + int k2ColLen = k2KeyLen - k2RowLen - k2FamLen - KEY_SIZES; + + //Compare columns + ret = Bytes.compareTo(k1Bytes, k1Offset, k1ColLen, k2Bytes, k2Offset, + k2ColLen); + if(ret <= -1){ + return -1; + } else if(ret >= 1){ + return 1; + } + k1Offset += k1ColLen; + k2Offset += k2ColLen; + + //compare types + byte k1type = k1Bytes[k1Offset + Bytes.SIZEOF_LONG]; + byte k2type = k2Bytes[k2Offset + Bytes.SIZEOF_LONG]; + ret = (0xff & k1type) - (0xff & k2type); + + if(ret == 0){ + ret = Bytes.compareTo(k2Bytes, k2Offset, Bytes.SIZEOF_LONG, k1Bytes, + k1Offset, Bytes.SIZEOF_LONG); + if(ret >= 1){ + ret = 2; + } else if(ret <= -1){ + ret = -2; + } + return ret; + } else if(ret <= -1){ + ret = Bytes.compareTo(k2Bytes, k2Offset, Bytes.SIZEOF_LONG, k1Bytes, + k1Offset, Bytes.SIZEOF_LONG); + if(ret <= -1){ + return 3; + } else if(ret == 0){ + return 3; + } else { + return -1; + } + } else { + ret = Bytes.compareTo(k2Bytes, k2Offset, Bytes.SIZEOF_LONG, k1Bytes, + k1Offset, Bytes.SIZEOF_LONG); + if(ret <= -1){ + return -3; + } else if(ret == 0){ + return -3; + } else { + return 1; + } + } + } + + + @Override + public void mergeGets(){ + if(newColumns == null){ + return; + } + int oldPos = 0; + int newPos = 0; + int oldSize = columns.size(); + int newSize = newColumns.size(); +// System.out.println("oldSize " +oldSize); +// System.out.println("newSize " +newSize); + + int size = oldSize + newSize; + List mergedList = new ArrayList(size); + List mergedVersions = new ArrayList(size); + + if(oldSize == 0){ + reinit(); + return; + } + if(newSize == 0){ + return; + } + + int res = 0; + KeyValue newe = null; + boolean newDone = false; + KeyValue olde = null; + while(true){ + newe = newColumns.get(newPos); + olde = columns.get(oldPos); + res = Bytes.compareTo(newe.getBuffer(), newe.getOffset(), newe.getLength(), + olde.getBuffer(), olde.getOffset(), olde.getLength()); + if(res <= -1){ + mergedList.add(newe); + mergedVersions.add(newVersions.get(newPos)); + if(++newPos >= newSize){ + newDone = true; + break; + } + } else if(res >= 1){ + mergedList.add(olde); + mergedVersions.add(newVersions.get(oldPos)); + if(++oldPos >= oldSize){ + break; + } + } else { + while(true){ + System.out.println("Error!! Same value int old and new, in" + + "mergeGets"); + } + } + } + if(newDone){ + while(oldPos < oldSize){ + mergedList.add(columns.get(oldPos)); + mergedVersions.add(newVersions.get(oldPos++)); + } + } else { + while(newPos < newSize){ + mergedList.add(newColumns.get(newPos)); + mergedVersions.add(newVersions.get(newPos++)); + } + } + reinit(); + } + + private void reinit(){ + columns.addAll(newColumns); + versions.addAll(newVersions); + newColumns.clear(); + newVersions.clear(); + newColumn = null; + endColumns = false; + } + + /** + * For now only returns the family and the columns + * Not the fastest implementation though columns is a LinkedList, but not too + * concerned about that right now. + * TODO speed it up + * + * @return a string representation of the object + */ + public String toString(){ + StringBuffer sb = new StringBuffer(); + sb.append("/"); + sb.append(new String(this.family)); + sb.append(", columns["); + int i=0; +// byte [] col = null; + KeyValue col = null; + int len = 0; + for(; i 0){ + col = columns.get(i); + sb.append(new String(col.getBuffer(), col.getOffset(), col.getLength())); + sb.append("-"); + //The number of versions fetched + if(i < versions.size() - 1){ + sb.append(versions.get(i)); + } +// col = columns.get(i); +// len = col.length; +// sb.append(new String(col, 0, col.length-2)); +// sb.append("-"); +// //The number of versions fetched +// sb.append(col[len-1]); + } + sb.append("]"); + + return sb.toString(); + } + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/Deletes.java b/src/java/org/apache/hadoop/hbase/regionserver/Deletes.java new file mode 100644 index 0000000..5e046d3 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/Deletes.java @@ -0,0 +1,45 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue; + +/** + * Class that holds a list of deletes and a timestamp of a DeleteFamily entry + * The DeleteFamily timestamp is 0 if there is now such entry in the list + * + */ +public class Deletes { + List deletes = null; + long deleteFamily = 0L; + + /** + * + * @param deletes + * @param deleteFamily + */ + public Deletes(List deletes, long deleteFamily){ + this.deletes = deletes; + this.deleteFamily = deleteFamily; + } + +// public boolean hasDeleteFamily(){ +// return deleteFamily != 0; +// } + + /** + * @return the DeleteFamily timestamp, 0 if no such entry + */ + public long getDeleteFamily(){ + return deleteFamily; + } + + /** + * + * @return the list of deletes + */ + public List getDeletes(){ + return deletes; + } + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java index b2ad69a..b33a30a 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -58,8 +58,14 @@ import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.io.BatchOperation; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.Family; +import org.apache.hadoop.hbase.io.Get; +import org.apache.hadoop.hbase.io.GetColumns; +import org.apache.hadoop.hbase.io.GetFamilies; +import org.apache.hadoop.hbase.io.GetRow; import org.apache.hadoop.hbase.io.HbaseMapWritable; import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.io.RowUpdates; import org.apache.hadoop.hbase.io.Reference.Range; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.util.Bytes; @@ -883,7 +889,6 @@ public class HRegion implements HConstants { dse.initCause(t); throw dse; } - // If we get to here, the HStores have been written. If we get an // error in completeCacheFlush it will release the lock it is holding @@ -1120,6 +1125,54 @@ public class HRegion implements HConstants { } } + + /** + * + * @param get + * @param lockid + * @return the result + * @throws IOException + */ + public List newget(Get get, List result, + final Integer lockid) + throws IOException { +// if (LOG.isDebugEnabled()) { +// LOG.debug("newget: entering"); +// } + ServerGet serverGet = null; + + if(get instanceof GetRow){ + ((GetRow)get).setFamilies(regionInfo.getTableDesc().getFamiliesKeys()); + serverGet = new ServerGetFamilies(get); + } else if(get instanceof GetFamilies){ + serverGet = new ServerGetFamilies(get); + + } else if(get instanceof GetColumns){ + serverGet = new ServerGetColumns(get); + + } else { + serverGet = new ServerGetTop(get); + } + + Integer lid = getLock(lockid, get.getRow()); + + try { + for (Family family : get.getFamilies()) { + //For locality groups will probably end up with a list of stores + //so have to add a loop or something + Store store = stores.get(family.getFamily()); + if (store != null) { + serverGet.setFamily(family.getFamily()); + serverGet.setColumns(family.getColumns()); + store.newget(serverGet, result); + } + } + return result; + } finally { + if(lockid == null) releaseRowLock(lid); + } + } + /** * Return all the data for the row that matches row exactly, * or the one that immediately preceeds it, at or immediately before @@ -1328,6 +1381,51 @@ public class HRegion implements HConstants { } } + + public void updateRow(RowUpdates rups, Integer lockid, boolean writeToWAL) + throws IOException { +// System.out.println("hreUR " + System.nanoTime()); + + checkReadOnly(); + + //Think that this should be moved to the client +// validateValuesLength(b); + + // Do a rough check that we have resources to accept a write. The check is + // 'rough' in that between the resource check and the call to obtain a + // read lock, resources may run out. For now, the thought is that this + // will be extremely rare; we'll deal with it when it happens. + checkResources(); + splitsAndClosesLock.readLock().lock(); + try { + // We obtain a per-row lock, so other clients will block while one client + // performs an update. The read lock is released by the client calling + // #commit or #abort or if the HRegionServer lease on the lock expires. + // See HRegionServer#RegionListener for how the expire on HRegionServer + // invokes a HRegion#abort. + byte [] row = rups.getRow(); + // If we did not pass an existing row lock, obtain a new one + Integer lid = getLock(lockid, row); + try { +// for(PutFamily family : rups.getFamilies()){ + for(Family family : rups.getFamilies()){ + byte[] familyName = family.getFamily(); + checkFamily(familyName); +// if (LOG.isDebugEnabled()) { +// LOG.debug("updateRow: updating family " + new String(familyName)); +// } + newUpdate(familyName, family.getColumns(), writeToWAL); + } + } + + finally { + if(lockid == null) releaseRowLock(lid); + } + } finally { + splitsAndClosesLock.readLock().unlock(); + } + } + /** * Performs an atomic check and save operation. Checks if * the specified expected values have changed, and if not @@ -1762,6 +1860,40 @@ public class HRegion implements HConstants { } } + public void newUpdate(byte [] family, List bss, boolean writeToWAL) + throws IOException { +// System.out.println("hreNU " + System.nanoTime()); + boolean flush = false; + this.updatesLock.readLock().lock(); + List kvs = new ArrayList(bss.size()); + for(byte[] bs : bss){ + kvs.add(new KeyValue(bs)); + } + + try { + if (writeToWAL) { + this.log.append(regionInfo.getRegionName(), + regionInfo.getTableDesc().getName(), kvs, + (regionInfo.isMetaRegion() || regionInfo.isRootRegion())); + } + long size = 0; + Store store = getStore(family); + for (KeyValue kv: kvs) { + //This instantiation of a new KeyValue can probably be skipped but it + //takes some rewriting of the code that follows +// System.out.println("regio " +System.nanoTime()); + size = this.memcacheSize.addAndGet(store.newAdd(kv)); + } + flush = isFlushSize(size); + } finally { + this.updatesLock.readLock().unlock(); + } + if (flush) { + // Request a cache flush. Do it outside update lock. + requestFlush(); + } + } + private void requestFlush() { if (this.flushListener == null) { return; @@ -1845,6 +1977,24 @@ public class HRegion implements HConstants { } } + + /* + * Make sure this is a valid column for the current table + * @param columnName + * @throws NoSuchColumnFamilyException + */ + private void checkFamily(final byte[] family) + throws NoSuchColumnFamilyException { + if (family == null) { + return; + } + if (!regionInfo.getTableDesc().hasFamily(family)) { + throw new NoSuchColumnFamilyException("Column family on " + + Bytes.toString(family) + " does not exist in region " + + this + " in table " + regionInfo.getTableDesc()); + } + } + /** * Obtain a lock on the given row. Blocks until success. * diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 00614a5..0a4d6c4 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -83,8 +83,11 @@ import org.apache.hadoop.hbase.client.ServerConnectionManager; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.Get; +import org.apache.hadoop.hbase.io.GetColumns; import org.apache.hadoop.hbase.io.HbaseMapWritable; import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.io.RowUpdates; import org.apache.hadoop.hbase.ipc.HBaseRPC; import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler; import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion; @@ -1609,6 +1612,30 @@ public class HRegionServer implements HConstants, HRegionInterface, } } + public KeyValue [] newGet(final byte [] regionName, Get get, + final long lockId) + throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Entering newGet"); + } + checkOpen(); + requestCount.incrementAndGet(); + try { + HRegion region = getRegion(regionName); + List result = new ArrayList(); + + region.newget(get, result, getLockFromId(lockId)); + + if (LOG.isDebugEnabled()) { + LOG.debug("newGet: result.isEmpty " +result.isEmpty()); + } + return result.isEmpty() ? null : result.toArray(new KeyValue[0]); + + } catch (Throwable t) { + throw convertThrowableToIOE(cleanup(t)); + } + } + public RowResult getClosestRowBefore(final byte [] regionName, final byte [] row, final byte [] columnFamily) throws IOException { @@ -1696,6 +1723,29 @@ public class HRegionServer implements HConstants, HRegionInterface, } return -1; } + + + public int updateRow(final byte [] regionName, final RowUpdates rups) + throws IOException { + int i = 0; + checkOpen(); + try { + boolean writeToWAL = true; + this.cacheFlusher.reclaimMemcacheMemory(); + this.requestCount.incrementAndGet(); + Integer lock = getLockFromId(rups.getRowLock()); + HRegion region = getRegion(regionName); + region.updateRow(rups, lock, writeToWAL); + } catch(WrongRegionException ex) { + return i; + } catch (NotServingRegionException ex) { + return i; + } catch (Throwable t) { + throw convertThrowableToIOE(cleanup(t)); + } + return -1; + } + public boolean checkAndSave(final byte [] regionName, final BatchUpdate b, final HbaseMapWritable expectedValues) diff --git a/src/java/org/apache/hadoop/hbase/regionserver/Key.java b/src/java/org/apache/hadoop/hbase/regionserver/Key.java new file mode 100644 index 0000000..d08bd05 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/Key.java @@ -0,0 +1,107 @@ +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.hadoop.hbase.util.Bytes; + +public class Key { + private byte[] buffer = null; + private int offset = 0; +// private int keyOffset = 0; +// private int keyLen = 0; +// private int rowOffset = 0; +// private int rowLen = 0; +// private int famOffset = 0; +// private int famLen = 0; +// private int colOffset = 0; +// private int colLen = 0; +// private int tsOffset = 0; +// private int typeOffset = 0; + +/******************************************************************************* + * Constructors + ******************************************************************************/ + public Key(Key k){ + this.buffer = k.getBuffer(); + this.offset = k.getOffset(); + } + public Key(KeyValue kv){ + this.buffer = kv.getBuffer(); + this.offset = kv.getOffset(); + } + public Key(byte[] bytes, int offset){ + this.buffer = bytes; + this.offset = offset; + } + +/******************************************************************************* + * Methods + ******************************************************************************/ + public byte [] getBuffer(){ + return this.buffer; + } + + public int getOffset(){ + return this.offset; + } + + public void set(Key k){ + this.buffer = k.getBuffer(); + this.offset = k.getOffset(); + } +// public void set(KeyValue kv){ +// this.buffer = kv.getBuffer(); +// this.offset = kv.getOffset(); +// } + + + /** + * @return a string representing the object + */ + @Override + public String toString(){ + if(buffer == null){ + return ""; + } + + int off = 0; + //Getting keyLen + int keyLen = Bytes.toInt(this.buffer, off); + off += Bytes.SIZEOF_INT; + + //Skipping Value length + off += Bytes.SIZEOF_INT; + + //Getting row + short rowLen = Bytes.toShort(this.buffer, off); + off += Bytes.SIZEOF_SHORT; + String row = rowLen <= 0 ? "" : Bytes.toString(buffer, off, rowLen); + off += rowLen; + + //Getting family + byte famLen = this.buffer[off]; + off += Bytes.SIZEOF_BYTE; + String fam = famLen <= 0 ? "" : Bytes.toString(buffer, off, famLen); + off += famLen; + + //Getting column + int colLen = 2*Bytes.SIZEOF_INT + keyLen - off - Bytes.SIZEOF_LONG - + Bytes.SIZEOF_BYTE; + + String col = colLen <= 0 ? "" : Bytes.toString(buffer, off, colLen); + off += colLen; + + //Getting timestamp + long ts = 0L; + byte type = 0; + if(2*Bytes.SIZEOF_INT + keyLen - off == 9){ + ts = Bytes.toLong(buffer, off, Bytes.SIZEOF_LONG); + off += Bytes.SIZEOF_LONG; + type = buffer[off]; + } + + return (row + "/" + fam + "/" + col + "/" + ts + "/" + + KeyValue.Type.codeToType(type)); + } + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java b/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java index 0e1595d..c658758 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java @@ -26,6 +26,7 @@ import java.lang.management.RuntimeMXBean; import java.rmi.UnexpectedException; import java.util.ArrayList; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.NavigableSet; @@ -82,6 +83,10 @@ class Memcache { // TODO: Fix this guess by studying jprofiler private final static int ESTIMATED_KV_HEAP_TAX = 60; + + //Not sure if this should be here, might put it in KeyValue + private static final int KEY_SIZES = Bytes.SIZEOF_SHORT + Bytes.SIZEOF_BYTE + + Bytes.SIZEOF_LONG + Bytes.SIZEOF_BYTE; /** * Default constructor. Used for tests. @@ -93,7 +98,16 @@ class Memcache { /** * Constructor. * @param ttl The TTL for cache entries, in milliseconds. +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/regionserver/Memcache.java +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/regionserver/Memcache.java + * @param c +======= + * @param c + * @param rc +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/regionserver/Memcache.java +======= * @param c +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/regionserver/Memcache.java */ public Memcache(final long ttl, final KeyValue.KVComparator c) { this.ttl = ttl; @@ -198,6 +212,336 @@ class Memcache { return size; } + + /** + * Write an update + * @param kv + * @return approximate size of the passed key and value. + */ + long newAdd(final KeyValue kv, boolean multiFamily) { + long size = -1; + this.lock.readLock().lock(); + long deleteSize = 0L; + try { + + //Have to find out what want to do here, to find the fastest way of removing + //things that are under a delete. + //Actions that will take place here are: + //1. Insert a put at the right place + //2. Insert a deleteFamily and a deleteColumn entry and deleting all the + //related entries already in there. + //3. Insert a delete, with timestamp, and finding the put in memcache, + //deleting both of them. + + //first check what type the current kv is + byte type = kv.getType(); + + boolean notpresent = false; + if(type != KeyValue.Type.Put.getCode()){ + List deletes = new ArrayList(); + SortedSet tailSet = null; + if(type == KeyValue.Type.DeleteFamily.getCode()){ + //need to check row/fam and bigger ts + //cases for mem: + //1. r/f same but ts bigger, next from headset + //2. r/f same and ts smaller or equal, add to deleteList + //3. r or f not the same, done get next kv + tailSet = this.memcache.tailSet(kv); + int tailsize = tailSet.size(); + int ret = 0; + for(KeyValue mem : tailSet){ + ret = deleteFamilyCompare(mem, kv, multiFamily); + if(ret == 0){ + deletes.add(mem); + continue; + } else if(ret == 1){ + break; + } + } + notpresent = this.memcache.add(kv); + size = heapSize(kv, notpresent); + } else if(type == KeyValue.Type.DeleteColumn.getCode()){ + deletes = new ArrayList(); + //Need to check row/fam/col and bigger ts + tailSet = this.memcache.tailSet(kv); + int ret = 0; + for(KeyValue mem : tailSet){ + ret = deleteColumnCompare(mem, kv, multiFamily); + if(ret == 0){ + deletes.add(mem); + continue; + } else if(ret == 1){ + break; + } + } + notpresent = this.memcache.add(kv); + size = heapSize(kv, notpresent); + } else { + deletes = new ArrayList(); + //Need to check row/fam/col/ts + tailSet = this.memcache.tailSet(kv); + int tailsize = tailSet.size(); + + int ret = 0; + for(KeyValue mem : tailSet){ + ret = deleteCompare(mem, kv, multiFamily); + if(ret == 0){ + deletes.add(mem); + break; + } else if(ret == 1){ + break; + } + } + } + + for(KeyValue delete : deletes){ + notpresent = this.memcache.remove(delete); + deleteSize += heapSize(delete, notpresent); + } + } else { + notpresent = this.memcache.add(kv); + int memCacheSize = this.memcache.size(); + size = heapSize(kv, notpresent); + } + + } finally { + this.lock.readLock().unlock(); + } + return size - deleteSize; + } + + /** + * + * @param mem + * @param up + * @return -1 to get next from headset, 0 to add the current headSet kv to the + * delete list and 1 if done. + */ + private int deleteFamilyCompare(KeyValue mem, KeyValue up, + boolean multiFamily){ + byte [] memBytes = mem.getBuffer(); + int memOffset = mem.getOffset(); + + byte [] upBytes = up.getBuffer(); + int upOffset = up.getOffset(); + + //Getting key lengths + int memKeyLen = Bytes.toInt(memBytes, memOffset); + memOffset = Bytes.SIZEOF_INT; + int upKeyLen = Bytes.toInt(upBytes, upOffset); + upOffset = Bytes.SIZEOF_INT; + + //Skipping value lengths + memOffset += Bytes.SIZEOF_INT; + upOffset += Bytes.SIZEOF_INT; + + //Getting row lengths + short memRowLen = Bytes.toShort(memBytes, memOffset); + memOffset += Bytes.SIZEOF_SHORT; + short upRowLen = Bytes.toShort(upBytes, upOffset); + upOffset += Bytes.SIZEOF_SHORT; + //Compare rows + int ret = Bytes.compareTo(memBytes, memOffset, memRowLen, upBytes, + upOffset, upRowLen); + if(ret != 0){ + return 1; + } + memOffset += memRowLen; + upOffset += upRowLen; + + //Getting family lengths + byte memFamLen = memBytes[memOffset]; + memOffset += Bytes.SIZEOF_BYTE; + byte upFamLen = upBytes[upOffset]; + upOffset += Bytes.SIZEOF_BYTE; + + //Compare families + if(multiFamily){ + ret = Bytes.compareTo(memBytes, memOffset, memFamLen, upBytes, upOffset, + upFamLen); + if(ret != 0){ + return 1; + } + } + memOffset += memFamLen; + upOffset += upFamLen; + + //TODO Skipping columns can make this calculation more efficient + + //Get column lengths + int memColLen = memKeyLen - memRowLen - memFamLen - KEY_SIZES; + int upColLen = upKeyLen - upRowLen - upFamLen - KEY_SIZES; + + memOffset += memColLen; + upOffset += upColLen; + + //Compare ts + ret = Bytes.compareTo(memBytes, memOffset, Bytes.SIZEOF_LONG, upBytes, + upOffset, Bytes.SIZEOF_LONG); + if(ret <= 0){ + return 0; + } + + return -1; + } + + /** + * + * @param mem + * @param up + * @return -1 to get next from headset, 0 to add the current headSet kv to the + * delete list and 1 if done. + */ + private int deleteColumnCompare(KeyValue mem, KeyValue up, + boolean multiFamily){ + byte [] memBytes = mem.getBuffer(); + int memOffset = mem.getOffset(); + + byte [] upBytes = up.getBuffer(); + int upOffset = up.getOffset(); + + //Getting key lengths + int memKeyLen = Bytes.toInt(memBytes, memOffset); + memOffset = Bytes.SIZEOF_INT; + int upKeyLen = Bytes.toInt(upBytes, upOffset); + upOffset = Bytes.SIZEOF_INT; + + //Skipping value lengths + memOffset += Bytes.SIZEOF_INT; + upOffset += Bytes.SIZEOF_INT; + + //Getting row lengths + short memRowLen = Bytes.toShort(memBytes, memOffset); + memOffset += Bytes.SIZEOF_SHORT; + short upRowLen = Bytes.toShort(upBytes, upOffset); + upOffset += Bytes.SIZEOF_SHORT; + //Compare rows + int ret = Bytes.compareTo(memBytes, memOffset, memRowLen, upBytes, + upOffset, upRowLen); + if(ret != 0){ + return 1; + } + memOffset += memRowLen; + upOffset += upRowLen; + + //Getting family lengths + byte memFamLen = memBytes[memOffset]; + memOffset += Bytes.SIZEOF_BYTE; + byte upFamLen = upBytes[upOffset]; + upOffset += Bytes.SIZEOF_BYTE; + + //Compare families + if(multiFamily){ + ret = Bytes.compareTo(memBytes, memOffset, memFamLen, upBytes, upOffset, + upFamLen); + if(ret != 0){ + return 1; + } + } + memOffset += memFamLen; + upOffset += upFamLen; + + //Get column lengths + int memColLen = memKeyLen - memRowLen - memFamLen - KEY_SIZES; + int upColLen = upKeyLen - upRowLen - upFamLen - KEY_SIZES; + + //Compare columns + ret = Bytes.compareTo(memBytes, memOffset, memColLen, upBytes, upOffset, + upColLen); + if(ret != 0){ + return 1; + } + memOffset += memColLen; + upOffset += upColLen; + + //Compare ts + ret = Bytes.compareTo(memBytes, memOffset, Bytes.SIZEOF_LONG, upBytes, + upOffset, Bytes.SIZEOF_LONG); + if(ret <= 0){ + return 0; + } + + return -1; + } + + /** + * + * @param mem + * @param up + * @return -1 to get next from headset, 0 to add the current headSet kv to the + * delete list and 1 if done. + */ + private int deleteCompare(KeyValue mem, KeyValue up, boolean multiFamily){ + byte [] memBytes = mem.getBuffer(); + int memOffset = mem.getOffset(); + + byte [] upBytes = up.getBuffer(); + int upOffset = up.getOffset(); + + //Getting key lengths + int memKeyLen = Bytes.toInt(memBytes, memOffset); + memOffset = Bytes.SIZEOF_INT; + int upKeyLen = Bytes.toInt(upBytes, upOffset); + upOffset = Bytes.SIZEOF_INT; + + //Skipping value lengths + memOffset += Bytes.SIZEOF_INT; + upOffset += Bytes.SIZEOF_INT; + + //Getting row lengths + short memRowLen = Bytes.toShort(memBytes, memOffset); + memOffset += Bytes.SIZEOF_SHORT; + short upRowLen = Bytes.toShort(upBytes, upOffset); + upOffset += Bytes.SIZEOF_SHORT; + //Compare rows + int ret = Bytes.compareTo(memBytes, memOffset, memRowLen, upBytes, + upOffset, upRowLen); + if(ret != 0){ + return 1; + } + memOffset += memRowLen; + upOffset += upRowLen; + + //Getting family lengths + byte memFamLen = memBytes[memOffset]; + memOffset += Bytes.SIZEOF_BYTE; + byte upFamLen = upBytes[upOffset]; + upOffset += Bytes.SIZEOF_BYTE; + + //Compare families + if(multiFamily){ + ret = Bytes.compareTo(memBytes, memOffset, memFamLen, upBytes, upOffset, + upFamLen); + if(ret != 0){ + return 1; + } + } + memOffset += memFamLen; + upOffset += upFamLen; + + //Get column lengths + int memColLen = memKeyLen - memRowLen - memFamLen - KEY_SIZES; + int upColLen = upKeyLen - upRowLen - upFamLen - KEY_SIZES; + + //Compare columns + ret = Bytes.compareTo(memBytes, memOffset, memColLen, upBytes, upOffset, + upColLen); + if(ret != 0){ + return 1; + } + memOffset += memColLen; + upOffset += upColLen; + + //Compare ts + ret = Bytes.compareTo(memBytes, memOffset, Bytes.SIZEOF_LONG, upBytes, + upOffset, Bytes.SIZEOF_LONG); + if(ret == 0){ + return 0; + } + + return -1; + } + /* * Calculate how the memcache size has changed, approximately. Be careful. * If class changes, be sure to change the size calculation. @@ -383,6 +727,85 @@ class Memcache { return hasEnough; } + + /** + * + * @param sget + * @param results + * @return 0 if need to go to next storefile, 1 if done + * @throws IOException + */ + int newget(ServerGet sget, List results, boolean multiFamily) + throws IOException { + this.lock.readLock().lock(); + int retCode = 0; + try { + // Used to be synchronized but now with weak iteration, no longer needed. + retCode = internalNewGet(this.memcache, sget, results, multiFamily); + if(retCode == -1){ + throw new IOException("Internal error in get, return code = -1"); + } else if(retCode == 0){ + sget.mergeGets(); + sget.clear(); + retCode = internalNewGet(this.snapshot, sget, results, multiFamily); + if(retCode == -1){ + throw new IOException("Internal error in get, return code = -1"); + } + } + } finally { + this.lock.readLock().unlock(); + } + return retCode; + } + + /** + * + * @param set + * @param sget + * @param result + * @return -1 if error ,0 if not ready and 1 if ready + */ + private int internalNewGet(SortedSet set, ServerGet sget, + List result, boolean multiFamily) + throws IOException{ + if (set.isEmpty()){ + return 0; + } + //Getting only the things that are related to this row + //TODO add family and column in the future, right now it just adds + //complexity since you need the first column and/or ts and maybe there is + //no column to look for yet as in the cases of getFamilies and getTop + set = set.tailSet(new KeyValue(sget.getRow())); + + //TODO have to remember to check the order of the set, so that tailSet + //returns the things that are smaller and not bigger + int retCode = 0; + // The cases that we need at this level: + //0 next + //1 include + //2 next store + //3 done + for(KeyValue kv : set){ + retCode = sget.compareTo(kv, multiFamily); + switch(retCode) { + //Do not include in result, look at next kv + case 0: break; + + //Include in result + case 1: result.add(kv); break; + + //Go to next storefile + case 2: return 0; + + //Done, early out + case 3: return 1; + + default : return -1; + } + } + return 0; + } + /** * @param row Row to look for. * @param candidateKeys Map of candidate keys (Accumulation over lots of diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ServerGet.java b/src/java/org/apache/hadoop/hbase/regionserver/ServerGet.java new file mode 100644 index 0000000..f07bbf1 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/ServerGet.java @@ -0,0 +1,86 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; + +import java.util.List; + +import org.apache.hadoop.hbase.filter.RowFilterInterface; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.TimeRange; + +public interface ServerGet { + /** + * Sets all the internal variables to original settings, needs to be done + * in between every storeFile + */ + public void clear(); + + /** + * + * Return codes that need to be implemented by method + * -1 if cur should not be included in result + * 0 if cur should be included in result + * 1 if cur has reached the put/delete border + * 2 if cur has reached the next row + * 3 if cur is past this StoreFiles boundary for this Get + * 4 if cur is past this Stores boundary for this get + * + * @param kv + * @return the return code + */ + public int compareTo(KeyValue kv) throws IOException; + /** + * + * Return codes that need to be implemented by method + * -1 if cur should not be included in result + * 0 if cur should be included in result + * 1 if cur has reached the put/delete border + * 2 if cur has reached the next row + * 3 if cur is past this StoreFiles boundary for this Get + * 4 if cur is past this Stores boundary for this get + * + * @param kv + * @param multiFamily, if the store includes multiple families + * @return the return code + */ + public int compareTo(KeyValue kv, boolean multiFamily) throws IOException; + +// public byte [] getFirstColumn(); + public byte [] getRow(); + + public TimeRange getTimeRange(); + + public List getColumns(); + public void setColumns(List columns); +// public void setColumns(List columns); +// public void setColumnsFromBytes(List columns); + public List getVersions(); +// public byte [][] getColumns(); +// public void setColumns(byte [][] columns); + + public List getDeletes(); + public void setDeletes(List deletes); + public List getNewDeletes(); + + public byte [] getFamily(); + public void setFamily(byte [] family); + + public void setFilter(RowFilterInterface filter); + + public long getNow(); + public void setNow(); + public void setNow(long now); + + public long getTTL(); + public void setTTL(long ttl); + + public Deletes mergeDeletes(List l1, List l2); + public Deletes mergeDeletes(List l1, List l2, + boolean multiFamily); + + public void mergeGets(); + + public String toString(); + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ServerGetColumns.java b/src/java/org/apache/hadoop/hbase/regionserver/ServerGetColumns.java new file mode 100644 index 0000000..7b49525 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/ServerGetColumns.java @@ -0,0 +1,221 @@ +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.io.Get; +import org.apache.hadoop.hbase.util.Bytes; + + +public class ServerGetColumns extends AbstractServerGet { + +// private List newDeletes = new ArrayList(); + + private List columnsToDelete = new ArrayList(); + +// private static final int KEY_OFFSET = 2*Bytes.SIZEOF_INT; + + private boolean outOfTimeRange = false; + +// private static final int NEXT_KV = 0; +// private static final int ADD = 1; +// private static final int NEXT_SF = 2; +// private static final int DONE = 3; + + public ServerGetColumns(Get get){ + super(get); + } + +// public List getNewDeletes(){ +// return newDeletes; +// } + + + /** + * @param kv, the KeyValue to compare with + * @return int the return code + */ +// public int compareTo(KeyValue kv){ +// return compareTo(kv, false); +// } + + public int compareTo(KeyValue kv, boolean multiFamily){ + if(isDone()){ + return super.DONE; + } + + int initialOffset = kv.getOffset(); + int offset = initialOffset; + byte [] bytes = kv.getBuffer(); + + //Getting key length + int keyLen = Bytes.toInt(bytes, offset); + offset += Bytes.SIZEOF_INT; + + //Skipping valueLength + offset += Bytes.SIZEOF_INT; + + //Getting row length + short rowLen = Bytes.toShort(bytes, offset); + offset += Bytes.SIZEOF_SHORT; + + byte [] row = super.getRow(); + int ret = Bytes.compareTo(row, 0, row.length, bytes, offset, rowLen); + if(ret <= -1){ + if(outOfTimeRange){ + return super.DONE; + } + return super.NEXT_SF; + } else if(ret >= 1){ + return super.NEXT_KV; + } + offset += rowLen; + + //This is only for the future if we have more than on family in the same + //storefile, can be turned off for now + //Getting family length + byte famLen = bytes[offset]; + offset += Bytes.SIZEOF_BYTE; + + if(multiFamily){ + byte [] family = super.getFamily(); + ret = Bytes.compareTo(family, 0, family.length, bytes, offset, rowLen); + if(ret <= -1){ + if(outOfTimeRange){ + return super.DONE; + } + return super.NEXT_SF; + } else if(ret >= 1){ + return super.NEXT_KV; + } + } + offset += famLen; + + //Getting column length + int colLen = super.KEY_OFFSET + keyLen - (offset-initialOffset) - + Bytes.SIZEOF_LONG - Bytes.SIZEOF_BYTE; + //Could be switched to something like: + //initloffset + keylength - TIMESTAMP_TYPE_SIZE - loffset; + + //Checking TTL and TimeRange before column so that if the entry turns out + //to be a delete we can be sure that all the deletes in the delete list + //are valid deletes + int tsOffset = offset + colLen; + long ts = Bytes.toLong(bytes, tsOffset); + ret = super.checkTTL(ts); + if(ret == 0){ + return super.NEXT_KV; + } + + ret = super.get.getTimeRange().withinTimeRange(ts); + if(ret != 1){ + return super.NEXT_KV; + } + + //Check if kv is a delete + if(super.isDelete(bytes, initialOffset, keyLen)){ + super.newDeletes.add(kv); + return super.NEXT_KV; + } + + //There is not going to be any ts in here, but all tss will be taken care + //of in the TimeRange check + ret = compareColumn(bytes, offset, colLen); + if(ret <= -1){ + return super.NEXT_SF; + } else if(ret >= 1){ + return super.NEXT_KV; + } + offset += colLen; + + if(!super.deletes.isEmpty()){ + ret = super.isDeleted(bytes, initialOffset, rowLen, famLen, + colLen, multiFamily); + if(ret != 0){ + if(ret >= 1){ + outOfTimeRange = true; + } + return super.NEXT_KV; + } + } + + //TODO + //ret = compareFilter(kv); + + //Includes remove from getList, TODO change for other calls + return updateVersions(); + } + +/******************************************************************************* +* Helpers +*******************************************************************************/ + //TODO These methods needs to be changed depending on the type of get you are + //dealing with + + private boolean isDone(){ + int len = super.columns.size(); + if(super.columns.isEmpty()){ + return true; + } + return false; + } + + + /** + * This method are very different than the other compares, because it will + * loop through the columns in the column list until the current kv is found + * or the column is smaller than the kv. + * + */ + private int compareColumn(byte[] bytes, int offset, int length){ + if(columns != null){ + if(super.column == null){ + super.column = super.columns.get(super.columnPos); + } + } + int res = 0; + while(true){ + res = Bytes.compareTo(column.getBuffer(), column.getOffset(), + column.getLength(), bytes, offset, length); + if(res >= 0){ + return res; + } +// int size = columns.size(); + if(columnPos < columns.size()-1){ + column = columns.get(++super.columnPos); + } else { + return res; + } + } + } + + + + + private int updateVersions(){ + short version = 0; + int size = 0; + try{ + size = super.versions.size(); + version = super.versions.get(super.columnPos); + } catch (Exception e){ + super.versions.add(version); + size = super.versions.size(); +// return super.ADD; + } + int pos = super.columnPos; + version++; + if(version >= super.getMaxVersions()){ + //Remove this position from list, for now, might do something else later + super.columns.remove(super.columnPos--); + } else { + ((ArrayList)super.versions).set(super.columnPos, version); + } + size = super.versions.size(); + return super.ADD; + } + + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ServerGetFamilies.java b/src/java/org/apache/hadoop/hbase/regionserver/ServerGetFamilies.java new file mode 100644 index 0000000..ef79afa --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/ServerGetFamilies.java @@ -0,0 +1,278 @@ +package org.apache.hadoop.hbase.regionserver; + +//import java.io.IOException; +//import java.util.SortedSet; +//import java.util.TreeSet; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.Get; +import org.apache.hadoop.hbase.regionserver.Key; + +//import org.apache.hadoop.hbase.io.GetColumns; +//import org.apache.hadoop.hbase.io.GetFamilies; +import org.apache.hadoop.hbase.util.Bytes; + +public class ServerGetFamilies extends AbstractServerGet{ + +// private List newColumns = null; +// private List newColumns = null; +// private List newVersions = null; +// +// private KeyValue newColumn = null; +// private boolean endColumns = false; + + public ServerGetFamilies(Get getFamilies){ + super(getFamilies); + super.newColumns = new ArrayList(); + super.newVersions = new ArrayList(); + } + + public List getNewColumns(){ + return newColumns; + } + + @Override + public int compareTo(KeyValue kv, boolean multiFamily) { + if(isDone()){ + return super.DONE; + } + + int initialOffset = kv.getOffset(); + int offset = initialOffset; + byte [] bytes = kv.getBuffer(); + + //Getting key length + int keyLen = Bytes.toInt(bytes, offset); + offset += Bytes.SIZEOF_INT; + + //Skipping valueLength + offset += Bytes.SIZEOF_INT; + + //Getting row length + short rowLen = Bytes.toShort(bytes, offset); + offset += Bytes.SIZEOF_SHORT; + + byte [] row = super.getRow(); + int ret = Bytes.compareTo(row, 0, row.length, bytes, offset, rowLen); + if(ret <= -1){ + return super.NEXT_SF; + } else if(ret >= 1){ + return super.NEXT_KV; + } + offset += rowLen; + //This is only for the future if we have more than on family in the same + //storefile, can be turned off for now + //Getting family length + byte famLen = bytes[offset]; + offset += Bytes.SIZEOF_BYTE; + + if(multiFamily){ + byte [] family = super.getFamily(); + ret = Bytes.compareTo(family, 0, family.length, bytes, offset, rowLen); + if(ret <= -1){ + return super.NEXT_SF; + } else if(ret >= 1){ + return super.NEXT_KV; + } + } + offset += famLen; + + //Getting column length + int colLen = super.KEY_OFFSET + keyLen - (offset-initialOffset) - + Bytes.SIZEOF_LONG - Bytes.SIZEOF_BYTE; + //Could be switched to something like: + //initloffset + keylength - TIMESTAMP_TYPE_SIZE - loffset; + + //Checking TTL and TimeRange before column so that if the entry turns out + //to be a delete we can be sure that all the deletes in the delete list + //are valid deletes + int tsOffset = offset + colLen; + long ts = Bytes.toLong(bytes, tsOffset); + ret = super.checkTTL(ts); + if(ret == 0){ + return super.NEXT_KV; + } + ret = super.get.getTimeRange().withinTimeRange(ts); + if(ret != 1){ + return super.NEXT_KV; + } + //Check if kv is a delete + if(super.isDelete(bytes, initialOffset, keyLen)){ + super.newDeletes.add(kv); + return super.NEXT_KV; + } + //TODO + //ret = compareFilter(kv); + + //Check if this kv is deleted + if(!super.deletes.isEmpty()){ + ret = super.isDeleted(bytes, initialOffset, rowLen, famLen, + colLen, multiFamily); + if(ret != 0){ + return super.NEXT_KV; + } + } + //There is not going to be any ts in here, but all tss will be taken care + //of in the TimeRange check + ret = compareColumn(bytes, offset, colLen); + + //Includes remove from getList, TODO change for other calls + return updateVersions(ret); + } + + //TODO These methods needs to be changed depending on the type of get you are + //dealing with + + +// public List mergeGets(boolean multiFamily){ +// public void mergeGets(boolean multiFamily){ +// int oldPos = 0; +// int newPos = 0; +// int oldSize = super.columns.size(); +// int newSize = newColumns.size(); +// System.out.println("oldSize " +oldSize); +// System.out.println("newSize " +newSize); +// +// int size = oldSize + newSize; +// List mergedList = new ArrayList(size); +// List mergedVersions = new ArrayList(size); +// +// if(oldSize == 0){ +// reinit(); +// return; +// } +// if(newSize == 0){ +// return; +// } +// +// int res = 0; +// KeyValue newe = null; +// boolean newDone = false; +// KeyValue olde = null; +// while(true){ +// newe = newColumns.get(newPos); +// olde = super.columns.get(oldPos); +// res = Bytes.compareTo(newe.getBuffer(), newe.getOffset(), newe.getLength(), +// olde.getBuffer(), olde.getOffset(), olde.getLength()); +// if(res <= -1){ +// mergedList.add(newe); +// mergedVersions.add(newVersions.get(newPos)); +// if(++newPos >= newSize){ +// newDone = true; +// break; +// } +// } else if(res >= 1){ +// mergedList.add(olde); +// mergedVersions.add(newVersions.get(oldPos)); +// if(++oldPos >= oldSize){ +// break; +// } +// } else { +// while(true){ +// System.out.println("Error!! Same value int old and new, in" + +// "mergeGets"); +// } +// } +// } +// if(newDone){ +// while(oldPos < oldSize){ +// mergedList.add(super.columns.get(oldPos)); +// mergedVersions.add(newVersions.get(oldPos++)); +// } +// } else { +// while(newPos < newSize){ +// mergedList.add(newColumns.get(newPos)); +// mergedVersions.add(newVersions.get(newPos++)); +// } +// } +// reinit(); +// } + + /** + * + * + * @param bytes + * @param offset + * @param length + * @return -1 if this column cannot be found in newColumn nor columns, 0 if + * this as the same as newColumn and 1 if the same as column + */ + private int compareColumn(byte[] bytes, int offset, int length) { + int res = 0; + + if(newColumn != null){ + res = Bytes.compareTo(newColumn.getBuffer(), newColumn.getOffset(), + newColumn.getLength(), bytes, offset, length); + if(res == 0){ + return 0; + } + else if(res >= 1){ + for(int i=0; i<10; i++){ + System.out.println("Error in compareColumn, res " +res); + } + } + } + + if(!endColumns){ + if(columns != null && columns.size() > 0){ + if(super.column == null){ + super.column = super.columns.get(super.columnPos); + } + + while(true){ + res = Bytes.compareTo(super.column.getBuffer(), super.column.getOffset(), + super.column.getLength(), bytes, offset, length); + //so when the entry we are looking at at the moment is smaller than + // column we know that it is not in there , so add to newColumns + if(res >= 1 ){ + break; + } else if(res == 0){ + return 1; + } else { + if(super.columnPos < super.columns.size()-1){ + super.column = super.columns.get(++super.columnPos); + } else { + endColumns = true; + break; + } + } + } + } + } + newColumn = new KeyValue(bytes, offset, length); + newColumns.add(newColumn); + return -1; + } + + //if storefile header time is older than oldest time asked for in TimeRange + //then abort or it this is the last storefile + private boolean isDone(){ + return false; + } + + private int updateVersions(int returnCode){ + if(returnCode == -1){ + newVersions.add((short)1); +// System.out.println("adding a new entry to list, size " + newColumns.size()); + return super.ADD; + } else if(returnCode == 0){ + int pos = newVersions.size(); + short value = newVersions.get(pos); + if(++value > super.getMaxVersions()){ + return super.NEXT_KV; + } + newVersions.set(pos, value); + return super.ADD; + } else { + short value = super.versions.get(columnPos); + if(++value > super.getMaxVersions()){ + return super.NEXT_KV; + } + super.versions.set(super.columnPos, value); + return super.ADD; + } + } + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ServerGetTop.java b/src/java/org/apache/hadoop/hbase/regionserver/ServerGetTop.java new file mode 100644 index 0000000..18dfbb2 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/ServerGetTop.java @@ -0,0 +1,127 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.Get; +import org.apache.hadoop.hbase.io.GetTop; +import org.apache.hadoop.hbase.util.Bytes; + +public class ServerGetTop extends AbstractServerGet { +// private static final Log LOG = LogFactory.getLog(ServerGetTop.class); + + private int counter = 0; + + public ServerGetTop(Get get){ + super(get); + this.counter = ((GetTop)get).getFetches(); + } + + @Override + public int compareTo(KeyValue kv, boolean multiFamily) { + if(isDone()){ + return super.DONE; + } + + int initialOffset = kv.getOffset(); + int offset = initialOffset; + byte [] bytes = kv.getBuffer(); + + //Getting key length + int keyLen = Bytes.toInt(bytes, offset); + offset += Bytes.SIZEOF_INT; + + //Skipping valueLength + offset += Bytes.SIZEOF_INT; + + //Getting row length + short rowLen = Bytes.toShort(bytes, offset); + offset += Bytes.SIZEOF_SHORT; + + byte [] row = super.getRow(); + int ret = Bytes.compareTo(row, 0, row.length, bytes, offset, rowLen); + if(ret <= -1){ + return super.NEXT_SF; + } else if(ret >= 1){ + return super.NEXT_KV; + } + offset += rowLen; +// if (LOG.isDebugEnabled()) { +// LOG.debug("t " + System.nanoTime()); +// } + //This is only for the future if we have more than on family in the same + //storefile, can be turned off for now + //Getting family length + byte famLen = bytes[offset]; + offset += Bytes.SIZEOF_BYTE; + + if(multiFamily){ + byte [] family = super.getFamily(); + ret = Bytes.compareTo(family, 0, family.length, bytes, offset, rowLen); + if(ret <= -1){ + return super.NEXT_SF; + } else if(ret >= 1){ + return super.NEXT_KV; + } + } + offset += famLen; + + //Getting column length + int colLen = super.KEY_OFFSET + keyLen - (offset-initialOffset) - + Bytes.SIZEOF_LONG - Bytes.SIZEOF_BYTE; + //Could be switched to something like: + //initloffset + keylength - TIMESTAMP_TYPE_SIZE - loffset; + + //Checking TTL and TimeRange before column so that if the entry turns out + //to be a delete we can be sure that all the deletes in the delete list + //are valid deletes + int tsOffset = offset + colLen; + long ts = Bytes.toLong(bytes, tsOffset); + ret = super.checkTTL(ts); + if(ret == 0){ + return super.NEXT_KV; + } + + ret = super.get.getTimeRange().withinTimeRange(ts); + if(ret != 1){ + return super.NEXT_KV; + } +// if (LOG.isDebugEnabled()) { +// LOG.debug("t " + System.nanoTime()); +// } +// if (LOG.isDebugEnabled()) { +// LOG.debug("t " + System.nanoTime()); +// } + //Check if kv is a delete + if(super.isDelete(bytes, initialOffset, keyLen)){ + super.newDeletes.add(kv); + return super.NEXT_KV; + } + + //TODO + //ret = compareFilter(kv); + + //Check if this kv is deleted + if(!super.deletes.isEmpty()){ + ret = super.isDeleted(bytes, initialOffset, rowLen, famLen, + colLen, multiFamily); + if(ret != 0){ + return super.NEXT_KV; + } + } + + //Includes remove from getList, TODO change for other calls + counter--; + return super.ADD; + } + + private boolean isDone(){ + if(counter <= 0){ + return true; + } + return false; + } + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/SingleLinkedList.java b/src/java/org/apache/hadoop/hbase/regionserver/SingleLinkedList.java new file mode 100644 index 0000000..b7c14c9 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/SingleLinkedList.java @@ -0,0 +1,163 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; + + +public class SingleLinkedList implements List{ + protected int size = 0; + protected Entry head = new Entry(null, null); + protected Entry tail = new Entry(null, null); + + public SingleLinkedList(){} + + + public boolean add(E e){ + Entry newEntry = new Entry(e, null); + if(head.next == null){ + head.next = newEntry; + } + tail.next = newEntry; + tail = newEntry; + size++; + return true; + } + + public int size() { + return size; + } + + public void clear(){ + size = 0; + head = null; + tail = null; + } + + private static class Entry { + E element; + Entry next; + + Entry(E element, Entry next) { + this.element = element; + this.next = next; + } + } + + public Iterator iterator() { + return new ListItr(); + } + + private class ListItr implements Iterator { + private Entry curr = null; + private Entry prev = null; + + ListItr() { + curr = head; + } + + public boolean hasNext(){ + return curr.next != null; + } + + public E next(){ + prev = curr; + curr = curr.next; + return curr.element; + } + + //Since there is only one global prev, it is only possible to do one remove + //after a next, multiple removes after each other will not work + public void remove(){ + if(prev != null){ + prev.next = curr.next; + } + curr = prev; + } + + } + + + /** + * Not supported methods + */ + + public void add(int index, Object o) { + throw new UnsupportedOperationException(); + } + public boolean addAll(Collection c) { + throw new UnsupportedOperationException(); + } + public boolean addAll(int index, Collection c) { + throw new UnsupportedOperationException(); + } + + public boolean contains(Object o) { + throw new UnsupportedOperationException(); + } + public boolean containsAll(Collection c) { + throw new UnsupportedOperationException(); + } + + public boolean equals(Object o) { + throw new UnsupportedOperationException(); + } + + public E get(int index) { + throw new UnsupportedOperationException(); + } + + public boolean hashcode() { + throw new UnsupportedOperationException(); + } + + public int indexOf(Object o) { + throw new UnsupportedOperationException(); + } + + public boolean isEmpty() { + throw new UnsupportedOperationException(); + } + + public int lastIndexOf(Object o) { + throw new UnsupportedOperationException(); + } + + public ListIterator listIterator() { + throw new UnsupportedOperationException(); + } + public ListIterator listIterator(int index) { + throw new UnsupportedOperationException(); + } + + public E remove(int index) { + throw new UnsupportedOperationException(); + } + public boolean remove(Object o) { + throw new UnsupportedOperationException(); + } + public boolean removeAll(Collection c) { + throw new UnsupportedOperationException(); + } + + public boolean retainAll(Collection c) { + throw new UnsupportedOperationException(); + } + + public Object set(int index, Object o) { + throw new UnsupportedOperationException(); + } + + public List subList(int fromIndex, int toIndex) { + throw new UnsupportedOperationException(); + } + + public Object[] toArray() { + throw new UnsupportedOperationException(); + } + public Object[] toArray(Object[] os) { + throw new UnsupportedOperationException(); + } + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/Store.java b/src/java/org/apache/hadoop/hbase/regionserver/Store.java index 1b4d2f7..a3d248c 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -222,6 +222,19 @@ public class Store implements HConstants { } /** +<<<<<<< HEAD:src/java/org/apache/hadoop/hbase/regionserver/Store.java + * @param storename + * @return Return hash of store name; can be used as key in map. + */ + static Integer getStoreNameMapKey(final byte [] storename) { + // If index < -1, presume passed column is a family name absent colon + // delimiter + return Bytes.mapKey(storename, storename.length); + } + + /** +======= +>>>>>>> hbase/trunk:src/java/org/apache/hadoop/hbase/regionserver/Store.java * @param tabledir * @param encodedName Encoded region name. * @param family @@ -392,6 +405,23 @@ public class Store implements HConstants { } } + + /** + * Adds a value to the memcache + * + * @param kv + * @return memcache size delta + */ + protected long newAdd(final KeyValue kv) { + lock.readLock().lock(); + try { +// System.out.println("Store " + System.nanoTime()); + return this.memcache.newAdd(kv, family.getMultiFamily()); + } finally { + lock.readLock().unlock(); + } + } + /** * @return All store files. */ @@ -1130,6 +1160,99 @@ public class Store implements HConstants { return true; } + + /** + * Return all the available columns for the given key. The key indicates a + * row and timestamp, but not a column name. + * + * The returned object should map column names to Cells. + */ + void newget(ServerGet sget, List result) + throws IOException { + this.lock.readLock().lock(); + sget.setTTL(ttl); + sget.setNow(); + int retCode = 0; + + // get from the memcache first. + boolean multiFamily = family.getMultiFamily(); + retCode = this.memcache.newget(sget, result, multiFamily); + + if(retCode == 1){ + return; + } + try { + for(Map.Entry entry : + this.storefiles.descendingMap().entrySet()){ + sget.mergeGets(); + sget.clear(); + retCode = newgetFromStoreFile(entry.getValue(), sget, result, + multiFamily); + if(retCode == 1){ + break; + } + } + } finally { + this.lock.readLock().unlock(); + } + } + + /** + * + * @param sf + * @param sget + * @param result + * @return + * @throws IOException + */ + private int newgetFromStoreFile(StoreFile sf, ServerGet sget, + List result, boolean multiFamily) + throws IOException { + HFileScanner scanner = sf.getReader().getScanner(); + + //TODO check how efficient the seekTo is and add an extra method for + //building the the search key, keep it simple for now + int retCode = 0; + boolean ret = false; + byte[] bs = new KeyValue(sget.getRow(), "".getBytes()).getBuffer(); + retCode = scanner.seekTo( + new KeyValue(sget.getRow(), "".getBytes()).getBuffer()); + if(retCode == -1){ + scanner.seekTo(); + } + + KeyValue kv = null; + // The cases that we need at this level: + //0 next + //1 include + //2 next store + //3 done + do{ + kv = scanner.getKeyValue(); + //Should add a setting the the family descriptor that lets you know + //if there are multiple families in this store + retCode = sget.compareTo(kv, multiFamily); + switch(retCode) { + //Do not include in result, look at next kv + case 0: break; + + //Include in result + case 1: result.add(kv); break; + + //Go to next storefile + case 2: return 0; + + //Done, early out + case 3: return 1; + + default : throw new IOException( + "Internal error from store, retCode = " + retCode); + } + } while(scanner.next()); + + return 0; + } + /* * @param wantedVersions How many versions were asked for. * @return wantedVersions or this families' VERSIONS. diff --git a/src/test/org/apache/hadoop/hbase/HBaseTestCase.java b/src/test/org/apache/hadoop/hbase/HBaseTestCase.java index 3abe340..bf35b74 100644 --- a/src/test/org/apache/hadoop/hbase/HBaseTestCase.java +++ b/src/test/org/apache/hadoop/hbase/HBaseTestCase.java @@ -33,10 +33,12 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.KeyValue; 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.Get; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; @@ -439,6 +441,11 @@ public abstract class HBaseTestCase extends TestCase { public void flushcache() throws IOException { this.region.flushcache(); } + + public void newget(Get get, List result, + final Integer lockid) throws IOException { + region.newget(get, result, lockid); + } } /** diff --git a/src/test/org/apache/hadoop/hbase/TestKeyValue.java b/src/test/org/apache/hadoop/hbase/TestKeyValue.java index 7271d4d..a5cb7ae 100644 --- a/src/test/org/apache/hadoop/hbase/TestKeyValue.java +++ b/src/test/org/apache/hadoop/hbase/TestKeyValue.java @@ -35,6 +35,41 @@ import org.apache.hadoop.hbase.util.Bytes; public class TestKeyValue extends TestCase { private final Log LOG = LogFactory.getLog(this.getClass().getName()); + KeyValue kv1 = null; + KeyValue kv2 = null; + KeyValue kv3 = null; + KeyValue kv4 = null; + KeyValue kv5 = null; + KeyValue kv6 = null; + KeyValue kv7 = null; + KeyValue kv8 = null; + + byte[] row1 = "row1".getBytes(); + byte[] row2 = "row2".getBytes(); + byte[] fam1 = "fam1".getBytes(); + byte[] fam2 = "fam2".getBytes(); + byte[] col1 = "col1".getBytes(); + byte[] col2 = "col2".getBytes(); + byte[] val1 = "val1".getBytes(); + byte[] val2 = "val2".getBytes(); + long ts1 = System.nanoTime(); + long ts2 = System.nanoTime(); + + KeyValue.Type type1 = KeyValue.Type.Put; + KeyValue.Type type2 = KeyValue.Type.Delete; + + protected void setUp() throws Exception { + super.setUp(); + kv1 = new KeyValue(row1, fam1, col1, ts1, type1, val1); + kv2 = new KeyValue(row1, fam1, col1, ts1, type1, val1); + kv3 = new KeyValue(row2, fam1, col1, ts1, type1, val1); + kv4 = new KeyValue(row1, fam2, col1, ts1, type1, val1); + kv5 = new KeyValue(row1, fam1, col2, ts1, type1, val1); + kv6 = new KeyValue(row1, fam1, col1, ts2, type1, val1); + kv7 = new KeyValue(row1, fam1, col1, ts1, type2, val1); + kv8 = new KeyValue(row1, fam1, col1, ts1, type1, val2); + } + public void testBasics() throws Exception { LOG.info("LOWKEY: " + KeyValue.LOWESTKEY.toString()); check(Bytes.toBytes(getName()), @@ -247,4 +282,73 @@ public class TestKeyValue extends TestCase { assertTrue(count++ == k.getTimestamp()); } } + + public void testEquals(){ + //Does not compare the value, but only the Key part of the KeyValue + assertTrue(kv1.equals(kv2)); + + assertFalse(kv1.equals(kv3)); + + assertFalse(kv1.equals(kv4)); + + assertFalse(kv1.equals(kv5)); + + assertFalse(kv1.equals(kv6)); + + assertFalse(kv1.equals(kv7)); + + assertTrue(kv1.equals(kv8)); + } + + public void testKeyComparator(){ + internalTestComparator(new KeyValue.KeyComparator()); + } + + public void testMetaComparator(){ + long now = System.nanoTime(); + KeyValue kv1 = new KeyValue(".META.,users,0", now); + KeyValue kv2 = new KeyValue(".META.,items,0", now); + KeyValue kv3 = new KeyValue(".META.,stories,0", now); + +// internalTestComparator(new KeyValue.MetaKeyComparator()); + } + + public void testRootComparator(){} + + private void internalTestComparator(KeyValue.KeyComparator comp){ + int res = 0; + + res = compare(comp, kv1.getBuffer(), kv2.getBuffer()); + assertEquals(0, res); + + res = compare(comp, kv1.getBuffer(), kv3.getBuffer()); + assertTrue(res <= -1); + + res = compare(comp, kv1.getBuffer(), kv4.getBuffer()); + assertTrue(res <= -1); + + res = compare(comp, kv1.getBuffer(), kv5.getBuffer()); + assertTrue(res <= -1); + + res = compare(comp, kv1.getBuffer(), kv6.getBuffer()); + assertTrue(res >= 1); + + res = compare(comp, kv1.getBuffer(), kv7.getBuffer()); + assertTrue(res >= 1); + + res = compare(comp, kv1.getBuffer(), kv8.getBuffer()); + assertEquals(0, res); + + res = compare(comp, kv3.getBuffer(), kv1.getBuffer()); + assertTrue(res >= 1); + } + + + private int compare(KeyValue.KeyComparator comp, byte[] left, byte[] right){ + int llength = Bytes.toInt(left); + int rlength = Bytes.toInt(right); + int offset = Bytes.SIZEOF_INT + Bytes.SIZEOF_INT; + return comp.compare(left, offset, llength, right, offset, rlength); + } + } \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/TestSerialization.java b/src/test/org/apache/hadoop/hbase/TestSerialization.java index 94e09a4..271a576 100644 --- a/src/test/org/apache/hadoop/hbase/TestSerialization.java +++ b/src/test/org/apache/hadoop/hbase/TestSerialization.java @@ -20,11 +20,20 @@ package org.apache.hadoop.hbase; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + import org.apache.hadoop.hbase.io.BatchOperation; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.Family; +import org.apache.hadoop.hbase.io.Get; +import org.apache.hadoop.hbase.io.GetColumns; import org.apache.hadoop.hbase.io.HbaseMapWritable; import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.io.RowUpdates; +import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; @@ -162,4 +171,118 @@ public class TestSerialization extends HBaseTestCase { } assertEquals(firstCount, secondCount); } + + + public void testFamily() throws Exception { + byte [] famName = getName().getBytes(); + List cols = new ArrayList(); + cols.add((getName()+"1").getBytes()); + cols.add((getName()+"2").getBytes()); + cols.add((getName()+"3").getBytes()); + Family fam = new Family(famName, cols); + + byte [] mb = Writables.getBytes(fam); + Family deserializedFam = + (Family)Writables.getWritable(mb, new Family()); + assertTrue(Bytes.equals(fam.getFamily(), deserializedFam.getFamily())); + cols = fam.getColumns(); + List deserializedCols = deserializedFam.getColumns(); + assertEquals(cols.size(), deserializedCols.size()); + + for(int i=0; i fams = get.getFamilies(); + List deserializedFams = deserializedGet.getFamilies(); + assertEquals(fams.size(), deserializedFams.size()); + for(int i=0; i cols = fams.get(i).getColumns(); + List deserializedCols = deserializedFams.get(i).getColumns(); + assertEquals(cols.size(), deserializedCols.size()); + + for(int j=0; j rupsFams = rups.getFamilies(); + List deserializedRupsFams = deserializedRups.getFamilies(); + assertEquals(rupsFams.size(), deserializedRupsFams.size()); + for(int i=0; i rupsCols = rupsFams.get(i).getColumns(); + List deserializedRupsCols = + deserializedRupsFams.get(i).getColumns(); + assertEquals(rupsCols.size(), deserializedRupsCols.size()); + for(int j=0; j columns = null; + + + protected void setUp() + throws Exception{ + super.setUp(); + + HColumnDescriptor colDes = + new HColumnDescriptor((familyName+':').getBytes()); + HBaseAdmin admin = new HBaseAdmin(conf); + HTableDescriptor testTableADesc = + new HTableDescriptor(tableAname); + testTableADesc.addFamily(colDes); +// testTableADesc.addFamily(column2); + admin.createTable(testTableADesc); + + table = new HTable(conf, tableAname); + + columns = new ArrayList(); + for(int i=0; i columns = new ArrayList(); +// for(int i=0; i)null)); + } + + private void substrFilterTests(RowFilterInterface filter) + throws Exception { + assertTrue("substrTrue", filter.filterColumn(ROW, COLUMN, FULLSTRING_1)); + assertFalse("substrFalse", filter.filterColumn(ROW, COLUMN, FULLSTRING_2)); + assertFalse("substrFilterAllRemaining", filter.filterAllRemaining()); + assertFalse("substrFilterNotNull", filter.filterRow((List)null)); + } + + private void regexFilterTests(RowFilterInterface filter) + throws Exception { + assertTrue("regexTrue", filter.filterColumn(ROW, COLUMN, FULLSTRING_1)); + assertFalse("regexFalse", filter.filterColumn(ROW, COLUMN, FULLSTRING_2)); + assertFalse("regexFilterAllRemaining", filter.filterAllRemaining()); + assertFalse("regexFilterNotNull", filter.filterRow((List)null)); + } + + private RowFilterInterface serializationTest(RowFilterInterface filter) + throws Exception { + // Decompose filter to bytes. + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(stream); + filter.write(out); + out.close(); + byte[] buffer = stream.toByteArray(); + + // Recompose filter. + DataInputStream in = + new DataInputStream(new ByteArrayInputStream(buffer)); + RowFilterInterface newFilter = new ColumnValueFilter(); + newFilter.readFields(in); + + return newFilter; + } + + RowFilterInterface basicFilter; + RowFilterInterface substrFilter; + RowFilterInterface regexFilter; + + @Override + protected void setUp() throws Exception { + super.setUp(); + basicFilter = basicFilterNew(); + substrFilter = substrFilterNew(); + regexFilter = regexFilterNew(); + } + + /** + * Tests identification of the stop row + * @throws Exception + */ + public void testStop() throws Exception { + basicFilterTests(basicFilter); + substrFilterTests(substrFilter); + regexFilterTests(regexFilter); + } + + /** + * Tests serialization + * @throws Exception + */ + public void testSerialization() throws Exception { + RowFilterInterface newFilter = serializationTest(basicFilter); + basicFilterTests(newFilter); + newFilter = serializationTest(substrFilter); + substrFilterTests(newFilter); + newFilter = serializationTest(regexFilter); + regexFilterTests(newFilter); + } + +} diff --git a/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java new file mode 100644 index 0000000..a4d07c2 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java @@ -0,0 +1,94 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.filter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + +import junit.framework.TestCase; + +/** + * Tests the inclusive stop row filter + */ +public class TestInclusiveStopRowFilter extends TestCase { + private final byte [] STOP_ROW = Bytes.toBytes("stop_row"); + private final byte [] GOOD_ROW = Bytes.toBytes("good_row"); + private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz"); + + RowFilterInterface mainFilter; + + @Override + protected void setUp() throws Exception { + super.setUp(); + mainFilter = new InclusiveStopRowFilter(STOP_ROW); + } + + /** + * Tests identification of the stop row + * @throws Exception + */ + public void testStopRowIdentification() throws Exception { + stopRowTests(mainFilter); + } + + /** + * Tests serialization + * @throws Exception + */ + public void testSerialization() throws Exception { + // Decompose mainFilter to bytes. + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(stream); + mainFilter.write(out); + out.close(); + byte[] buffer = stream.toByteArray(); + + // Recompose mainFilter. + DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer)); + RowFilterInterface newFilter = new InclusiveStopRowFilter(); + newFilter.readFields(in); + + // Ensure the serialization preserved the filter by running a full test. + stopRowTests(newFilter); + } + + private void stopRowTests(RowFilterInterface filter) throws Exception { + assertFalse("Filtering on " + Bytes.toString(GOOD_ROW), filter.filterRowKey(GOOD_ROW)); + assertFalse("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)); + assertFalse("Filtering on " + Bytes.toString(STOP_ROW), filter.filterColumn(STOP_ROW, null, null)); + assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW), filter.filterColumn(PAST_STOP_ROW, + null, null)); + + assertFalse("FilterAllRemaining", filter.filterAllRemaining()); + assertFalse("FilterNotNull", filter.filterRow((List)null)); + + assertFalse("Filter a null", filter.filterRowKey(null)); + } +} diff --git a/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java new file mode 100644 index 0000000..bc7abfd --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java @@ -0,0 +1,98 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.filter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; + +import org.apache.hadoop.hbase.util.Bytes; + + +import junit.framework.TestCase; + +/** + * Tests for the page row filter + */ +public class TestPageRowFilter extends TestCase { + + RowFilterInterface mainFilter; + static final int ROW_LIMIT = 3; + + @Override + protected void setUp() throws Exception { + super.setUp(); + mainFilter = new PageRowFilter(ROW_LIMIT); + } + + /** + * test page size filter + * @throws Exception + */ + public void testPageSize() throws Exception { + pageSizeTests(mainFilter); + } + + /** + * Test filter serialization + * @throws Exception + */ + public void testSerialization() throws Exception { + // Decompose mainFilter to bytes. + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(stream); + mainFilter.write(out); + out.close(); + byte[] buffer = stream.toByteArray(); + + // Recompose mainFilter. + DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer)); + RowFilterInterface newFilter = new PageRowFilter(); + newFilter.readFields(in); + + // Ensure the serialization preserved the filter by running a full test. + pageSizeTests(newFilter); + } + + private void pageSizeTests(RowFilterInterface filter) throws Exception { + testFiltersBeyondPageSize(filter, ROW_LIMIT); + // Test reset works by going in again. + filter.reset(); + testFiltersBeyondPageSize(filter, ROW_LIMIT); + } + + private void testFiltersBeyondPageSize(final RowFilterInterface filter, + final int pageSize) { + for (int i = 0; i < (pageSize * 2); i++) { + byte [] row = Bytes.toBytes(Integer.toString(i)); + boolean filterOut = filter.filterRowKey(row); + if (!filterOut) { + assertFalse("Disagrees with 'filter'", filter.filterAllRemaining()); + } else { + // Once we have all for a page, calls to filterAllRemaining should + // stay true. + assertTrue("Disagrees with 'filter'", filter.filterAllRemaining()); + assertTrue(i >= pageSize); + } + filter.rowProcessed(filterOut, row); + } + } +} diff --git a/src/test/org/apache/hadoop/hbase/filter/TestPrefixRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestPrefixRowFilter.java new file mode 100644 index 0000000..11f0704 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestPrefixRowFilter.java @@ -0,0 +1,99 @@ +/** + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.filter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.UnsupportedEncodingException; + +import junit.framework.TestCase; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Tests for a prefix row filter + */ +public class TestPrefixRowFilter extends TestCase { + RowFilterInterface mainFilter; + static final char FIRST_CHAR = 'a'; + static final char LAST_CHAR = 'e'; + static final String HOST_PREFIX = "org.apache.site-"; + static byte [] GOOD_BYTES = null; + + static { + try { + GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING); + } catch (UnsupportedEncodingException e) { + fail(); + } + } + + @Override + protected void setUp() throws Exception { + super.setUp(); + this.mainFilter = new PrefixRowFilter(Bytes.toBytes(HOST_PREFIX)); + } + + /** + * Tests filtering using a regex on the row key + * @throws Exception + */ + public void testPrefixOnRow() throws Exception { + prefixRowTests(mainFilter); + } + + /** + * Test serialization + * @throws Exception + */ + public void testSerialization() throws Exception { + // Decompose mainFilter to bytes. + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(stream); + mainFilter.write(out); + out.close(); + byte[] buffer = stream.toByteArray(); + + // Recompose filter. + DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer)); + RowFilterInterface newFilter = new PrefixRowFilter(); + newFilter.readFields(in); + + // Ensure the serialization preserved the filter by running all test. + prefixRowTests(newFilter); + } + + private void prefixRowTests(RowFilterInterface filter) throws Exception { + for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) { + byte [] t = createRow(c); + assertFalse("Failed with characer " + c, filter.filterRowKey(t)); + } + String yahooSite = "com.yahoo.www"; + assertTrue("Failed with character " + + yahooSite, filter.filterRowKey(Bytes.toBytes(yahooSite))); + } + + private byte [] createRow(final char c) { + return Bytes.toBytes(HOST_PREFIX + Character.toString(c)); + } +} diff --git a/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java new file mode 100644 index 0000000..cb1845c --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java @@ -0,0 +1,200 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.filter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.UnsupportedEncodingException; +import java.util.Map; +import java.util.TreeMap; + +import junit.framework.TestCase; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.regionserver.HLogEdit; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Tests for regular expression row filter + */ +public class TestRegExpRowFilter extends TestCase { + TreeMap colvalues; + RowFilterInterface mainFilter; + static final char FIRST_CHAR = 'a'; + static final char LAST_CHAR = 'e'; + static final String HOST_PREFIX = "org.apache.site-"; + static byte [] GOOD_BYTES = null; + + static { + try { + GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING); + } catch (UnsupportedEncodingException e) { + fail(); + } + } + + @Override + protected void setUp() throws Exception { + super.setUp(); + this.colvalues = new TreeMap(Bytes.BYTES_COMPARATOR); + for (char c = FIRST_CHAR; c < LAST_CHAR; c++) { + colvalues.put(Bytes.toBytes(new String(new char [] {c})), + new Cell(GOOD_BYTES, HConstants.LATEST_TIMESTAMP)); + } + this.mainFilter = new RegExpRowFilter(HOST_PREFIX + ".*", colvalues); + } + + /** + * Tests filtering using a regex on the row key + * @throws Exception + */ + public void testRegexOnRow() throws Exception { + regexRowTests(mainFilter); + } + + /** + * Tests filtering using a regex on row and colum + * @throws Exception + */ + public void testRegexOnRowAndColumn() throws Exception { + regexRowColumnTests(mainFilter); + } + + /** + * Only return values that are not null + * @throws Exception + */ + public void testFilterNotNull() throws Exception { + filterNotNullTests(mainFilter); + } + + /** + * Test serialization + * @throws Exception + */ + public void testSerialization() throws Exception { + // Decompose mainFilter to bytes. + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(stream); + mainFilter.write(out); + out.close(); + byte[] buffer = stream.toByteArray(); + + // Recompose filter. + DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer)); + RowFilterInterface newFilter = new RegExpRowFilter(); + newFilter.readFields(in); + + // Ensure the serialization preserved the filter by running all test. + regexRowTests(newFilter); + newFilter.reset(); + regexRowColumnTests(newFilter); + newFilter.reset(); + filterNotNullTests(newFilter); + } + + private void regexRowTests(RowFilterInterface filter) throws Exception { + for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) { + byte [] t = createRow(c); + assertFalse("Failed with characer " + c, filter.filterRowKey(t)); + } + String yahooSite = "com.yahoo.www"; + assertTrue("Failed with character " + + yahooSite, filter.filterRowKey(Bytes.toBytes(yahooSite))); + } + + private void regexRowColumnTests(RowFilterInterface filter) + throws UnsupportedEncodingException { + + for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) { + byte [] t = createRow(c); + for (Map.Entry e: this.colvalues.entrySet()) { + assertFalse("Failed on " + c, + filter.filterColumn(t, e.getKey(), e.getValue().getValue())); + } + } + // Try a row and column I know will pass. + char c = 'c'; + byte [] r = createRow(c); + byte [] col = Bytes.toBytes(Character.toString(c)); + assertFalse("Failed with character " + c, + filter.filterColumn(r, col, GOOD_BYTES)); + + // Do same but with bad bytes. + assertTrue("Failed with character " + c, + filter.filterColumn(r, col, "badbytes".getBytes(HConstants.UTF8_ENCODING))); + + // Do with good bytes but bad column name. Should not filter out. + assertFalse("Failed with character " + c, + filter.filterColumn(r, Bytes.toBytes("badcolumn"), GOOD_BYTES)); + + // Good column, good bytes but bad row. + assertTrue("Failed with character " + c, + filter.filterColumn(Bytes.toBytes("bad row"), + Bytes.toBytes("badcolumn"), GOOD_BYTES)); + } + + private void filterNotNullTests(RowFilterInterface filter) throws Exception { + // Modify the filter to expect certain columns to be null: + // Expecting a row WITH columnKeys: a-d, WITHOUT columnKey: e + ((RegExpRowFilter)filter).setColumnFilter(new byte [] {LAST_CHAR}, null); + + char secondToLast = (char)(LAST_CHAR - 1); + char thirdToLast = (char)(LAST_CHAR - 2); + + // Modify the row to be missing an expected columnKey (d) + colvalues.remove(new byte [] {(byte)secondToLast}); + + // Try a row that is missing an expected columnKey. + // Testing row with columnKeys: a-c + assertTrue("Failed with last columnKey " + thirdToLast, filter. + filterRow(colvalues)); + + // Try a row that has all expected columnKeys, and NO null-expected + // columnKeys. + // Testing row with columnKeys: a-d + colvalues.put(new byte [] {(byte)secondToLast}, + new Cell(GOOD_BYTES, HConstants.LATEST_TIMESTAMP)); + assertFalse("Failed with last columnKey " + secondToLast, filter. + filterRow(colvalues)); + + // Try a row that has all expected columnKeys AND a null-expected columnKey. + // Testing row with columnKeys: a-e + colvalues.put(new byte [] {LAST_CHAR}, + new Cell(GOOD_BYTES, HConstants.LATEST_TIMESTAMP)); + assertTrue("Failed with last columnKey " + LAST_CHAR, filter. + filterRow(colvalues)); + + // Try a row that has all expected columnKeys and a null-expected columnKey + // that maps to a null value. + // Testing row with columnKeys: a-e, e maps to null + colvalues.put(new byte [] {LAST_CHAR}, + new Cell(HLogEdit.DELETED_BYTES, HConstants.LATEST_TIMESTAMP)); + assertFalse("Failed with last columnKey " + LAST_CHAR + " mapping to null.", + filter.filterRow(colvalues)); + } + + private byte [] createRow(final char c) { + return Bytes.toBytes(HOST_PREFIX + Character.toString(c)); + } +} diff --git a/src/test/org/apache/hadoop/hbase/filter/TestRowFilterAfterWrite.java b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterAfterWrite.java new file mode 100644 index 0000000..f2669a8 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterAfterWrite.java @@ -0,0 +1,201 @@ +/** + * Copyright 2008 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.filter; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import junit.framework.Assert; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseClusterTestCase; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Scanner; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; + +/** Test regexp filters HBASE-476 */ +public class TestRowFilterAfterWrite extends HBaseClusterTestCase { + + private static final Log LOG = LogFactory.getLog(TestRowFilterAfterWrite.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 TestRowFilterAfterWrite() { + 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 columnMap = new HashMap(); + 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 e : result.entrySet()) { + LOG.info(" column: " + e.getKey() + " value: " + + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); + } + } + count++; + } + + } finally { + scanner.close(); + } + return count; + } + } +} diff --git a/src/test/org/apache/hadoop/hbase/filter/TestRowFilterOnMultipleFamilies.java b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterOnMultipleFamilies.java new file mode 100644 index 0000000..91af8b2 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterOnMultipleFamilies.java @@ -0,0 +1,128 @@ +/** + * Copyright 2008 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.filter; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import junit.framework.Assert; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseClusterTestCase; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Scanner; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Test for regexp filters (HBASE-527) + */ +public class TestRowFilterOnMultipleFamilies extends HBaseClusterTestCase { + private static final Log LOG = LogFactory.getLog(TestRowFilterOnMultipleFamilies.class.getName()); + + static final String TABLE_NAME = "TestTable"; + static final String COLUMN1 = "A:col1"; + static final byte [] TEXT_COLUMN1 = Bytes.toBytes(COLUMN1); + static final String COLUMN2 = "B:col2"; + static final byte [] TEXT_COLUMN2 = Bytes.toBytes(COLUMN2); + + private static final byte [][] columns = {TEXT_COLUMN1, TEXT_COLUMN2}; + + private static final int NUM_ROWS = 10; + private static final byte[] VALUE = "HELLO".getBytes(); + + /** @throws IOException */ + public void testMultipleFamilies() throws IOException { + HTableDescriptor desc = new HTableDescriptor(TABLE_NAME); + desc.addFamily(new HColumnDescriptor("A:")); + desc.addFamily(new HColumnDescriptor("B:")); + + // Create a table. + HBaseAdmin admin = new HBaseAdmin(this.conf); + admin.createTable(desc); + + // insert some data into the test table + HTable table = new HTable(conf, TABLE_NAME); + + for (int i = 0; i < NUM_ROWS; i++) { + BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i)); + b.put(TEXT_COLUMN1, VALUE); + b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes()); + table.commit(b); + } + + LOG.info("Print table contents using scanner before map/reduce for " + TABLE_NAME); + scanTable(TABLE_NAME, true); + LOG.info("Print table contents using scanner+filter before map/reduce for " + TABLE_NAME); + scanTableWithRowFilter(TABLE_NAME, true); + } + + private void scanTable(final String tableName, final boolean printValues) throws IOException { + HTable table = new HTable(conf, tableName); + Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); + int numFound = doScan(scanner, printValues); + Assert.assertEquals(NUM_ROWS, numFound); + } + + private void scanTableWithRowFilter(final String tableName, final boolean printValues) throws IOException { + HTable table = new HTable(conf, tableName); + Map columnMap = new HashMap(); + 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 e : result.entrySet()) { + LOG.info(" column: " + e.getKey() + " value: " + + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); + } + } + Assert.assertEquals(2, result.size()); + count++; + } + + } finally { + scanner.close(); + } + return count; + } + } +} diff --git a/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java new file mode 100644 index 0000000..e5821ba --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java @@ -0,0 +1,188 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.filter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.UnsupportedEncodingException; +import java.util.HashSet; +import java.util.Set; +import java.util.TreeMap; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.util.Bytes; + + +import junit.framework.TestCase; + +/** + * Tests filter sets + */ +public class TestRowFilterSet extends TestCase { + + RowFilterInterface filterMPALL; + RowFilterInterface filterMPONE; + static final int MAX_PAGES = 5; + static final char FIRST_CHAR = 'a'; + static final char LAST_CHAR = 'e'; + TreeMap colvalues; + static byte[] GOOD_BYTES = null; + static byte[] BAD_BYTES = null; + + static { + try { + GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING); + BAD_BYTES = "def".getBytes(HConstants.UTF8_ENCODING); + } catch (UnsupportedEncodingException e) { + fail(); + } + } + + @Override + protected void setUp() throws Exception { + super.setUp(); + + colvalues = new TreeMap(Bytes.BYTES_COMPARATOR); + for (char c = FIRST_CHAR; c < LAST_CHAR; c++) { + colvalues.put(new byte [] {(byte)c}, + new Cell(GOOD_BYTES, HConstants.LATEST_TIMESTAMP)); + } + + Set filters = new HashSet(); + filters.add(new PageRowFilter(MAX_PAGES)); + filters.add(new RegExpRowFilter(".*regex.*", colvalues)); + filters.add(new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("yyy")))); + filters.add(new WhileMatchRowFilter(new RegExpRowFilter(".*match.*"))); + filterMPALL = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL, + filters); + filterMPONE = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ONE, + filters); + } + + /** + * Test "must pass one" + * @throws Exception + */ + public void testMPONE() throws Exception { + MPONETests(filterMPONE); + } + + /** + * Test "must pass all" + * @throws Exception + */ + public void testMPALL() throws Exception { + MPALLTests(filterMPALL); + } + + /** + * Test serialization + * @throws Exception + */ + public void testSerialization() throws Exception { + // Decompose filterMPALL to bytes. + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(stream); + filterMPALL.write(out); + out.close(); + byte[] buffer = stream.toByteArray(); + + // Recompose filterMPALL. + DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer)); + RowFilterInterface newFilter = new RowFilterSet(); + newFilter.readFields(in); + + // Ensure the serialization preserved the filter by running a full test. + MPALLTests(newFilter); + } + + private void MPONETests(RowFilterInterface filter) throws Exception { + // A row that shouldn't cause any filters to return true. + RFSAssertion(filter, "regex_match", false); + + // A row that should cause the WhileMatchRowFilter to filter all remaining. + RFSAssertion(filter, "regex_only", false); + + // Make sure the overall filterAllRemaining is unchanged (correct for + // MUST_PASS_ONE). + assertFalse(filter.filterAllRemaining()); + + // A row that should cause the RegExpRowFilter to fail and the + // StopRowFilter to filter all remaining. + RFSAssertion(filter, "yyy_match", false); + + // Accept several more rows such that PageRowFilter will exceed its limit. + for (int i=0; i<=MAX_PAGES-3; i++) + filter.rowProcessed(false, Bytes.toBytes("unimportant_key")); + + // A row that should cause the RegExpRowFilter to filter this row, making + // all the filters return true and thus the RowFilterSet as well. + RFSAssertion(filter, "bad_column", true); + + // Make sure the overall filterAllRemaining is unchanged (correct for + // MUST_PASS_ONE). + assertFalse(filter.filterAllRemaining()); + } + + private void MPALLTests(RowFilterInterface filter) throws Exception { + // A row that shouldn't cause any filters to return true. + RFSAssertion(filter, "regex_match", false); + + // A row that should cause WhileMatchRowFilter to filter all remaining. + RFSAssertion(filter, "regex_only", true); + + // Make sure the overall filterAllRemaining is changed (correct for + // MUST_PASS_ALL). + RFSAssertReset(filter); + + // A row that should cause the RegExpRowFilter to fail and the + // StopRowFilter to filter all remaining. + RFSAssertion(filter, "yyy_match", true); + + // Make sure the overall filterAllRemaining is changed (correct for + // MUST_PASS_ALL). + RFSAssertReset(filter); + + // A row that should cause the RegExpRowFilter to fail. + boolean filtered = filter.filterColumn(Bytes.toBytes("regex_match"), + new byte [] { FIRST_CHAR }, BAD_BYTES); + assertTrue("Filtering on 'regex_match' and bad column data.", filtered); + filterMPALL.rowProcessed(filtered, Bytes.toBytes("regex_match")); + } + + private void RFSAssertion(RowFilterInterface filter, String toTest, + boolean assertTrue) throws Exception { + byte [] testText = Bytes.toBytes(toTest); + boolean filtered = filter.filterRowKey(testText); + assertTrue("Filtering on '" + toTest + "'", + assertTrue? filtered : !filtered); + filter.rowProcessed(filtered, testText); + } + + private void RFSAssertReset(RowFilterInterface filter) throws Exception{ + assertTrue(filter.filterAllRemaining()); + // Reset for continued testing + filter.reset(); + assertFalse(filter.filterAllRemaining()); + } +} diff --git a/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java new file mode 100644 index 0000000..5addd03 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java @@ -0,0 +1,94 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.filter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + +import junit.framework.TestCase; + +/** + * Tests the stop row filter + */ +public class TestStopRowFilter extends TestCase { + private final byte [] STOP_ROW = Bytes.toBytes("stop_row"); + private final byte [] GOOD_ROW = Bytes.toBytes("good_row"); + private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz"); + + RowFilterInterface mainFilter; + + @Override + protected void setUp() throws Exception { + super.setUp(); + mainFilter = new StopRowFilter(STOP_ROW); + } + + /** + * Tests identification of the stop row + * @throws Exception + */ + public void testStopRowIdentification() throws Exception { + stopRowTests(mainFilter); + } + + /** + * Tests serialization + * @throws Exception + */ + public void testSerialization() throws Exception { + // Decompose mainFilter to bytes. + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(stream); + mainFilter.write(out); + out.close(); + byte[] buffer = stream.toByteArray(); + + // Recompose mainFilter. + DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer)); + RowFilterInterface newFilter = new StopRowFilter(); + newFilter.readFields(in); + + // Ensure the serialization preserved the filter by running a full test. + stopRowTests(newFilter); + } + + private void stopRowTests(RowFilterInterface filter) throws Exception { + assertFalse("Filtering on " + Bytes.toString(GOOD_ROW), filter.filterRowKey(GOOD_ROW)); + assertTrue("Filtering on " + Bytes.toString(STOP_ROW), filter.filterRowKey(STOP_ROW)); + assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW), filter.filterRowKey(PAST_STOP_ROW)); + + assertFalse("Filtering on " + Bytes.toString(GOOD_ROW), filter.filterColumn(GOOD_ROW, null, + null)); + assertTrue("Filtering on " + Bytes.toString(STOP_ROW), filter.filterColumn(STOP_ROW, null, null)); + assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW), filter.filterColumn(PAST_STOP_ROW, + null, null)); + + assertFalse("FilterAllRemaining", filter.filterAllRemaining()); + assertFalse("FilterNotNull", filter.filterRow((List)null)); + + assertFalse("Filter a null", filter.filterRowKey(null)); + } +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java new file mode 100644 index 0000000..81986ef --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java @@ -0,0 +1,151 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.filter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + +import junit.framework.TestCase; + + +/** + * Tests for the while-match filter + */ +public class TestWhileMatchRowFilter extends TestCase { + + WhileMatchRowFilter wmStopRowFilter; + WhileMatchRowFilter wmRegExpRowFilter; + + @Override + protected void setUp() throws Exception { + super.setUp(); + wmStopRowFilter = new WhileMatchRowFilter(new StopRowFilter( + Bytes.toBytes("s"))); + wmRegExpRowFilter = new WhileMatchRowFilter(new RegExpRowFilter( + ".*regex.*")); + } + + /** + * Tests while match stop row + * @throws Exception + */ + public void testWhileMatchStopRow() throws Exception { + whileMatchStopRowTests(wmStopRowFilter); + } + + /** + * Tests while match regex + * @throws Exception + */ + public void testWhileMatchRegExp() throws Exception { + whileMatchRegExpTests(wmRegExpRowFilter); + } + + /** + * Tests serialization + * @throws Exception + */ + public void testSerialization() throws Exception { + // Decompose wmRegExpRowFilter to bytes. + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(stream); + wmRegExpRowFilter.write(out); + out.close(); + byte[] buffer = stream.toByteArray(); + + // Recompose wmRegExpRowFilter. + DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer)); + WhileMatchRowFilter newFilter = new WhileMatchRowFilter(); + newFilter.readFields(in); + + // Ensure the serialization preserved the filter by running a full test. + whileMatchRegExpTests(newFilter); + } + + private void whileMatchStopRowTests(WhileMatchRowFilter filter) throws + Exception { + RowFilterInterface innerFilter = filter.getInternalFilter(); + String toTest; + + // Test cases that should pass the row + toTest = "apples"; + assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest))); + assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes( + toTest))); + + // Test cases that should fail the row + toTest = "tuna"; + assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest))); + assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes( + toTest))); + + // The difference in switch + assertTrue("filter: filterAllRemaining", filter.filterAllRemaining()); + assertFalse("innerFilter: filterAllRemaining pre-reset", + innerFilter.filterAllRemaining()); + + // Test resetting + filter.reset(); + assertFalse("filter: filterAllRemaining post-reset", + filter.filterAllRemaining()); + + // Test filterNotNull for functionality only (no switch-cases) + assertFalse("filter: filterNotNull", filter.filterRow((List)null)); + } + + private void whileMatchRegExpTests(WhileMatchRowFilter filter) throws + Exception { + RowFilterInterface innerFilter = filter.getInternalFilter(); + String toTest; + + // Test cases that should pass the row + toTest = "regex_match"; + assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest))); + assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes( + toTest))); + + // Test cases that should fail the row + toTest = "not_a_match"; + assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest))); + assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes( + toTest))); + + // The difference in switch + assertTrue("filter: filterAllRemaining", filter.filterAllRemaining()); + assertFalse("innerFilter: filterAllRemaining pre-reset", + innerFilter.filterAllRemaining()); + + // Test resetting + filter.reset(); + assertFalse("filter: filterAllRemaining post-reset", + filter.filterAllRemaining()); + + // Test filter(Text, Text, byte[]) for functionality only (no switch-cases) + toTest = "asdf_regex_hjkl"; + assertFalse("filter: '" + toTest + "'", filter.filterColumn(Bytes.toBytes(toTest), + null, null)); + } +} diff --git a/src/test/org/apache/hadoop/hbase/io/TestFamily.java b/src/test/org/apache/hadoop/hbase/io/TestFamily.java new file mode 100644 index 0000000..5bd3046 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/io/TestFamily.java @@ -0,0 +1,64 @@ +package org.apache.hadoop.hbase.io; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + + +import junit.framework.TestCase; + +public class TestFamily extends TestCase { + private final boolean PRINT = false; + + private byte[] row = "row1".getBytes(); + private long ts = System.currentTimeMillis(); + private byte[] familyName = "fam1".getBytes(); + private byte[] col1 = "col1".getBytes(); + private byte[] col2 = "col2".getBytes(); + private byte[] col3 = "col3".getBytes(); + + List columns = new ArrayList(); + Family family = null; + + + protected void setUp() throws Exception{ + super.setUp(); + + columns.add(col2); + columns.add(col1); + columns.add(col3); + + family = new Family(familyName, columns); + } + + public void testSorted(){ + if(PRINT){ + System.out.println(family.toString()); + } + family.sortColumns(); + List sortedCols = family.getColumns(); + for(int i=0; i results = null; + + TimeRange tr = null; + + @Override + protected void setUp() throws Exception { + super.setUp(); + // this.miniHdfs = new MiniDFSCluster(this.conf, 1, true, null); + // // Set the hbase.rootdir to be the home directory in mini dfs. + // this.conf.set(HConstants.HBASE_DIR, + // this.miniHdfs.getFileSystem().getHomeDirectory().toString()); + createRegion(); + tr = new TimeRange(); + } + + protected void tearDown(){ + closeRegion(); + } + + // This test does not include any test of the actual client call and keeping + // old way of inserting things into HBase + public void testGetColumns() + throws IOException{ + results = new ArrayList(); + + oldPut(); + KeyValue oldKv = new KeyValue(row, column2, val); + // Create Get object + Get get = new GetColumns(row, getFam, col2, (short)1, tr); + if(PRINT) System.out.println("get " + get); + + //TODO see why this loops forever, something with the lock in + //updateStorefiles this.lock.writeLock().lock(); + //Testing getting from memcache +// results = region.newget(get, results, null); +// System.out.println("got result with size " + results.size()); +// if (results.size() > 0) { +// KeyValue res = results.get(0); +// int ret = Bytes.compareTo(oldKv.getBuffer(), oldKv.getOffset(), oldKv +// .getKeyLength() - 9, res.getBuffer(), res.getOffset(), res +// .getKeyLength() - 9); +// assertEquals(0, ret); +// } + + flush(); + oldPut(); + + get = new GetColumns(row, getFam, col2, (short)2, tr); + results = region.newget(get, results, null); + if(PRINT) System.out.println("got result with size " + results.size()); + + if (results.size() > 0) { + KeyValue res = results.get(0); + int ret = Bytes.compareTo(oldKv.getBuffer(), oldKv.getOffset(), oldKv + .getKeyLength() - 9, res.getBuffer(), res.getOffset(), res + .getKeyLength() - 9); + assertEquals(0, ret); + } + + flush(); + oldPut(); + + // Testing getting from memcache and storeFile 2 versions + comparing timers + // to see which one is faster + int v2Fetch = 3; + results = new ArrayList(); + List columns = new ArrayList(2); + columns.add(col1); + columns.add(col3); + columns.add(col5); + get = new GetColumns(row, getFam, columns, (short)v2Fetch, tr); + + long start = 0L; + long stop = 0L; + + start = System.nanoTime(); + results = region.newget(get, results, null); + stop = System.nanoTime(); + if(PRINT) System.out.println("GetColumns"); + if(PRINT) System.out.println("new timer " +(stop-start)); + int newVersions = results.size(); + if (results.size() > 0) { + KeyValue res = results.get(0); + int ret = Bytes.compareTo(oldKv.getBuffer(), oldKv.getOffset(), oldKv + .getKeyLength() - 9, res.getBuffer(), res.getOffset(), res + .getKeyLength() - 9); + assertEquals(0, ret); + } + + + //Old way of getting data + Map oldRes = null; + byte [] row = get.getRow(); + NavigableSet cols = + new ConcurrentSkipListSet(Bytes.BYTES_COMPARATOR); + cols.add(column1); + cols.add(column3); + cols.add(column5); + oldRes = region.getFull(row, cols, LATEST_TIMESTAMP, v2Fetch, null); + oldRes = null; + start = System.nanoTime(); + oldRes = region.getFull(row, cols, LATEST_TIMESTAMP, v2Fetch, null); + stop = System.nanoTime(); + if(PRINT) System.out.println("old timer " +(stop-start)); + int oldVersions = 0; + for(Map.Entry entry : oldRes.entrySet()){ + oldVersions += entry.getValue().getNumValues(); + } + assertEquals(oldVersions, newVersions); + } + + + public void testGetFamilies() + throws IOException { + long start = 0L; + long stop = 0L; + + results = new ArrayList(); + oldPut(); + flush(); + + Get get = new GetFamilies(row, getFam, (short)1, tr); + region.newget(get, results, null); + + oldPut(); + flush(); + + results.clear(); + + int v2Fetch = 2; + get = new GetFamilies(row, getFam, (short)v2Fetch, tr); + start = System.nanoTime(); + region.newget(get, results, null); + stop = System.nanoTime(); + if(PRINT) System.out.println("GetFamilies"); + if(PRINT) System.out.println("new timer " +(stop-start)); + + NavigableSet cols = + new ConcurrentSkipListSet(Bytes.BYTES_COMPARATOR); + cols.add(fam); + start = System.nanoTime(); + Map oldRes = region.getFull(row, cols, LATEST_TIMESTAMP, + v2Fetch, null); + stop = System.nanoTime(); + if(PRINT) System.out.println("old timer " +(stop-start)); + int oldVersions = 0; + for(Map.Entry entry : oldRes.entrySet()){ + oldVersions += entry.getValue().getNumValues(); + } + assertEquals(oldVersions, results.size()); + } + + public void testGetTop() + throws IOException { + results = new ArrayList(); + long start = 0L; + long stop = 0L; + int nrToFetch = 5; + + + oldPut(); + + flush(); + oldPut(); + Get get = new GetTop(row, fam, nrToFetch, tr); + region.newget(get, results, null); + + flush(); + oldPut(); + + nrToFetch = 5; + if(PRINT) System.out.println("Trying to fetch " +nrToFetch+ + " KeyValues"); + results = new ArrayList(); + get = new GetTop(row, fam, nrToFetch, tr); + + start = System.nanoTime(); + region.newget(get, results, null); + stop = System.nanoTime(); + if(PRINT) System.out.println("GetTop"); + if(PRINT) System.out.println("new timer " +(stop-start)); + if(PRINT) System.out.println("result size " +results.size()); + assertEquals(nrToFetch, results.size()); + + } + + + private void oldPut() + throws IOException{ + BatchUpdate batchUpdate = new BatchUpdate(row); + batchUpdate.put(column1, val); + batchUpdate.put(column2, val); + batchUpdate.put(column3, val); + batchUpdate.put(column4, val); + batchUpdate.put(column5, val); + region.batchUpdate(batchUpdate, null); + } + + private void createRegion(){ + try { + HTableDescriptor htd = new HTableDescriptor(getName()); + htd.addFamily(new HColumnDescriptor(fam)); + // FileSystem filesystem = FileSystem.get(conf); + // Path rootdir = filesystem.makeQualified( + // new Path(conf.get(HConstants.HBASE_DIR))); + // filesystem.mkdirs(rootdir); + + region = createNewHRegion(htd, null, null); + } catch(Exception e){} + } + + private void flush() + throws IOException{ + // flush + region.flushcache(); + } + + private void closeRegion(){ + if (region != null) { + try { + region.close(); + region.getLog().closeAndDelete(); + } catch (Exception e) { + e.printStackTrace(); + } + } + } + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestNewGet2.java b/src/test/org/apache/hadoop/hbase/regionserver/TestNewGet2.java new file mode 100644 index 0000000..094f9ad --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestNewGet2.java @@ -0,0 +1,198 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseTestCase; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.HBaseTestCase.HRegionIncommon; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.Get; +import org.apache.hadoop.hbase.io.GetColumns; +import org.apache.hadoop.hbase.io.GetFamilies; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.hdfs.MiniDFSCluster; + +import junit.framework.TestCase; + +public class TestNewGet2 extends HBaseTestCase { + static final Log LOG = LogFactory.getLog(TestNewGet2.class); + private MiniDFSCluster cluster = null; + private HRegion region = null; + + + private static final byte [] CONTENTS = Bytes.toBytes("contents:"); + private static final byte [] ROW_KEY = + HRegionInfo.ROOT_REGIONINFO.getRegionName(); + private static final String SERVER_ADDRESS = "foo.bar.com:1234"; + + + @Override + public void setUp() throws Exception { + try { + this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null); + // Set the hbase.rootdir to be the home directory in mini dfs. + this.conf.set(HConstants.HBASE_DIR, + this.cluster.getFileSystem().getHomeDirectory().toString()); + } catch (IOException e) { + shutdownDfs(cluster); + } + super.setUp(); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + shutdownDfs(cluster); + // ReflectionUtils.printThreadInfo(new PrintWriter(System.out), + // "Temporary end-of-test thread dump debugging HADOOP-2040: " + getName()); + } + + public void testGet() throws IOException { + try { + + HTableDescriptor desc = new HTableDescriptor("test"); + desc.addFamily(new HColumnDescriptor(CONTENTS)); + desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY)); + + region = createNewHRegion(desc, null, null); + HRegionIncommon r = new HRegionIncommon(region); + + // Write information to the table + BatchUpdate batchUpdate = null; + batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis()); + batchUpdate.put(CONTENTS, CONTENTS); + batchUpdate.put(HConstants.COL_REGIONINFO, + Writables.getBytes(HRegionInfo.ROOT_REGIONINFO)); + r.commit(batchUpdate); + + batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis()); + batchUpdate.put(HConstants.COL_SERVER, + Bytes.toBytes(new HServerAddress(SERVER_ADDRESS).toString())); + batchUpdate.put(HConstants.COL_STARTCODE, Bytes.toBytes(12345)); + batchUpdate.put(Bytes.toString(HConstants.COLUMN_FAMILY) + + "region", Bytes.toBytes("region")); + r.commit(batchUpdate); + + // Verify that get works the same from memcache as when reading from disk + // NOTE dumpRegion won't work here because it only reads from disk. + + verifyNewGet(r, SERVER_ADDRESS); + +// verifyGet(r, SERVER_ADDRESS); +// +// // Close and re-open region, forcing updates to disk +// +// region.close(); +// region = openClosedRegion(region); +// r = new HRegionIncommon(region); +// +// // Read it back +// +// verifyGet(r, SERVER_ADDRESS); +// +// // Update one family member and add a new one +// +// batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis()); +// batchUpdate.put(Bytes.toString(HConstants.COLUMN_FAMILY) + "region", +// "region2".getBytes(HConstants.UTF8_ENCODING)); +// String otherServerName = "bar.foo.com:4321"; +// batchUpdate.put(HConstants.COL_SERVER, +// Bytes.toBytes(new HServerAddress(otherServerName).toString())); +// batchUpdate.put(Bytes.toString(HConstants.COLUMN_FAMILY) + "junk", +// "junk".getBytes(HConstants.UTF8_ENCODING)); +// r.commit(batchUpdate); +// +// verifyGet(r, otherServerName); +// +// // Close region and re-open it +// +// region.close(); +// region = openClosedRegion(region); +// r = new HRegionIncommon(region); +// +// // Read it back +// +// verifyGet(r, otherServerName); + + } finally { + if (region != null) { + // Close region once and for all + region.close(); + region.getLog().closeAndDelete(); + } + } + } + + private void verifyGet(final HRegionIncommon r, final String expectedServer) + throws IOException { + // This should return a value because there is only one family member + Cell value = r.get(ROW_KEY, CONTENTS); + assertNotNull(value); + + // This should not return a value because there are multiple family members + value = r.get(ROW_KEY, HConstants.COLUMN_FAMILY); + assertNull(value); + + // Find out what getFull returns + Map values = r.getFull(ROW_KEY); + + // assertEquals(4, values.keySet().size()); + for (Map.Entry entry : values.entrySet()) { + byte[] column = entry.getKey(); + Cell cell = entry.getValue(); + if (Bytes.equals(column, HConstants.COL_SERVER)) { + String server = Writables.cellToString(cell); + assertEquals(expectedServer, server); + LOG.info(server); + } + } + } + + private void verifyNewGet(final HRegionIncommon r, final String expectedServer) + throws IOException { + // This should return a value because there is only one family member +// Get get = new GetColumns(rows[i], family, column, (byte)1, null); + Get get = new GetFamilies(ROW_KEY, CONTENTS, (byte)1); + List result = new ArrayList(); + Integer lock = null; + r.newget(get, result, lock); + System.out.println("Result"); + for(KeyValue kv : result){ + System.out.println("kv " +kv); + } + +// Cell value = r.get(ROW_KEY, CONTENTS); +// assertNotNull(value); +// +// // This should not return a value because there are multiple family members +// value = r.get(ROW_KEY, HConstants.COLUMN_FAMILY); +// assertNull(value); +// +// // Find out what getFull returns +// Map values = r.getFull(ROW_KEY); +// +// // assertEquals(4, values.keySet().size()); +// for (Map.Entry entry : values.entrySet()) { +// byte[] column = entry.getKey(); +// Cell cell = entry.getValue(); +// if (Bytes.equals(column, HConstants.COL_SERVER)) { +// String server = Writables.cellToString(cell); +// assertEquals(expectedServer, server); +// LOG.info(server); +// } +// } + } + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestNewMemcache.java b/src/test/org/apache/hadoop/hbase/regionserver/TestNewMemcache.java new file mode 100644 index 0000000..477fe43 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestNewMemcache.java @@ -0,0 +1,237 @@ +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + +import junit.framework.TestCase; +import java.util.Comparator; + +public class TestNewMemcache extends TestCase { + private final boolean PRINT = false; + + private Memcache memcache; + + private boolean multiFamily; + private KeyValue.KVComparator kvComparator; + + private byte [] putRow1 = "row1".getBytes(); + private byte [] putRow2 = "row2".getBytes(); + private byte [] putFam1 = "fam1".getBytes(); + private byte [] putFam2 = "fam2".getBytes(); + private byte [] putCol1 = "col1".getBytes(); + private byte [] putCol2 = "col2".getBytes(); + + private byte [] putVal1 = "val1".getBytes(); + private long putTs = 0L; + + private KeyValue put1; + private KeyValue put2; + private KeyValue put3; + private KeyValue put4; + private KeyValue del1; + private KeyValue delCol1; + private KeyValue delFam1; + + + @Override + public void setUp() throws Exception { + super.setUp(); + this.kvComparator = KeyValue.COMPARATOR; + this.memcache = new Memcache(HConstants.FOREVER, kvComparator); + this.multiFamily = false; + this.putTs = System.nanoTime(); + + this.put1 = new KeyValue(putRow1, putFam1, putCol1, putTs, + KeyValue.Type.Put, putVal1); + this.put2 = new KeyValue(putRow1, putFam2, putCol2, putTs, + KeyValue.Type.Put, putVal1); + this.put3 = new KeyValue(putRow2, putFam1, putCol1, putTs, + KeyValue.Type.Put, putVal1); + + this.del1 = new KeyValue(putRow1, putFam1, putCol1, putTs, + KeyValue.Type.Delete, putVal1); + + this.putTs = System.nanoTime(); + this.delCol1 = new KeyValue(putRow1, putFam1, putCol1, putTs, + KeyValue.Type.DeleteColumn, putVal1); + this.put4 = new KeyValue(putRow2, putFam1, putCol1, putTs, + KeyValue.Type.Put, putVal1); + + + this.putTs = System.nanoTime(); + this.delFam1 = new KeyValue(putRow1, putFam1, putCol1, putTs, + KeyValue.Type.DeleteFamily, putVal1); + } + + public void testComparator(){ + int ret = 0; + ret = kvComparator.compare(put2, put3); + if(PRINT){ + System.out.println("ret " +ret); + } + assertTrue(ret <= -1); + + //check timestamp diff + ret = kvComparator.compare(put3, put4); + if(PRINT){ + System.out.println("ret " +ret); + } + assertTrue(ret >= 1); + + //check type diff + ret = kvComparator.compare(put1, del1); + if(PRINT){ + System.out.println("ret " +ret); + } + assertTrue(ret >= 1); + } + + public void testAdd(){ + memcache.newAdd(put1, multiFamily); + + KeyValue kv = memcache.memcache.first(); + int ret = kvComparator.compare(put1, kv); + assertEquals(0, ret); + } + + public void testMultiAdd(){ + memcache.newAdd(put1, multiFamily); + assertEquals(1, memcache.memcache.size()); + + memcache.newAdd(put1, multiFamily); + assertEquals(1, memcache.memcache.size()); + + + memcache.newAdd(put2, multiFamily); + assertEquals(2, memcache.memcache.size()); + + memcache.newAdd(put3, multiFamily); + assertEquals(3, memcache.memcache.size()); + + memcache.newAdd(put4, multiFamily); + assertEquals(4, memcache.memcache.size()); + + if(PRINT){ + System.out.println(new Exception().getStackTrace()[0].getMethodName()); + printMemCache(); + System.out.println(); + } + + } + + + + public void testAddAndDelete(){ + //Adding put + memcache.newAdd(put1, multiFamily); + + //Adding delete for the same ts + memcache.newAdd(del1, multiFamily); + + + if(PRINT){ + System.out.println(new Exception().getStackTrace()[0].getMethodName()); + printMemCache(); + System.out.println(); + } + assertEquals(0, memcache.memcache.size()); + } + + public void testAddAndDeleteColumn(){ + //Adding put + memcache.newAdd(put1, multiFamily); + + //Adding deletecolumn + memcache.newAdd(delCol1, multiFamily); + assertEquals(1, memcache.memcache.size()); + + if(PRINT){ + System.out.println(new Exception().getStackTrace()[0].getMethodName()); + printMemCache(); + System.out.println(); + } + } + + public void testAddAndDeleteFamily(){ + //Adding put + memcache.newAdd(put1, multiFamily); + + //Adding deleteFamily + memcache.newAdd(delFam1, multiFamily); + + assertEquals(1, memcache.memcache.size()); + + if(PRINT){ + System.out.println(new Exception().getStackTrace()[0].getMethodName()); + printMemCache(); + System.out.println(); + } + } + + public void testAddAndMultiDelete(){ + //Adding put + memcache.newAdd(put1, multiFamily); + assertEquals(1, memcache.memcache.size()); + + //Adding delete for the same ts + memcache.newAdd(del1, multiFamily); + assertEquals(0, memcache.memcache.size()); + + //Adding delete for the same ts + memcache.newAdd(delCol1, multiFamily); + assertEquals(1, memcache.memcache.size()); + + //Adding delete for the same ts + memcache.newAdd(delFam1, multiFamily); + assertEquals(1, memcache.memcache.size()); + + if(PRINT){ + System.out.println(new Exception().getStackTrace()[0].getMethodName()); + printMemCache(); + System.out.println(); + } + } + + + public void testAddAddAddMultiDeleteAdd(){ + //Adding put with a different row + memcache.newAdd(put3, multiFamily); + assertEquals(1, memcache.memcache.size()); + + //Adding put with a different family + memcache.newAdd(put2, multiFamily); + assertEquals(2, memcache.memcache.size()); + + //Adding put + memcache.newAdd(put1, multiFamily); + assertEquals(3, memcache.memcache.size()); + + if(PRINT){ + printMemCache(); + } + //Adding delete for the same ts + memcache.newAdd(del1, multiFamily); + assertEquals(2, memcache.memcache.size()); + + //Adding delete for the same ts + memcache.newAdd(delCol1, multiFamily); + assertEquals(3, memcache.memcache.size()); + + //Adding delete for the same ts + memcache.newAdd(delFam1, multiFamily); + assertEquals(2, memcache.memcache.size()); + + if(PRINT){ + System.out.println(new Exception().getStackTrace()[0].getMethodName()); + printMemCache(); + System.out.println(); + } + } + + private void printMemCache(){ + for(KeyValue keyvalue : memcache.memcache){ + System.out.println("keyValue " +keyvalue); + } + } +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestNewPut.java b/src/test/org/apache/hadoop/hbase/regionserver/TestNewPut.java new file mode 100644 index 0000000..ce92363 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestNewPut.java @@ -0,0 +1,114 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.HBaseTestCase; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Family; +import org.apache.hadoop.hbase.io.RowUpdates; + +import junit.framework.TestCase; + +public class TestNewPut extends HBaseTestCase implements HConstants { + + private static byte[] row = "row1".getBytes(); + private static byte[] fam = "fam1:".getBytes(); + private static byte[] col1 = "col1".getBytes(); + private static byte[] col2 = "col2".getBytes(); + private static byte[] col3 = "col3".getBytes(); + private static byte[] col4 = "col4".getBytes(); + private static byte[] col5 = "col5".getBytes(); + + private static byte[] val1 = "val1".getBytes(); + private static byte[] val2 = "val2".getBytes(); + private static byte[] val3 = "val3".getBytes(); + private static byte[] val4 = "val4".getBytes(); + private static byte[] val5 = "val5".getBytes(); + + private static byte[] column1 = "fam1:col1".getBytes(); + private static byte[] column2 = "fam1:col2".getBytes(); + private static byte[] column3 = "fam1:col3".getBytes(); + private static byte[] column4 = "fam1:col4".getBytes(); + private static byte[] column5 = "fam1:col5".getBytes(); + + private static List columns = null; + + private HRegion region = null; + private RowUpdates updates = null; + + Integer lockid = null; + +// boolean writeToWAL = true; + boolean writeToWAL = false; + Family family = null; + + long ts = 0L; + + @Override + protected void setUp() throws Exception { + super.setUp(); + HTableDescriptor htd = new HTableDescriptor(getName()); + htd.addFamily(new HColumnDescriptor(fam)); + + region = createNewHRegion(htd, null, null); + +// columns = new ArrayList(); +// columns.add(col1); +// columns.add(col2); +// columns.add(col3); +// columns.add(col4); +// columns.add(col5); + family = new Family(fam, col1, val1); + family.add(col2, val2); + family.add(col3, val3); + family.add(col4, val4); + family.add(col5, val5); + + ts = System.currentTimeMillis(); + updates = new RowUpdates(row, family, ts); + updates.createKeyValuesFromColumns(); +// lockid = -1; + } + + public void testPutRowUpdate() throws IOException { + //Create a RowUpdate + long start = 0L; + long stop = 0L; + start = System.nanoTime(); + + System.out.println("start " +start); + region.updateRow(updates, lockid, writeToWAL); + stop = System.nanoTime(); + + System.out.println("stop " +stop); + + System.out.println("timer " +(stop-start)); + + System.out.println("\n"); + +// store-mem + + BatchUpdate bu = new BatchUpdate(row); + bu.put(column1, val1); + bu.put(column2, val2); + bu.put(column3, val3); + bu.put(column4, val4); + bu.put(column5, val5); + + start = System.nanoTime(); + System.out.println("start " +start); + region.batchUpdate(bu, lockid, writeToWAL); + stop = System.nanoTime(); + System.out.println("stop " +stop); + System.out.println("timer " +(stop-start)); +// region.updateRow(RowUpdates rups, Integer lockid, boolean writeToWAL); +// region.newUpdate(byte [] family, List bss, boolean writeToWAL) + + } + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestServerGetColumns.java b/src/test/org/apache/hadoop/hbase/regionserver/TestServerGetColumns.java new file mode 100644 index 0000000..3496579 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestServerGetColumns.java @@ -0,0 +1,675 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.Get; +import org.apache.hadoop.hbase.io.GetColumns; +import org.apache.hadoop.hbase.io.Family; +import org.apache.hadoop.hbase.io.TimeRange; + + +import junit.framework.TestCase; + +public class TestServerGetColumns extends TestCase { + + private final boolean PRINT = false; + byte [] row1 = null; + byte [] fam1 = null; + byte [] col1 = null; + byte [] val1 = null; + + byte [] row2 = null; + byte [] fam2 = null; + byte [] col2 = null; + byte [] val2 = null; + + byte [] fam3 = null; + byte [] col3 = null; + byte [] val3 = null; + + byte [] fam4 = null; + byte [] col4 = null; + byte [] val4 = null; + + List families1 = null; + long ts = 0L; + long ts1 = 0L; + long ts2 = 0L; + long ts3 = 0L; + short versionsToFetch = 0; + + KeyValue putKv1 = null; + KeyValue putKv2 = null; + KeyValue putKv3 = null; + KeyValue putKv4 = null; + KeyValue putKv5 = null; + + KeyValue delKv1 = null; + KeyValue delKv2 = null; + KeyValue delKv21 = null; + KeyValue delKv3 = null; + KeyValue delKv4 = null; + KeyValue delKv41 = null; + KeyValue delKv5 = null; + + Get get = null; + ServerGet sget = null; + + TimeRange tr = null; + + protected void setUp() throws Exception { + super.setUp(); + row1 = "row1".getBytes(); + fam1 = "fam1".getBytes(); + col1 = "col1".getBytes(); + val1 = "val1".getBytes(); + + row2 = "row2".getBytes(); + fam2 = "fam2".getBytes(); + col2 = "col2".getBytes(); + val2 = "val2".getBytes(); + + fam3 = "fam3".getBytes(); + col3 = "col3".getBytes(); + val3 = "val3".getBytes(); + + fam4 = "fam4".getBytes(); + col4 = "col4".getBytes(); + val4 = "val4".getBytes(); + + families1 = new ArrayList(); + families1.add(new Family(fam1, col1)); + ts1 = System.nanoTime(); + versionsToFetch = 1; + + putKv1 = new KeyValue(row1, fam1, col1, ts1, KeyValue.Type.Put, val1); + putKv2 = new KeyValue(row1, fam1, col2, ts1, KeyValue.Type.Put, val2); + putKv3 = new KeyValue(row2, fam1, col2, ts1, KeyValue.Type.Put, val2); + + delKv1 = new KeyValue(row1, fam1, col1, ts1, KeyValue.Type.Delete, val1); + delKv2 = new KeyValue(row1, fam1, col2, ts1, KeyValue.Type.Delete, val1); + delKv21 = new KeyValue(row1, fam1, col2, ts1, KeyValue.Type.DeleteColumn, val1); + + + ts2 = System.nanoTime(); + putKv4 = new KeyValue(row1, fam1, col1, ts2, KeyValue.Type.Put, val1); + delKv3 = new KeyValue(row1, fam1, col1, ts2, KeyValue.Type.Delete, val1); + delKv4 = new KeyValue(row1, fam1, col2, ts1, KeyValue.Type.DeleteColumn, val1); + delKv41 = new KeyValue(row1, fam1, col2, ts2, KeyValue.Type.DeleteColumn, val1); + + ts3 = System.nanoTime(); + putKv5 = new KeyValue(row1, fam1, col1, ts3, KeyValue.Type.Put, val1); + delKv5 = new KeyValue(row1, fam1, col3, ts3, KeyValue.Type.Delete, val1); + + Get get = new GetColumns(row1, families1, versionsToFetch, ts1); + sget = new ServerGetColumns(get); + sget.setFamily(families1.get(0).getFamily()); + sget.setColumns(families1.get(0).getColumns()); + sget.setNow(System.nanoTime()); + sget.setTTL(-1L); + } + + + private String toString(byte[] bytes) { + String ret = ""; + for(int i=0;i families = new ArrayList(); + families.add(family); + + get = new GetColumns(row1, families, versionsToFetch, ts1); + sget = new ServerGetColumns(get); + sget.setFamily(families.get(0).getFamily()); + sget.setColumns(families.get(0).getColumns()); + sget.setNow(System.nanoTime()); + sget.setTTL(-1L); + + //Create a KeyValue object + List kvs = new ArrayList(); + kvs.add(putKv1); + kvs.add(putKv2); + + Iterator iter = kvs.iterator(); + //Compare + int ret = 0; + while(iter.hasNext()){ + ret = sget.compareTo(iter.next(), false); + if(PRINT){ + if(ret == 1){ + System.out.println("adding to result"); + } else if(ret == 2){ + System.out.println("skipping to next store"); + } else if(ret == 3){ + System.out.println("done!!"); + } + System.out.println("fetching next value from store\n"); + } + } + assertEquals(0, ret); + } + + public void testCompare_deleteList() + throws IOException{ + //Create a ServerGet object + byte [] col2 = "col2".getBytes(); + byte [] col3 = "col3".getBytes(); + + Family family = new Family(fam1, col1); + family.add(col2); + family.add(col3); + + List families = new ArrayList(); + families.add(family); + + get = new GetColumns(row1, families, versionsToFetch, ts1); + sget = new ServerGetColumns(get); + sget.setFamily(families.get(0).getFamily()); + sget.setColumns(families.get(0).getColumns()); + sget.setNow(System.nanoTime()); + sget.setTTL(-1L); + + //Create a KeyValue object + List kvs = new ArrayList(); + kvs.add(putKv1); + kvs.add(putKv2); + + //Creating a deleteList + List deletes = new ArrayList(); + deletes.add(delKv1); + + //Adding deleteList to serverGet + sget.setDeletes(deletes); + + Iterator iter = kvs.iterator(); + //Compare + int ret = 0; + int found = 0; + while(iter.hasNext()){ + ret = sget.compareTo(iter.next(), false); + if(PRINT){ + if(ret == 1){ + System.out.println("adding to result"); + } else if(ret == 2){ + System.out.println("skipping to next store"); + } else if(ret == 3){ + System.out.println("done!!"); + } + System.out.println("fetching next value from store\n"); + } + found += ret; + } + if(PRINT) System.out.println("found " +found); + assertEquals(1, found); + + } + + public void testNextStoreFile() + throws IOException{ + //Create a ServerGet object + byte [] col2 = "col2".getBytes(); + byte [] col3 = "col3".getBytes(); + + Family family = new Family(fam1, col1); + family.add(col2); + family.add(col3); + + List families = new ArrayList(); + families.add(family); + + get = new GetColumns(row1, families, versionsToFetch, ts); + sget = new ServerGetColumns(get); + sget.setFamily(families.get(0).getFamily()); + sget.setColumns(families.get(0).getColumns()); + + //Create a KeyValue object + List kvs = new ArrayList(); + kvs.add(putKv1); + kvs.add(putKv3); + + Iterator iter = kvs.iterator(); + //Compare + int ret = 0; + while(iter.hasNext()){ + ret = sget.compareTo(iter.next(), false); + if(PRINT){ + if(ret == 1){ + System.out.println("adding to result"); + } else if(ret == 2){ + System.out.println("skipping to next store"); + break; + } else if(ret == 3){ + System.out.println("done!!"); + break; + } + System.out.println("fetching next value from store\n"); + } + } + if(PRINT){ + System.out.println("ret " +ret); + } + assertEquals(2, ret); + } + + + + public void testDone() + throws IOException{ + //Create a KeyValue object + List kvs = new ArrayList(); + kvs.add(putKv1); + kvs.add(putKv2); + kvs.add(putKv3); + + Iterator iter = kvs.iterator(); + //Compare + int ret = 0; + while(iter.hasNext()){ + ret = sget.compareTo(iter.next(), false); + if(PRINT){ + if(ret == 1){ + System.out.println("adding to result"); + } else if(ret == 2){ + System.out.println("skipping to next store"); + break; + } else if(ret == 3){ + System.out.println("done!!"); + break; + } + System.out.println("fetching next value from store\n"); + } + } + if(PRINT){ + System.out.println("ret " +ret); + } + assertEquals(3, ret); + } + + public void testMergeDeletes_Delete_Delete(){ + //Create delete keyLists to merge + List l1 = new ArrayList(); + List l2 = new ArrayList(); + + l1.add(delKv1); + if(PRINT){ + printList(l1); + } + + l2.add(delKv3); + if(PRINT){ + printList(l2); + } + + //merge lists + Deletes mergedDeletes = null; + mergedDeletes = sget.mergeDeletes(l1, l2); + + //check result + for(KeyValue key : mergedDeletes.getDeletes()){ + assertSame(key, delKv3); + if(PRINT) System.out.println("key " +key); + } + + //merge lists + if(PRINT) System.out.println(); + mergedDeletes = sget.mergeDeletes(l2, l1); + + //check result + for(KeyValue key : mergedDeletes.getDeletes()){ + assertSame(key, delKv3); + if(PRINT) System.out.println("key " +key); + } + } + + + public void testMergeDeletes_Delete_DeleteColumn(){ + //Create delete keyLists to merge + List l1 = new ArrayList(); + List l2 = new ArrayList(); + + l1.add(delKv2); + if(PRINT){ + printList(l1); + } + + l2.add(delKv4); + if(PRINT){ + printList(l2); + } + + //merge lists + Deletes mergedDeletes = null; + mergedDeletes = sget.mergeDeletes(l1, l2); + + //check result + for(KeyValue key : mergedDeletes.getDeletes()){ + assertSame(key, delKv4); + if(PRINT){ + System.out.println("key " +key); + } + } + + //merge lists + if(PRINT) System.out.println(); + mergedDeletes = sget.mergeDeletes(l2, l1); + + //check result + for(KeyValue key : mergedDeletes.getDeletes()){ + assertSame(key, delKv4); + if(PRINT){ + System.out.println("key " +key); + } + } + } + + + public void testMergeDeletes_DeleteColumn_DeleteColumn(){ + //Create delete keyLists to merge + List l1 = new ArrayList(); + List l2 = new ArrayList(); + + l1.add(delKv21); + if(PRINT){ + printList(l1); + } + + l2.add(delKv41); + if(PRINT){ + printList(l2); + } + + //merge lists + Deletes mergedDeletes = null; + mergedDeletes = sget.mergeDeletes(l1, l2); + + //check result + for(KeyValue key : mergedDeletes.getDeletes()){ + assertSame(key, delKv41); + if(PRINT) System.out.println("key " +key); + } + + //merge lists + if(PRINT) System.out.println(); + mergedDeletes = sget.mergeDeletes(l2, l1); + + //check result + for(KeyValue key : mergedDeletes.getDeletes()){ + assertSame(key, delKv41); + if(PRINT) System.out.println("key " +key); + } + } + + + public void testMergeDeletes(){ + //Create delete keyLists to merge + List l1 = new ArrayList(); + List l2 = new ArrayList(); + + KeyValue oldKey1 = delKv1; + KeyValue oldKey2 = delKv4; + l1.add(oldKey1); + l1.add(oldKey2); + if(PRINT){ + printList(l1); + } + + KeyValue newKey1 = delKv3; + KeyValue newKey2 = delKv2; + KeyValue newKey3 = delKv5; + l2.add(newKey1); + l2.add(newKey2); + l2.add(newKey3); + if(PRINT){ + printList(l2); + } + + KeyValue [] resultArr = new KeyValue[3]; + resultArr[0] = newKey1; + resultArr[1] = oldKey2; + resultArr[2] = newKey3; + + //merge lists + Deletes mergedDeletes = sget.mergeDeletes(l1, l2); + + //check result + int i = 0; + for(KeyValue key : mergedDeletes.getDeletes()){ + assertSame(key, resultArr[i++]); + if(PRINT) System.out.println("key " +key); + } + + } + + + public void testCompare_updateVersions() + throws IOException{ + versionsToFetch = 4; + ts = System.nanoTime(); + tr = new TimeRange(ts, ts1); + get = new GetColumns(row1, families1, versionsToFetch, tr); + sget = new ServerGetColumns(get); + sget.setFamily(families1.get(0).getFamily()); + sget.setColumns(families1.get(0).getColumns()); + sget.setNow(System.nanoTime()); + sget.setTTL(-1L); + + List kvs = new ArrayList(); + kvs.add(putKv5); + kvs.add(putKv4); + kvs.add(putKv1); + + Iterator iter = kvs.iterator(); + + //Compare + int ret = 0; + while(iter.hasNext()){ + ret = sget.compareTo(iter.next(), false); + if(PRINT) { + if(ret == 1){ + System.out.println("adding to result"); + } else if(ret == 2){ + System.out.println("skipping to next store"); + break; + } else if(ret == 3){ + System.out.println("done!!"); + break; + } + } + if(PRINT) System.out.println("fetching next value from store\n"); + } + + List versions = sget.getVersions(); + for(short version : versions){ + if(PRINT) System.out.println("versions fetched " +version); + assertEquals(3, version); + } + } + + public void testCompare_timeRange() + throws IOException{ + versionsToFetch = 4; + List kvs = new ArrayList(); + kvs.add(putKv5); + kvs.add(putKv4); + kvs.add(putKv1); + + Iterator iter = null; + + int ret = 0; + + List columns = null; + + int len = 0; + + //Getting 1 value + tr = new TimeRange(ts2, ts1); + get = new GetColumns(row1, families1, versionsToFetch, tr); + sget = new ServerGetColumns(get); + sget.setFamily(families1.get(0).getFamily()); + sget.setColumns(families1.get(0).getColumns()); + sget.setNow(System.nanoTime()); + sget.setTTL(-1L); + + iter = kvs.iterator(); + + //Compare + while(iter.hasNext()){ + ret = sget.compareTo(iter.next(), false); + if(PRINT) { + if(ret == 1){ + System.out.println("adding to result"); + } else if(ret == 2){ + System.out.println("skipping to next store"); + break; + } else if(ret == 3){ + System.out.println("done!!"); + break; + } + } + if(PRINT) System.out.println("fetching next value from store\n"); + } + + List versions = sget.getVersions(); + for(short version : versions){ + if(PRINT) System.out.println("versions fetched " +version); + assertEquals(1, version); + } + + //Getting 2 value + tr = new TimeRange(ts3, ts1); + Family family = new Family(fam1, col1); + families1 = new ArrayList(); + families1.add(family); +// families1 = new Family[]{new Family(fam1, col1)}; + + get = new GetColumns(row1, families1, versionsToFetch, tr); + sget = new ServerGetColumns(get); + sget.setFamily(families1.get(0).getFamily()); + sget.setColumns(families1.get(0).getColumns()); + sget.setNow(System.nanoTime()); + sget.setTTL(-1L); + + iter = kvs.iterator(); + + //Compare + while(iter.hasNext()){ + ret = sget.compareTo(iter.next(), false); + if(PRINT) { + if(ret == 1){ + System.out.println("adding to result"); + } else if(ret == 2){ + System.out.println("skipping to next store"); + break; + } else if(ret == 3){ + System.out.println("done!!"); + break; + } + } + if(PRINT) System.out.println("fetching next value from store\n"); + } + + versions = sget.getVersions(); + for(short version : versions){ + if(PRINT) System.out.println("versions fetched " +version); + assertEquals(2, version); + } + + + //Getting 3 value + ts = System.nanoTime(); + tr = new TimeRange(ts, ts1); +// families1 = new Family[]{new Family(fam1, col1)}; + get = new GetColumns(row1, families1, versionsToFetch, tr); + sget = new ServerGetColumns(get); + sget.setFamily(families1.get(0).getFamily()); + sget.setColumns(families1.get(0).getColumns()); + sget.setNow(System.nanoTime()); + sget.setTTL(-1L); + + iter = kvs.iterator(); + + //Compare + while(iter.hasNext()){ + ret = sget.compareTo(iter.next(), false); + if(PRINT) { + if(ret == 1){ + System.out.println("adding to result"); + } else if(ret == 2){ + System.out.println("skipping to next store"); + break; + } else if(ret == 3){ + System.out.println("done!!"); + break; + } + } + if(PRINT) System.out.println("fetching next value from store\n"); + } + + versions = sget.getVersions(); + for(short version : versions){ + if(PRINT) System.out.println("versions fetched " +version); + assertEquals(3, version); + } + } + + + private void printList(List list){ + for(Object o : list){ + System.out.println(o); + } + System.out.println(); + } + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestServerGetFamilies.java b/src/test/org/apache/hadoop/hbase/regionserver/TestServerGetFamilies.java new file mode 100644 index 0000000..d085244 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestServerGetFamilies.java @@ -0,0 +1,137 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.Get; +import org.apache.hadoop.hbase.io.GetFamilies; +import org.apache.hadoop.hbase.io.TimeRange; +import junit.framework.TestCase; + +public class TestServerGetFamilies extends TestCase { + + private final boolean PRINT = false; + + byte [] row1 = null; + byte [] fam1 = null; + byte [] col1 = null; + byte [] val1 = null; + + byte [] row2 = null; + byte [] fam2 = null; + byte [] col2 = null; + byte [] val2 = null; + + byte [] col3 = null; + byte [] col4 = null; + byte [] col5 = null; + byte [] col6 = null; + + + KeyValue putKv1 = null; + KeyValue putKv2 = null; + KeyValue putKv3 = null; + KeyValue putKv4 = null; + KeyValue putKv5 = null; + KeyValue putKv6 = null; + + short versionsToFetch = 0; + long ts = 0L; + + TimeRange tr = null; + Get get = null; + ServerGet sget = null; + + boolean multiFamily = false; + + protected void setUp() throws Exception { + super.setUp(); + ts = System.currentTimeMillis(); + + row1 = "row1".getBytes(); + fam1 = "fam1".getBytes(); + col1 = "col1".getBytes(); + val1 = "val1".getBytes(); + + row2 = "row2".getBytes(); + fam2 = "fam2".getBytes(); + col2 = "col2".getBytes(); + val2 = "val2".getBytes(); + + col3 = "col3".getBytes(); + col4 = "col4".getBytes(); + col5 = "col5".getBytes(); + col6 = "col6".getBytes(); + + putKv1 = new KeyValue(row1, fam1, col1, ts, KeyValue.Type.Put, val1); + putKv2 = new KeyValue(row1, fam1, col2, ts, KeyValue.Type.Put, val2); + putKv3 = new KeyValue(row1, fam1, col3, ts, KeyValue.Type.Put, val1); + putKv4 = new KeyValue(row1, fam1, col4, ts, KeyValue.Type.Put, val2); + putKv5 = new KeyValue(row1, fam1, col5, ts, KeyValue.Type.Put, val1); + putKv6 = new KeyValue(row2, fam1, col6, ts, KeyValue.Type.Put, val2); + + tr = new TimeRange(); + get = new GetFamilies(row1, fam1, versionsToFetch, tr); + + sget = new ServerGetFamilies(get); + sget.setFamily(fam1); + sget.setNow(System.nanoTime()); + sget.setTTL(-1L); + } + + + public void testCompareColumn() + throws IOException{ + List list = new ArrayList(); + list.add(putKv1); + list.add(putKv2); + + int res = 0; + for(int i=0; i< list.size(); i++){ + res = sget.compareTo(list.get(i), multiFamily); + assertEquals(1, res); + } + + } + + public void testUpdateVersions(){ + + } + + public void testCompareTo(){ + + } + + public void testMergeGets() + throws IOException{ + List list = new ArrayList(); + list.add(putKv1); + list.add(putKv2); + + int res = 0; + for(int i=0; i< list.size(); i++){ + res = sget.compareTo(list.get(i), multiFamily); + assertEquals(1, res); +// System.out.println("res " +res); + } + +// System.out.println("newColumns.size " +((ServerGetFamilies)sget).getNewColumns().size()); + +// ((ServerGetFamilies)sget).mergeGets(multiFamily); +// System.out.println("mergedList.size " +mergedList.size()); + list.clear(); + list.add(putKv3); + list.add(putKv4); + + for(int i=0; i< list.size(); i++){ + res = sget.compareTo(list.get(i), multiFamily); +// assertEquals(1, res); +// System.out.println("res " +res); + } + +// ((ServerGetFamilies)sget).mergeGets(multiFamily); + + } +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestSingleLinkedList.java b/src/test/org/apache/hadoop/hbase/regionserver/TestSingleLinkedList.java new file mode 100644 index 0000000..fc42537 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestSingleLinkedList.java @@ -0,0 +1,105 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import junit.framework.TestCase; + +public class TestSingleLinkedList extends TestCase { + int size = 10; + List compList = new LinkedList(); + List list = new SingleLinkedList(); + + @Override + protected void setUp() throws Exception { + super.setUp(); + addToList(compList); + addToList(list); + } + + public void testWrite(){ + assertEquals(size, list.size()); + Iterator iter = list.iterator(); + Iterator compIter = compList.iterator(); + while(iter.hasNext()){ + assertEquals(iter.next(), compIter.next()); + } + } + + public void testRemove(){ + Iterator iter = list.iterator(); + Integer curr = 0; + + Integer rem0 = 0; + Integer rem5 = 5; + Integer rem9 = 9; + while(iter.hasNext()){ + curr = iter.next(); + if(curr == rem0){ + iter.remove(); + } else if(curr == rem5){ + iter.remove(); + } else if(curr == rem9){ + iter.remove(); + } + } + iter = list.iterator(); + while(iter.hasNext()){ + curr = iter.next(); + assertFalse(curr == rem0); + assertFalse(curr == rem5); + assertFalse(curr == rem9); + } + } + + //The timing test shows that arrayList is the fastest since it allocates + //big blocks at a time. But the SingleLinkedList is faster than the Double + //and the memory footprint is smaller + public void testAddTiming(){ + List list = null; + size = 100000; + long start = 0L; + long stop = 0L; + + //ArrayList + System.out.println("ArrayList"); + start = System.nanoTime(); + list = new ArrayList(); + addToList(list); + stop = System.nanoTime(); + System.out.println("timer " +(stop - start)); + +// //SingleLinkedList +// System.out.println("SingleLinkedList"); +// start = System.nanoTime(); +// list = new SingleLinkedList(); +// addToList(list); +// stop = System.nanoTime(); +// System.out.println("timer " +(stop - start)); + +// //LinkedList +// System.out.println("LinkedList"); +// start = System.nanoTime(); +// list = new LinkedList(); +// addToList(list); +// stop = System.nanoTime(); +// System.out.println("timer " +(stop - start)); + + } + + //Helpers + private void addToList(List list){ + for(int i=0; i()); + assertEquals(handler.getTableNames().size(), 2); + handler.disableTable(tableBname); + assertFalse(handler.isTableEnabled(tableBname)); + handler.deleteTable(tableBname); + assertEquals(handler.getTableNames().size(), 1); + handler.disableTable(tableAname); + assertFalse(handler.isTableEnabled(tableAname)); + handler.enableTable(tableAname); + assertTrue(handler.isTableEnabled(tableAname)); + handler.disableTable(tableAname); + handler.deleteTable(tableAname); + + // Make sure that trying to create a table with a bad column name creates + // an IllegalArgument exception. + List cDescriptors = new ArrayList(); + ColumnDescriptor badDescriptor = new ColumnDescriptor(); + badDescriptor.name = badColumnName; + cDescriptors.add(badDescriptor); + String message = null; + try { + handler.createTable(tableBname, cDescriptors); + } catch (IllegalArgument ia) { + message = ia.message; + } + assertEquals("Family names must end in a colon: " + new String(badColumnName), message); + } + + /** + * Tests adding a series of Mutations and BatchMutations, including a + * delete mutation. Also tests data retrieval, and getting back multiple + * versions. + * + * @throws Exception + */ + public void doTestTableMutations() throws Exception { + // Setup + ThriftServer.HBaseHandler handler = new ThriftServer.HBaseHandler(); + handler.createTable(tableAname, getColumnDescriptors()); + + // Apply a few Mutations to rowA + handler.mutateRow(tableAname, rowAname, getMutations()); + + // Assert that the changes were made + assertTrue(Bytes.equals(valueAname, handler.get(tableAname, rowAname, columnAname).value)); + TRowResult rowResult1 = handler.getRow(tableAname, rowAname); + assertTrue(Bytes.equals(rowAname, rowResult1.row)); + assertTrue(Bytes.equals(valueBname, rowResult1.columns.get(columnBname).value)); + + // Apply a few BatchMutations for rowA and rowB + handler.mutateRows(tableAname, getBatchMutations()); + + // Assert that changes were made to rowA + boolean failed1 = false; + try { + handler.get(tableAname, rowAname, columnAname); + } catch (NotFound nf) { + failed1 = true; + } + assertTrue(failed1); + assertTrue(Bytes.equals(valueCname, handler.get(tableAname, rowAname, columnBname).value)); + List versions = handler.getVer(tableAname, rowAname, columnBname, MAXVERSIONS); + assertTrue(Bytes.equals(valueCname, versions.get(0).value)); + assertTrue(Bytes.equals(valueBname, versions.get(1).value)); + + // Assert that changes were made to rowB + TRowResult rowResult2 = handler.getRow(tableAname, rowBname); + assertTrue(Bytes.equals(rowBname, rowResult2.row)); + assertTrue(Bytes.equals(valueCname, rowResult2.columns.get(columnAname).value)); + assertTrue(Bytes.equals(valueDname, rowResult2.columns.get(columnBname).value)); + + // Apply some deletes + handler.deleteAll(tableAname, rowAname, columnBname); + handler.deleteAllRow(tableAname, rowBname); + + // Assert that the deletes were applied + boolean failed2 = false; + try { + handler.get(tableAname, rowAname, columnBname); + } catch (NotFound nf) { + failed2 = true; + } + assertTrue(failed2); + assertNull(handler.getRow(tableAname, rowBname)); + + // Teardown + handler.disableTable(tableAname); + handler.deleteTable(tableAname); + } + + /** + * Similar to testTableMutations(), except Mutations are applied with + * specific timestamps and data retrieval uses these timestamps to + * extract specific versions of data. + * + * @throws Exception + */ + public void doTestTableTimestampsAndColumns() throws Exception { + // Setup + ThriftServer.HBaseHandler handler = new ThriftServer.HBaseHandler(); + handler.createTable(tableAname, getColumnDescriptors()); + + // Apply timestamped Mutations to rowA + long time1 = System.currentTimeMillis(); + handler.mutateRowTs(tableAname, rowAname, getMutations(), time1); + + // Sleep to assure that 'time1' and 'time2' will be different even with a + // coarse grained system timer. + Thread.sleep(1000); + + // Apply timestamped BatchMutations for rowA and rowB + long time2 = System.currentTimeMillis(); + handler.mutateRowsTs(tableAname, getBatchMutations(), time2); + + // Apply an overlapping timestamped mutation to rowB + handler.mutateRowTs(tableAname, rowBname, getMutations(), time2); + + // Assert that the timestamp-related methods retrieve the correct data + assertEquals(handler.getVerTs(tableAname, rowAname, columnBname, time2, MAXVERSIONS).size(), 2); + assertEquals(handler.getVerTs(tableAname, rowAname, columnBname, time1, MAXVERSIONS).size(), 1); + + TRowResult rowResult1 = handler.getRowTs(tableAname, rowAname, time1); + TRowResult rowResult2 = handler.getRowTs(tableAname, rowAname, time2); + assertTrue(Bytes.equals(rowResult1.columns.get(columnAname).value, valueAname)); + assertTrue(Bytes.equals(rowResult1.columns.get(columnBname).value, valueBname)); + assertTrue(Bytes.equals(rowResult2.columns.get(columnBname).value, valueCname)); + assertFalse(rowResult2.columns.containsKey(columnAname)); + + List columns = new ArrayList(); + columns.add(columnBname); + + rowResult1 = handler.getRowWithColumns(tableAname, rowAname, columns); + assertTrue(Bytes.equals(rowResult1.columns.get(columnBname).value, valueCname)); + assertFalse(rowResult1.columns.containsKey(columnAname)); + + rowResult1 = handler.getRowWithColumnsTs(tableAname, rowAname, columns, time1); + assertTrue(Bytes.equals(rowResult1.columns.get(columnBname).value, valueBname)); + assertFalse(rowResult1.columns.containsKey(columnAname)); + + // Apply some timestamped deletes + handler.deleteAllTs(tableAname, rowAname, columnBname, time1); + handler.deleteAllRowTs(tableAname, rowBname, time2); + + // Assert that the timestamp-related methods retrieve the correct data + boolean failed = false; + try { + handler.getVerTs(tableAname, rowAname, columnBname, time1, MAXVERSIONS); + } catch (NotFound nf) { + failed = true; + } + assertTrue(failed); + assertTrue(Bytes.equals(handler.get(tableAname, rowAname, columnBname).value, valueCname)); + assertNull(handler.getRow(tableAname, rowBname)); + + // Teardown + handler.disableTable(tableAname); + handler.deleteTable(tableAname); + } + + /** + * Tests the four different scanner-opening methods (with and without + * a stoprow, with and without a timestamp). + * + * @throws Exception + */ + public void doTestTableScanners() throws Exception { + // Setup + ThriftServer.HBaseHandler handler = new ThriftServer.HBaseHandler(); + handler.createTable(tableAname, getColumnDescriptors()); + + // Apply timestamped Mutations to rowA + long time1 = System.currentTimeMillis(); + handler.mutateRowTs(tableAname, rowAname, getMutations(), time1); + + // Sleep to assure that 'time1' and 'time2' will be different even with a + // coarse grained system timer. + Thread.sleep(1000); + + // Apply timestamped BatchMutations for rowA and rowB + long time2 = System.currentTimeMillis(); + handler.mutateRowsTs(tableAname, getBatchMutations(), time2); + + // Test a scanner on all rows and all columns, no timestamp + int scanner1 = handler.scannerOpen(tableAname, rowAname, getColumnList(true, true)); + TRowResult rowResult1a = handler.scannerGet(scanner1); + assertTrue(Bytes.equals(rowResult1a.row, rowAname)); + assertEquals(rowResult1a.columns.size(), 1); + assertTrue(Bytes.equals(rowResult1a.columns.get(columnBname).value, valueCname)); + TRowResult rowResult1b = handler.scannerGet(scanner1); + assertTrue(Bytes.equals(rowResult1b.row, rowBname)); + assertEquals(rowResult1b.columns.size(), 2); + assertTrue(Bytes.equals(rowResult1b.columns.get(columnAname).value, valueCname)); + assertTrue(Bytes.equals(rowResult1b.columns.get(columnBname).value, valueDname)); + closeScanner(scanner1, handler); + + // Test a scanner on all rows and all columns, with timestamp + int scanner2 = handler.scannerOpenTs(tableAname, rowAname, getColumnList(true, true), time1); + TRowResult rowResult2a = handler.scannerGet(scanner2); + assertEquals(rowResult2a.columns.size(), 2); + assertTrue(Bytes.equals(rowResult2a.columns.get(columnAname).value, valueAname)); + assertTrue(Bytes.equals(rowResult2a.columns.get(columnBname).value, valueBname)); + closeScanner(scanner2, handler); + + // Test a scanner on the first row and first column only, no timestamp + int scanner3 = handler.scannerOpenWithStop(tableAname, rowAname, rowBname, + getColumnList(true, false)); + closeScanner(scanner3, handler); + + // Test a scanner on the first row and second column only, with timestamp + int scanner4 = handler.scannerOpenWithStopTs(tableAname, rowAname, rowBname, + getColumnList(false, true), time1); + TRowResult rowResult4a = handler.scannerGet(scanner4); + assertEquals(rowResult4a.columns.size(), 1); + assertTrue(Bytes.equals(rowResult4a.columns.get(columnBname).value, valueBname)); + + // Teardown + handler.disableTable(tableAname); + handler.deleteTable(tableAname); + } + + /** + * + * @return a List of ColumnDescriptors for use in creating a table. Has one + * default ColumnDescriptor and one ColumnDescriptor with fewer versions + */ + private List getColumnDescriptors() { + ArrayList cDescriptors = new ArrayList(); + + // A default ColumnDescriptor + ColumnDescriptor cDescA = new ColumnDescriptor(); + cDescA.name = columnAname; + cDescriptors.add(cDescA); + + // A slightly customized ColumnDescriptor (only 2 versions) + ColumnDescriptor cDescB = new ColumnDescriptor(columnBname, 2, "NONE", + false, 2147483647, "NONE", 0, 0, false, -1); + cDescriptors.add(cDescB); + + return cDescriptors; + } + + /** + * + * @param includeA whether or not to include columnA + * @param includeB whether or not to include columnB + * @return a List of column names for use in retrieving a scanner + */ + private List getColumnList(boolean includeA, boolean includeB) { + List columnList = new ArrayList(); + if (includeA) columnList.add(columnAname); + if (includeB) columnList.add(columnBname); + return columnList; + } + + /** + * + * @return a List of Mutations for a row, with columnA having valueA + * and columnB having valueB + */ + private List getMutations() { + List mutations = new ArrayList(); + mutations.add(new Mutation(false, columnAname, valueAname)); + mutations.add(new Mutation(false, columnBname, valueBname)); + return mutations; + } + + /** + * + * @return a List of BatchMutations with the following effects: + * (rowA, columnA): delete + * (rowA, columnB): place valueC + * (rowB, columnA): place valueC + * (rowB, columnB): place valueD + */ + private List getBatchMutations() { + List batchMutations = new ArrayList(); + // Mutations to rowA + List rowAmutations = new ArrayList(); + rowAmutations.add(new Mutation(true, columnAname, null)); + rowAmutations.add(new Mutation(false, columnBname, valueCname)); + batchMutations.add(new BatchMutation(rowAname, rowAmutations)); + // Mutations to rowB + List rowBmutations = new ArrayList(); + rowBmutations.add(new Mutation(false, columnAname, valueCname)); + rowBmutations.add(new Mutation(false, columnBname, valueDname)); + batchMutations.add(new BatchMutation(rowBname, rowBmutations)); + return batchMutations; + } + + /** + * Asserts that the passed scanner is exhausted, and then closes + * the scanner. + * + * @param scannerId the scanner to close + * @param handler the HBaseHandler interfacing to HBase + * @throws Exception + */ + private void closeScanner(int scannerId, ThriftServer.HBaseHandler handler) throws Exception { + boolean failed = false; + try { + handler.scannerGet(scannerId); + } catch (NotFound nf) { + failed = true; + } + assertTrue(failed); + handler.scannerClose(scannerId); + } +}