From 6af4292630daca370de7fce3a2b2b3401cd1abfa Mon Sep 17 00:00:00 2001 From: Ryan Rawson Date: Sat, 6 Jun 2009 01:26:21 +0000 Subject: [PATCH] HBASE-1304 - New client server implementation of how gets and puts are handled. -- Thanks to jgray,holstad,stack,rawson git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@782178 13f79535-47bb-0310-9956-ffa450edef68 --- bin/HBase.rb | 1 - .../hadoop/hbase/HColumnDescriptor.java | 110 +- .../org/apache/hadoop/hbase/HConstants.java | 90 +- src/java/org/apache/hadoop/hbase/HMerge.java | 76 +- .../apache/hadoop/hbase/HTableDescriptor.java | 125 +- .../org/apache/hadoop/hbase/KeyValue.java | 1042 ++++--- .../apache/hadoop/hbase/RegionHistorian.java | 83 +- .../hadoop/hbase/WritableComparator.java | 7 +- .../apache/hadoop/hbase/client/Delete.java | 312 +++ .../org/apache/hadoop/hbase/client/Get.java | 398 +++ .../hadoop/hbase/client/HBaseAdmin.java | 66 +- .../hadoop/hbase/client/HConnection.java | 3 +- .../hbase/client/HConnectionManager.java | 96 +- .../apache/hadoop/hbase/client/HTable.java | 2477 +++++++++-------- .../hadoop/hbase/client/HTablePool.java | 1 + .../hadoop/hbase/client/MetaScanner.java | 11 +- .../org/apache/hadoop/hbase/client/Put.java | 305 ++ .../apache/hadoop/hbase/client/Result.java | 479 ++++ .../hadoop/hbase/client/ResultScanner.java | 52 + .../org/apache/hadoop/hbase/client/Scan.java | 456 +++ .../apache/hadoop/hbase/client/Scanner.java | 18 +- .../hadoop/hbase/client/ScannerCallable.java | 45 +- .../client/UnmodifyableHColumnDescriptor.java | 8 - .../client/UnmodifyableHTableDescriptor.java | 22 +- .../tableindexed/IndexKeyGenerator.java | 29 - .../tableindexed/IndexNotFoundException.java | 47 - .../tableindexed/IndexSpecification.java | 190 -- .../client/tableindexed/IndexedTable.java | 224 -- .../tableindexed/IndexedTableAdmin.java | 154 - .../hbase/client/tableindexed/package.html | 46 - .../transactional/LocalTransactionLogger.java | 71 - .../transactional/TransactionLogger.java | 59 - .../transactional/TransactionManager.java | 152 - .../TransactionScannerCallable.java | 51 - .../transactional/TransactionState.java | 78 - .../transactional/TransactionalTable.java | 428 --- .../UnknownTransactionException.java | 43 - .../hbase/client/transactional/package.html | 61 - .../hbase/filter/ColumnValueFilter.java | 3 + .../apache/hadoop/hbase/filter/Filter.java | 111 + .../hbase/filter/InclusiveStopRowFilter.java | 13 +- .../hadoop/hbase/filter/PageFilter.java | 92 + .../hadoop/hbase/filter/PageRowFilter.java | 2 + .../hadoop/hbase/filter/PrefixRowFilter.java | 2 + .../hbase/filter/RowFilterInterface.java | 13 +- .../hadoop/hbase/filter/RowFilterSet.java | 4 + .../hbase/filter/RowInclusiveStopFilter.java | 89 + .../RowPrefixFilter.java} | 79 +- .../hbase/filter/RowWhileMatchFilter.java | 96 + .../hadoop/hbase/filter/StopRowFilter.java | 2 + .../hbase/filter/WhileMatchRowFilter.java | 2 + .../hadoop/hbase/filter/package-info.java | 2 +- .../hadoop/hbase/io/BatchOperation.java | 3 +- .../apache/hadoop/hbase/io/BatchUpdate.java | 1 + src/java/org/apache/hadoop/hbase/io/Cell.java | 6 +- .../hadoop/hbase/io/CodeToClassAndBack.java | 6 +- .../hadoop/hbase/io/HbaseObjectWritable.java | 71 +- .../org/apache/hadoop/hbase/io/HeapSize.java | 21 +- .../hbase/io/ImmutableBytesWritable.java | 16 +- .../org/apache/hadoop/hbase/io/Reference.java | 7 + .../org/apache/hadoop/hbase/io/RowResult.java | 20 +- .../org/apache/hadoop/hbase/io/TimeRange.java | 172 ++ .../apache/hadoop/hbase/io/hfile/HFile.java | 16 +- .../hbase/io/hfile/SimpleBlockCache.java | 6 + .../org/apache/hadoop/hbase/ipc/HBaseRPC.java | 1 + .../apache/hadoop/hbase/ipc/HBaseServer.java | 18 +- .../hadoop/hbase/ipc/HMasterInterface.java | 3 +- .../hadoop/hbase/ipc/HRegionInterface.java | 248 +- .../ipc/TransactionalRegionInterface.java | 24 +- .../hbase/mapred/TableInputFormatBase.java | 41 +- .../hbase/mapred/TableOutputFormat.java | 11 +- .../hadoop/hbase/master/BaseScanner.java | 50 +- .../hadoop/hbase/master/ChangeTableState.java | 20 +- .../hadoop/hbase/master/ColumnOperation.java | 8 +- .../apache/hadoop/hbase/master/HMaster.java | 131 +- .../hadoop/hbase/master/ModifyTableMeta.java | 7 +- .../hbase/master/ProcessRegionOpen.java | 12 +- .../hbase/master/ProcessServerShutdown.java | 25 +- .../hadoop/hbase/master/RegionManager.java | 40 +- .../hadoop/hbase/master/TableOperation.java | 22 +- .../hbase/regionserver/ColumnCount.java | 112 + .../hbase/regionserver/ColumnTracker.java | 78 + .../regionserver/CompactSplitThread.java | 21 +- .../hbase/regionserver/DeleteCompare.java | 120 + .../hbase/regionserver/DeleteTracker.java | 97 + .../regionserver/ExplicitColumnTracker.java | 157 ++ .../regionserver/FailedLogCloseException.java | 6 + .../hbase/regionserver/GetDeleteTracker.java | 405 +++ .../hbase/regionserver/HAbstractScanner.java | 214 -- .../hadoop/hbase/regionserver/HLog.java | 13 +- .../hadoop/hbase/regionserver/HLogKey.java | 3 + .../hadoop/hbase/regionserver/HRegion.java | 1387 ++++----- .../hbase/regionserver/HRegionServer.java | 324 +-- .../hbase/regionserver/InternalScanner.java | 16 +- .../hbase/regionserver/KeyValueHeap.java | 197 ++ .../KeyValueScanner.java} | 57 +- .../hadoop/hbase/regionserver/Memcache.java | 456 ++- .../hbase/regionserver/MemcacheFlusher.java | 8 +- .../MinorCompactingStoreScanner.java | 119 + .../hbase/regionserver/QueryMatcher.java | 373 +++ .../hbase/regionserver/ScanDeleteTracker.java | 161 ++ .../hbase/regionserver/ScanQueryMatcher.java | 242 ++ .../ScanWildcardColumnTracker.java | 124 + .../hadoop/hbase/regionserver/Store.java | 650 ++--- .../hadoop/hbase/regionserver/StoreFile.java | 44 +- .../hbase/regionserver/StoreFileGetScan.java | 110 + .../hbase/regionserver/StoreFileScanner.java | 335 +-- .../hbase/regionserver/StoreScanner.java | 432 ++- .../regionserver/WildcardColumnTracker.java | 314 +++ .../tableindexed/IndexMaintenanceUtils.java | 73 - .../tableindexed/IndexedRegion.java | 305 -- .../tableindexed/IndexedRegionServer.java | 74 - .../CleanOldTransactionsChore.java | 57 - .../transactional/TransactionState.java | 362 --- .../TransactionalHLogManager.java | 307 -- .../transactional/TransactionalRegion.java | 718 ----- .../TransactionalRegionServer.java | 304 -- .../hadoop/hbase/rest/RowController.java | 10 +- .../apache/hadoop/hbase/rest/RowModel.java | 87 +- .../hadoop/hbase/rest/ScannerModel.java | 67 +- .../apache/hadoop/hbase/rest/TableModel.java | 33 +- .../hadoop/hbase/rest/TimestampModel.java | 111 +- .../hbase/rest/parser/XMLRestParser.java | 11 +- .../rest/serializer/SimpleXMLSerializer.java | 4 - .../hadoop/hbase/thrift/ThriftServer.java | 170 +- .../hadoop/hbase/thrift/ThriftUtilities.java | 37 +- .../thrift/generated/ColumnDescriptor.java | 91 +- .../org/apache/hadoop/hbase/util/Bytes.java | 53 +- .../org/apache/hadoop/hbase/util/Merge.java | 28 +- .../apache/hadoop/hbase/util/MetaUtils.java | 125 +- .../org/apache/hadoop/hbase/util/Migrate.java | 21 +- .../hadoop/hbase/AbstractMergeTestBase.java | 10 +- .../org/apache/hadoop/hbase/DFSAbort.java | 2 +- .../apache/hadoop/hbase/HBaseTestCase.java | 240 +- .../hadoop/hbase/KeyValueTestUtil.java} | 48 +- .../apache/hadoop/hbase/MiniHBaseCluster.java | 2 +- .../hadoop/hbase/PerformanceEvaluation.java | 53 +- .../hadoop/hbase/TestEmptyMetaInfo.java | 27 +- .../apache/hadoop/hbase/TestHBaseCluster.java | 99 +- .../org/apache/hadoop/hbase/TestKeyValue.java | 124 +- .../hadoop/hbase/TestRegionRebalancing.java | 8 +- .../hbase/TestScanMultipleVersions.java | 67 +- .../apache/hadoop/hbase/TestScannerAPI.java | 166 -- .../hadoop/hbase/TestSerialization.java | 374 ++- .../org/apache/hadoop/hbase/TestTable.java | 15 +- .../apache/hadoop/hbase/TestZooKeeper.java | 7 +- .../hadoop/hbase/TimestampTestBase.java | 128 +- .../hadoop/hbase/client/TestBatchUpdate.java | 138 +- .../hadoop/hbase/client/TestClient.java | 2460 ++++++++++++++++ .../hadoop/hbase/client/TestForceSplit.java | 9 +- .../apache/hadoop/hbase/client/TestGet.java | 94 + .../hbase/client/TestGetRowVersions.java | 81 +- .../hadoop/hbase/client/TestHBaseAdmin.java | 323 +++ .../hadoop/hbase/client/TestHTable.java | 304 +- .../hadoop/hbase/client/TestListTables.java | 2 +- .../client/TestOldAPIGetRowVersions.java | 107 + .../hadoop/hbase/client/TestOldAPIHTable.java | 459 +++ .../hbase/client/TestOldAPITimestamp.java | 71 + .../apache/hadoop/hbase/client/TestPut.java | 202 ++ .../hadoop/hbase/client/TestScannerTimes.java | 63 +- .../client/tableindexed/TestIndexedTable.java | 131 - .../DisabledTestTransactions.java | 143 - .../transactional/StressTestTransactions.java | 420 --- ...sabledTestRowFilterOnMultipleFamilies.java | 33 +- ...PageRowFilter.java => TestPageFilter.java} | 52 +- .../hbase/filter/TestRowPrefixFilter.java | 91 + .../hbase/io/TestHbaseObjectWritable.java | 2 +- ...Index.java => DisabledTestTableIndex.java} | 31 +- ...e.java => DisabledTestTableMapReduce.java} | 21 +- .../DisabledTestRegionServerExit.java | 33 +- .../regionserver/KeyValueScanFixture.java | 91 + .../hbase/regionserver/OOMERegionServer.java | 10 +- .../regionserver/TestAtomicIncrement.java | 121 - .../hbase/regionserver/TestBloomFilters.java | 247 -- .../hbase/regionserver/TestCompaction.java | 58 +- .../hbase/regionserver/TestDeleteAll.java | 233 -- .../hbase/regionserver/TestDeleteCompare.java | 191 ++ .../hbase/regionserver/TestDeleteFamily.java | 224 -- .../TestExplicitColumnTracker.java | 144 + .../hadoop/hbase/regionserver/TestGet.java | 168 -- .../hadoop/hbase/regionserver/TestGet2.java | 721 ----- .../regionserver/TestGetDeleteTracker.java | 313 +++ .../hadoop/hbase/regionserver/TestHLog.java | 2 +- .../hbase/regionserver/TestHMemcache.java | 458 --- .../hbase/regionserver/TestHRegion.java | 1810 ++++++++---- .../hbase/regionserver/TestKeyValueHeap.java | 207 ++ .../regionserver/TestKeyValueScanFixture.java | 68 + .../hbase/regionserver/TestLogRolling.java | 11 +- .../hbase/regionserver/TestMemcache.java | 602 ++++ .../TestMinorCompactingStoreScanner.java | 83 + .../hbase/regionserver/TestQueryMatcher.java | 170 ++ .../regionserver/TestScanDeleteTracker.java | 112 + .../TestScanWildcardColumnTracker.java | 101 + .../hbase/regionserver/TestScanner.java | 120 +- .../hadoop/hbase/regionserver/TestSplit.java | 264 -- .../hadoop/hbase/regionserver/TestStore.java | 325 +++ .../hbase/regionserver/TestStoreScanner.java | 480 ++++ .../hbase/regionserver/TestTimestamp.java | 85 - .../TestWildcardColumnTracker.java | 336 +++ .../DisabledTestHLogRecovery.java | 279 -- .../DisabledTestTransactionalHLogManager.java | 308 -- ...ver.java => DisabledTestThriftServer.java} | 40 +- ...geTool.java => DisabledTestMergeTool.java} | 79 +- .../hadoop/hbase/util/MigrationTest.java | 23 +- .../hbase/util/SoftValueSortedMapTest.java | 2 +- .../hadoop/hbase/util/TestRootPath.java | 2 +- src/webapps/master/master.jsp | 14 - src/webapps/master/table.jsp | 8 +- 208 files changed, 19828 insertions(+), 14377 deletions(-) create mode 100644 src/java/org/apache/hadoop/hbase/client/Delete.java create mode 100644 src/java/org/apache/hadoop/hbase/client/Get.java create mode 100644 src/java/org/apache/hadoop/hbase/client/Put.java create mode 100644 src/java/org/apache/hadoop/hbase/client/Result.java create mode 100644 src/java/org/apache/hadoop/hbase/client/ResultScanner.java create mode 100644 src/java/org/apache/hadoop/hbase/client/Scan.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/tableindexed/IndexKeyGenerator.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/tableindexed/IndexNotFoundException.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/tableindexed/IndexSpecification.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/tableindexed/IndexedTable.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/tableindexed/IndexedTableAdmin.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/tableindexed/package.html delete mode 100644 src/java/org/apache/hadoop/hbase/client/transactional/LocalTransactionLogger.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/transactional/TransactionLogger.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/transactional/TransactionScannerCallable.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/transactional/TransactionState.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/transactional/UnknownTransactionException.java delete mode 100644 src/java/org/apache/hadoop/hbase/client/transactional/package.html create mode 100644 src/java/org/apache/hadoop/hbase/filter/Filter.java create mode 100644 src/java/org/apache/hadoop/hbase/filter/PageFilter.java create mode 100644 src/java/org/apache/hadoop/hbase/filter/RowInclusiveStopFilter.java rename src/java/org/apache/hadoop/hbase/{client/tableindexed/SimpleIndexKeyGenerator.java => filter/RowPrefixFilter.java} (50%) create mode 100644 src/java/org/apache/hadoop/hbase/filter/RowWhileMatchFilter.java create mode 100644 src/java/org/apache/hadoop/hbase/io/TimeRange.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/DeleteTracker.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/GetDeleteTracker.java delete mode 100644 src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java rename src/java/org/apache/hadoop/hbase/{client/transactional/CommitUnsuccessfulException.java => regionserver/KeyValueScanner.java} (53%) create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/MinorCompactingStoreScanner.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/QueryMatcher.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/ScanDeleteTracker.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/StoreFileGetScan.java create mode 100644 src/java/org/apache/hadoop/hbase/regionserver/WildcardColumnTracker.java delete mode 100644 src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexMaintenanceUtils.java delete mode 100644 src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexedRegion.java delete mode 100644 src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexedRegionServer.java delete mode 100644 src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionState.java delete mode 100644 src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalHLogManager.java delete mode 100644 src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegion.java delete mode 100644 src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionServer.java rename src/{java/org/apache/hadoop/hbase/client/tableindexed/ReverseByteArrayComparator.java => test/org/apache/hadoop/hbase/KeyValueTestUtil.java} (54%) delete mode 100644 src/test/org/apache/hadoop/hbase/TestScannerAPI.java create mode 100644 src/test/org/apache/hadoop/hbase/client/TestClient.java create mode 100644 src/test/org/apache/hadoop/hbase/client/TestGet.java create mode 100644 src/test/org/apache/hadoop/hbase/client/TestHBaseAdmin.java create mode 100644 src/test/org/apache/hadoop/hbase/client/TestOldAPIGetRowVersions.java create mode 100644 src/test/org/apache/hadoop/hbase/client/TestOldAPIHTable.java create mode 100644 src/test/org/apache/hadoop/hbase/client/TestOldAPITimestamp.java create mode 100644 src/test/org/apache/hadoop/hbase/client/TestPut.java delete mode 100644 src/test/org/apache/hadoop/hbase/client/tableindexed/TestIndexedTable.java delete mode 100644 src/test/org/apache/hadoop/hbase/client/transactional/DisabledTestTransactions.java delete mode 100644 src/test/org/apache/hadoop/hbase/client/transactional/StressTestTransactions.java rename src/test/org/apache/hadoop/hbase/filter/{DisabledTestPageRowFilter.java => TestPageFilter.java} (67%) create mode 100644 src/test/org/apache/hadoop/hbase/filter/TestRowPrefixFilter.java rename src/test/org/apache/hadoop/hbase/mapred/{TestTableIndex.java => DisabledTestTableIndex.java} (91%) rename src/test/org/apache/hadoop/hbase/mapred/{TestTableMapReduce.java => DisabledTestTableMapReduce.java} (93%) create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java delete mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestAtomicIncrement.java delete mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestBloomFilters.java delete mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestDeleteCompare.java delete mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestExplicitColumnTracker.java delete mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestGet.java delete mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestGetDeleteTracker.java delete mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestMemcache.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestMinorCompactingStoreScanner.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestScanDeleteTracker.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestScanWildcardColumnTracker.java delete mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestStore.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java delete mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java create mode 100644 src/test/org/apache/hadoop/hbase/regionserver/TestWildcardColumnTracker.java rename src/test/org/apache/hadoop/hbase/thrift/{TestThriftServer.java => DisabledTestThriftServer.java} (92%) rename src/test/org/apache/hadoop/hbase/util/{TestMergeTool.java => DisabledTestMergeTool.java} (78%) diff --git a/bin/HBase.rb b/bin/HBase.rb index 4df6e783398..70bc913784f 100644 --- a/bin/HBase.rb +++ b/bin/HBase.rb @@ -311,7 +311,6 @@ module HBase arg[IN_MEMORY]? JBoolean.valueOf(arg[IN_MEMORY]): HColumnDescriptor::DEFAULT_IN_MEMORY, arg[HColumnDescriptor::BLOCKCACHE]? JBoolean.valueOf(arg[HColumnDescriptor::BLOCKCACHE]): HColumnDescriptor::DEFAULT_BLOCKCACHE, arg[HColumnDescriptor::BLOCKSIZE]? JInteger.valueOf(arg[HColumnDescriptor::BLOCKSIZE]): HColumnDescriptor::DEFAULT_BLOCKSIZE, - arg[HColumnDescriptor::LENGTH]? JInteger.new(arg[HColumnDescriptor::LENGTH]): HColumnDescriptor::DEFAULT_LENGTH, arg[HColumnDescriptor::TTL]? JInteger.new(arg[HColumnDescriptor::TTL]): HColumnDescriptor::DEFAULT_TTL, arg[HColumnDescriptor::BLOOMFILTER]? JBoolean.valueOf(arg[HColumnDescriptor::BLOOMFILTER]): HColumnDescriptor::DEFAULT_BLOOMFILTER) end diff --git a/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java index 9553c6e7988..0ef06e328d2 100644 --- a/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.hfile.Compression; import org.apache.hadoop.hbase.io.hfile.HFile; @@ -93,20 +94,6 @@ 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 (UNSUPPORTED) + * @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 + * + * @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 + * @deprecated As of hbase 0.20.0, max value length no longer supported + */ +// 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) { +// this(familyName, maxVersions, compression, inMemory, blockCacheEnabled, +// blocksize, timeToLive, bloomFilter); +// } + /** * Constructor * @param familyName Column family name. Must be 'printable' -- digit or @@ -235,7 +254,6 @@ public class HColumnDescriptor implements ISerializable, WritableComparable. Family names cannot contain control characters: " + + ">. Family names cannot contain control characters or colons: " + Bytes.toString(b)); } } @@ -317,7 +334,7 @@ public class HColumnDescriptor implements ISerializable, WritableComparable 0) { - this.name = stripColon(this.name); - } +// if(KeyValue.getFamilyDelimiterIndex(this.name, 0, this.name.length) +// > 0) { +// this.name = stripColon(this.name); +// } } else { this.name = Bytes.readByteArray(in); } @@ -620,7 +617,6 @@ public class HColumnDescriptor implements ISerializable, WritableComparable"); - Cell regionInfo = currentRow.get(COL_REGIONINFO); - if (regionInfo == null || regionInfo.getValue().length == 0) { + byte [] regionInfoValue = currentRow.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER); + if (regionInfoValue == null || regionInfoValue.length == 0) { currentRow = metaScanner.next(); continue; } @@ -286,17 +288,18 @@ class HMerge implements HConstants { if(Bytes.equals(regionsToDelete[r], latestRegion.getRegionName())) { latestRegion = null; } - table.deleteAll(regionsToDelete[r]); + Delete delete = new Delete(regionsToDelete[r]); + table.delete(delete); if(LOG.isDebugEnabled()) { LOG.debug("updated columns in row: " + Bytes.toString(regionsToDelete[r])); } } newRegion.getRegionInfo().setOffline(true); - BatchUpdate update = new BatchUpdate(newRegion.getRegionName()); - update.put(COL_REGIONINFO, + Put put = new Put(newRegion.getRegionName()); + put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(newRegion.getRegionInfo())); - table.commit(update); + table.put(put); if(LOG.isDebugEnabled()) { LOG.debug("updated columns in row: " @@ -325,9 +328,10 @@ class HMerge implements HConstants { HRegionInfo.ROOT_REGIONINFO, null); root.initialize(null, null); + Scan scan = new Scan(); + scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER); InternalScanner rootScanner = - root.getScanner(COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW, - HConstants.LATEST_TIMESTAMP, null); + root.getScanner(scan); try { List results = new ArrayList(); @@ -366,23 +370,29 @@ class HMerge implements HConstants { throws IOException { byte[][] regionsToDelete = {oldRegion1, oldRegion2}; for(int r = 0; r < regionsToDelete.length; r++) { - BatchUpdate b = new BatchUpdate(regionsToDelete[r]); - b.delete(COL_REGIONINFO); - b.delete(COL_SERVER); - b.delete(COL_STARTCODE); - b.delete(COL_SPLITA); - b.delete(COL_SPLITB); - root.batchUpdate(b,null); - + Delete delete = new Delete(regionsToDelete[r]); + delete.deleteColumns(HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER); + delete.deleteColumns(HConstants.CATALOG_FAMILY, + HConstants.SERVER_QUALIFIER); + delete.deleteColumns(HConstants.CATALOG_FAMILY, + HConstants.STARTCODE_QUALIFIER); + delete.deleteColumns(HConstants.CATALOG_FAMILY, + HConstants.SPLITA_QUALIFIER); + delete.deleteColumns(HConstants.CATALOG_FAMILY, + HConstants.SPLITB_QUALIFIER); + root.delete(delete, null, true); + if(LOG.isDebugEnabled()) { LOG.debug("updated columns in row: " + Bytes.toString(regionsToDelete[r])); } } HRegionInfo newInfo = newRegion.getRegionInfo(); newInfo.setOffline(true); - BatchUpdate b = new BatchUpdate(newRegion.getRegionName()); - b.put(COL_REGIONINFO, Writables.getBytes(newInfo)); - root.batchUpdate(b,null); + Put put = new Put(newRegion.getRegionName()); + put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, + Writables.getBytes(newInfo)); + root.put(put); if(LOG.isDebugEnabled()) { LOG.debug("updated columns in row: " + Bytes.toString(newRegion.getRegionName())); } diff --git a/src/java/org/apache/hadoop/hbase/HTableDescriptor.java b/src/java/org/apache/hadoop/hbase/HTableDescriptor.java index c8eefa5a026..9ddc9d5b776 100644 --- a/src/java/org/apache/hadoop/hbase/HTableDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/HTableDescriptor.java @@ -27,10 +27,11 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.Set; import java.util.TreeMap; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification; +//import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.hfile.Compression; import org.apache.hadoop.hbase.rest.exception.HBaseRestException; @@ -45,7 +46,8 @@ import agilejson.TOJSON; * HTableDescriptor contains the name of an HTable, and its * column families. */ -public class HTableDescriptor implements WritableComparable, ISerializable { +public class HTableDescriptor implements WritableComparable, +ISerializable { // Changes prior to version 3 were not recorded here. // Version 3 adds metadata as a map where keys and values are byte[]. @@ -100,12 +102,14 @@ public class HTableDescriptor implements WritableComparable, I private volatile Boolean root = null; // Key is hash of the family name. - private final Map families = + public final Map families = new TreeMap(KeyValue.FAMILY_COMPARATOR); - +// private final Map families = +// new TreeMap(KeyValue.FAMILY_COMPARATOR); + // Key is indexId - private final Map indexes = - new HashMap(); +// private final Map indexes = +// new HashMap(); /** * Private constructor used internally creating table descriptors for @@ -125,24 +129,38 @@ public class HTableDescriptor implements WritableComparable, I * Private constructor used internally creating table descriptors for * catalog tables: e.g. .META. and -ROOT-. */ +// protected HTableDescriptor(final byte [] name, HColumnDescriptor[] families, +// Collection indexes, +// Map values) { +// this.name = name.clone(); +// this.nameAsString = Bytes.toString(this.name); +// setMetaFlags(name); +// for(HColumnDescriptor descriptor : families) { +// this.families.put(descriptor.getName(), descriptor); +// } +// for(IndexSpecification index : indexes) { +// this.indexes.put(index.getIndexId(), index); +// } +// for (Map.Entry entry: +// values.entrySet()) { +// this.values.put(entry.getKey(), entry.getValue()); +// } +// } protected HTableDescriptor(final byte [] name, HColumnDescriptor[] families, - Collection indexes, - Map values) { + Map values) { this.name = name.clone(); this.nameAsString = Bytes.toString(this.name); setMetaFlags(name); for(HColumnDescriptor descriptor : families) { this.families.put(descriptor.getName(), descriptor); } - for(IndexSpecification index : indexes) { - this.indexes.put(index.getIndexId(), index); - } for (Map.Entry entry: values.entrySet()) { this.values.put(entry.getKey(), entry.getValue()); } } - + + /** * Constructs an empty object. * For deserializing an HTableDescriptor instance only. @@ -198,7 +216,7 @@ public class HTableDescriptor implements WritableComparable, I desc.values.entrySet()) { this.values.put(e.getKey(), e.getValue()); } - this.indexes.putAll(desc.indexes); +// this.indexes.putAll(desc.indexes); } /* @@ -437,21 +455,17 @@ public class HTableDescriptor implements WritableComparable, I Bytes.toBytes(Integer.toString(memcacheFlushSize))); } - public Collection getIndexes() { - return indexes.values(); - } - - public IndexSpecification getIndex(String indexId) { - return indexes.get(indexId); - } - - public void addIndex(IndexSpecification index) { - indexes.put(index.getIndexId(), index); - } - - public void removeIndex(String indexId) { - indexes.remove(indexId); - } +// public Collection getIndexes() { +// return indexes.values(); +// } +// +// public IndexSpecification getIndex(String indexId) { +// return indexes.get(indexId); +// } +// +// public void addIndex(IndexSpecification index) { +// indexes.put(index.getIndexId(), index); +// } /** * Adds a column family. @@ -510,13 +524,13 @@ public class HTableDescriptor implements WritableComparable, I s.append(FAMILIES); s.append(" => "); s.append(families.values()); - if (!indexes.isEmpty()) { - // Don't emit if empty. Has to do w/ transactional hbase. - s.append(", "); - s.append("INDEXES"); - s.append(" => "); - s.append(indexes.values()); - } +// if (!indexes.isEmpty()) { +// // Don't emit if empty. Has to do w/ transactional hbase. +// s.append(", "); +// s.append("INDEXES"); +// s.append(" => "); +// s.append(indexes.values()); +// } s.append('}'); return s.toString(); } @@ -581,16 +595,16 @@ public class HTableDescriptor implements WritableComparable, I c.readFields(in); families.put(c.getName(), c); } - indexes.clear(); +// indexes.clear(); if (version < 4) { return; } - int numIndexes = in.readInt(); - for (int i = 0; i < numIndexes; i++) { - IndexSpecification index = new IndexSpecification(); - index.readFields(in); - addIndex(index); - } +// int numIndexes = in.readInt(); +// for (int i = 0; i < numIndexes; i++) { +// IndexSpecification index = new IndexSpecification(); +// index.readFields(in); +// addIndex(index); +// } } public void write(DataOutput out) throws IOException { @@ -610,10 +624,10 @@ public class HTableDescriptor implements WritableComparable, I HColumnDescriptor family = it.next(); family.write(out); } - out.writeInt(indexes.size()); - for(IndexSpecification index : indexes.values()) { - index.write(out); - } +// out.writeInt(indexes.size()); +// for(IndexSpecification index : indexes.values()) { +// index.write(out); +// } } // Comparable @@ -654,6 +668,13 @@ public class HTableDescriptor implements WritableComparable, I 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() { return getFamilies().toArray(new HColumnDescriptor[0]); @@ -689,22 +710,22 @@ public class HTableDescriptor implements WritableComparable, I /** Table descriptor for -ROOT- catalog table */ public static final HTableDescriptor ROOT_TABLEDESC = new HTableDescriptor( HConstants.ROOT_TABLE_NAME, - new HColumnDescriptor[] { new HColumnDescriptor(HConstants.COLUMN_FAMILY, + new HColumnDescriptor[] { new HColumnDescriptor(HConstants.CATALOG_FAMILY, 10, // Ten is arbitrary number. Keep versions to help debuggging. Compression.Algorithm.NONE.getName(), false, true, 8 * 1024, - Integer.MAX_VALUE, HConstants.FOREVER, false) }); + HConstants.FOREVER, false) }); /** Table descriptor for .META. catalog table */ public static final HTableDescriptor META_TABLEDESC = new HTableDescriptor( HConstants.META_TABLE_NAME, new HColumnDescriptor[] { - new HColumnDescriptor(HConstants.COLUMN_FAMILY, + new HColumnDescriptor(HConstants.CATALOG_FAMILY, 10, // Ten is arbitrary number. Keep versions to help debuggging. Compression.Algorithm.NONE.getName(), false, true, 8 * 1024, - Integer.MAX_VALUE, HConstants.FOREVER, false), - new HColumnDescriptor(HConstants.COLUMN_FAMILY_HISTORIAN, + HConstants.FOREVER, false), + new HColumnDescriptor(HConstants.CATALOG_HISTORIAN_FAMILY, HConstants.ALL_VERSIONS, Compression.Algorithm.NONE.getName(), false, false, 8 * 1024, - Integer.MAX_VALUE, HConstants.WEEK_IN_SECONDS, false)}); + HConstants.WEEK_IN_SECONDS, false)}); /* (non-Javadoc) * @see org.apache.hadoop.hbase.rest.xml.IOutputXML#toXML() diff --git a/src/java/org/apache/hadoop/hbase/KeyValue.java b/src/java/org/apache/hadoop/hbase/KeyValue.java index d7b44244868..4532ee956d0 100644 --- a/src/java/org/apache/hadoop/hbase/KeyValue.java +++ b/src/java/org/apache/hadoop/hbase/KeyValue.java @@ -25,11 +25,11 @@ import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.io.HeapSize; -import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.RawComparator; +import org.apache.hadoop.io.Writable; /** * An HBase Key/Value. Instances of this class are immutable. They are not @@ -59,6 +59,8 @@ public class KeyValue implements Writable, HeapSize { * Colon character in UTF-8 */ public static final char COLUMN_FAMILY_DELIMITER = ':'; + + public static final byte[] COLUMN_FAMILY_DELIM_ARRAY = new byte[]{COLUMN_FAMILY_DELIMITER}; /** * Comparator for plain key/values; i.e. non-catalog table key/values. @@ -118,26 +120,43 @@ public class KeyValue implements Writable, HeapSize { return compare(a, 0, a.length, b, 0, b.length); } }; + + /** + * Get the appropriate row comparator for the specified table. + * Hopefully we can get rid of this, I added this here because it's replacing + * something in HSK. We should move completely off of that. + * @param tableName + * @return + */ + public static RawComparator getRowComparator(byte [] tableName) { + if(Bytes.equals(HTableDescriptor.ROOT_TABLEDESC.getName(),tableName)) { + return ROOT_COMPARATOR.getRawComparator(); + } + if(Bytes.equals(HTableDescriptor.META_TABLEDESC.getName(), tableName)) { + return META_COMPARATOR.getRawComparator(); + } + return COMPARATOR.getRawComparator(); + } // Size of the timestamp and type byte on end of a key -- a long + a byte. - private static final int TIMESTAMP_TYPE_SIZE = + public static final int TIMESTAMP_TYPE_SIZE = Bytes.SIZEOF_LONG /* timestamp */ + Bytes.SIZEOF_BYTE /*keytype*/; // Size of the length shorts and bytes in key. - private static final int KEY_INFRASTRUCTURE_SIZE = + public static final int KEY_INFRASTRUCTURE_SIZE = Bytes.SIZEOF_SHORT /*rowlength*/ + Bytes.SIZEOF_BYTE /*columnfamilylength*/ + TIMESTAMP_TYPE_SIZE; // How far into the key the row starts at. First thing to read is the short // that says how long the row is. - private static final int ROW_OFFSET = + public static final int ROW_OFFSET = Bytes.SIZEOF_INT /*keylength*/ + Bytes.SIZEOF_INT /*valuelength*/; // Size of the length ints in a KeyValue datastructure. - private static final int KEYVALUE_INFRASTRUCTURE_SIZE = ROW_OFFSET; + public static final int KEYVALUE_INFRASTRUCTURE_SIZE = ROW_OFFSET; /** * Key type. @@ -145,10 +164,13 @@ public class KeyValue implements Writable, HeapSize { * enum ordinals . They change if item is removed or moved. Do our own codes. */ public static enum Type { + Minimum((byte)0), Put((byte)4), + Delete((byte)8), DeleteColumn((byte)12), DeleteFamily((byte)14), + // Maximum is used when searching; you look from maximum on down. Maximum((byte)255); @@ -233,178 +255,258 @@ public class KeyValue implements Writable, HeapSize { this.offset = offset; this.length = length; } + + /** Temporary constructors until 880/1249 is committed to remove deps */ + + /** + * Temporary. + */ + public KeyValue(final byte [] row, final byte [] column) { + this(row, column, HConstants.LATEST_TIMESTAMP, null); + } + + public KeyValue(final byte [] row, final byte [] column, long ts) { + this(row, column, ts, null); + } + + public KeyValue(final byte [] row, final byte [] column, long ts, + byte [] value) { + this(row, column, ts, Type.Put, value); + } + + public KeyValue(final byte [] row, final byte [] column, long ts, Type type, + byte [] value) { + int rlength = row == null ? 0 : row.length; + int vlength = value == null ? 0 : value.length; + int clength = column == null ? 0 : column.length; + this.bytes = createByteArray(row, 0, rlength, column, 0, clength, + ts, type, value, 0, vlength); + this.length = this.bytes.length; + this.offset = 0; + } + + /** Constructors that build a new backing byte array from fields */ - /** - * Constructs KeyValue structure filled with null value. - * @param row - row key (arbitrary byte array) - * @param timestamp - */ - public KeyValue(final String row, final long timestamp) { - this(Bytes.toBytes(row), timestamp); - } - /** * Constructs KeyValue structure filled with null value. * @param row - row key (arbitrary byte array) * @param timestamp */ public KeyValue(final byte [] row, final long timestamp) { - this(row, null, timestamp, Type.Put, null); + this(row, timestamp, Type.Put); } /** * Constructs KeyValue structure filled with null value. * @param row - row key (arbitrary byte array) - * @param column Column with delimiter between family and qualifier + * @param timestamp */ - public KeyValue(final String row, final String column) { - this(row, column, null); + public KeyValue(final byte [] row, final long timestamp, Type type) { + this(row, null, null, timestamp, type, null); } /** * Constructs KeyValue structure filled with null value. * @param row - row key (arbitrary byte array) - * @param column Column with delimiter between family and qualifier + * @param family family name + * @param qualifier column qualifier */ - public KeyValue(final byte [] row, final byte [] column) { - this(row, column, null); - } - - /** - * Constructs KeyValue structure filled with specified value. - * @param row - row key (arbitrary byte array) - * @param column Column with delimiter between family and qualifier - * @param value - */ - public KeyValue(final String row, final String column, final byte [] value) { - this(Bytes.toBytes(row), Bytes.toBytes(column), value); - } - - /** - * Constructs KeyValue structure filled with specified value. - * @param row - row key (arbitrary byte array) - * @param column Column with delimiter between family and qualifier - * @param value - */ - public KeyValue(final byte [] row, final byte [] column, final byte [] value) { - this(row, column, HConstants.LATEST_TIMESTAMP, value); - } - - - /** - * Constructs KeyValue structure filled with null value. - * @param row - row key (arbitrary byte array) - * @param column Column with delimiter between family and qualifier - * @param ts - */ - public KeyValue(final String row, final String column, final long ts) { - this(row, column, ts, null); + public KeyValue(final byte [] row, final byte [] family, + final byte [] qualifier) { + this(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Put); } /** * Constructs KeyValue structure filled with null value. * @param row - row key (arbitrary byte array) - * @param column Column with delimiter between family and qualifier - * @param ts + * @param family family name + * @param qualifier column qualifier */ - public KeyValue(final byte [] row, final byte [] column, final long ts) { - this(row, column, ts, Type.Put); + public KeyValue(final byte [] row, final byte [] family, + final byte [] qualifier, final byte [] value) { + this(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Put, value); } /** - * Constructs KeyValue structure filled with specified value. - * @param row - row key (arbitrary byte array) - * @param column Column with delimiter between family and qualifier - * @param timestamp - * @param value + * Constructs KeyValue structure filled with specified values. + * @param row row key + * @param family family name + * @param qualifier column qualifier + * @param timestamp version timestamp + * @param type key type + * @throws IllegalArgumentException */ - public KeyValue(final String row, final String column, - final long timestamp, final byte [] value) { - this(Bytes.toBytes(row), - column == null? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes(column), - timestamp, value); + public KeyValue(final byte[] row, final byte[] family, + final byte[] qualifier, final long timestamp, Type type) { + this(row, family, qualifier, timestamp, type, null); + } + + /** + * Constructs KeyValue structure filled with specified values. + * @param row row key + * @param family family name + * @param qualifier column qualifier + * @param timestamp version timestamp + * @param value column value + * @throws IllegalArgumentException + */ + public KeyValue(final byte[] row, final byte[] family, + final byte[] qualifier, final long timestamp, final byte[] value) { + this(row, family, qualifier, timestamp, Type.Put, value); + } + + /** + * Constructs KeyValue structure filled with specified values. + * @param row row key + * @param family family name + * @param qualifier column qualifier + * @param timestamp version timestamp + * @param type key type + * @param value column value + * @throws IllegalArgumentException + */ + 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); + } + + /** + * Constructs KeyValue structure filled with specified values. + * @param row row key + * @param family family name + * @param qualifier column qualifier + * @param qoffset qualifier offset + * @param qlength qualifier length + * @param timestamp version timestamp + * @param type key type + * @param value column value + * @param voffset value offset + * @param vlength value length + * @throws IllegalArgumentException + */ + public KeyValue(byte [] row, byte [] family, + byte [] qualifier, int qoffset, int qlength, long timestamp, Type type, + byte [] value, int voffset, int vlength) { + this(row, 0, row==null ? 0 : row.length, + family, 0, family==null ? 0 : family.length, + qualifier, qoffset, qlength, timestamp, type, + value, voffset, vlength); } /** - * Constructs KeyValue structure filled with specified value. - * @param row - row key (arbitrary byte array) - * @param column Column with delimiter between family and qualifier - * @param timestamp - * @param value - */ - public KeyValue(final byte [] row, final byte [] column, - final long timestamp, final byte [] value) { - this(row, column, timestamp, Type.Put, value); - } - - /** - * Constructs KeyValue structure filled with specified value. - * @param row - row key (arbitrary byte array) - * @param column Column with delimiter between family and qualifier - * @param timestamp - * @param type - * @param value - */ - public KeyValue(final String row, final String column, - final long timestamp, final Type type, final byte [] value) { - this(Bytes.toBytes(row), Bytes.toBytes(column), timestamp, type, - value); - } - - /** - * Constructs KeyValue structure filled with null value. - * @param row - row key (arbitrary byte array) - * @param column Column with delimiter between family and qualifier - * @param timestamp - * @param type - */ - public KeyValue(final byte [] row, final byte [] column, - final long timestamp, final Type type) { - this(row, 0, row.length, column, 0, column == null? 0: column.length, - timestamp, type, null, 0, -1); - } - - /** - * Constructs KeyValue structure filled with specified value. - * @param row - row key (arbitrary byte array) - * @param column Column with delimiter between family and qualifier - * @param timestamp - * @param type - * @param value - */ - public KeyValue(final byte [] row, final byte [] column, - final long timestamp, final Type type, final byte [] value) { - this(row, 0, row.length, column, 0, column == null? 0: column.length, - timestamp, type, value, 0, value == null? 0: value.length); - } - - /** - * Constructs KeyValue structure filled with specified value. - * @param row - row key (arbitrary byte array) - * @param roffset - * @param rlength - * @param column Column with delimiter between family and qualifier - * @param coffset Where to start reading the column. - * @param clength How long column is (including the family/qualifier delimiter. - * @param timestamp - * @param type - * @param value - * @param voffset - * @param vlength + * Constructs KeyValue structure filled with specified values. + *

+ * Column is split into two fields, family and qualifier. + * @param row row key + * @param roffset row offset + * @param rlength row length + * @param family family name + * @param foffset family offset + * @param flength family length + * @param qualifier column qualifier + * @param qoffset qualifier offset + * @param qlength qualifier length + * @param timestamp version timestamp + * @param type key type + * @param value column value + * @param voffset value offset + * @param vlength value length * @throws IllegalArgumentException */ public KeyValue(final byte [] row, final int roffset, final int rlength, - final byte [] column, final int coffset, int clength, + final byte [] family, final int foffset, final int flength, + final byte [] qualifier, final int qoffset, final int qlength, final long timestamp, final Type type, - final byte [] value, final int voffset, int vlength) { - this.bytes = createByteArray(row, roffset, rlength, column, coffset, - clength, timestamp, type, value, voffset, vlength); + final byte [] value, final int voffset, final int vlength) { + this.bytes = createByteArray(row, roffset, rlength, + family, foffset, flength, qualifier, qoffset, qlength, + timestamp, type, value, voffset, vlength); this.length = bytes.length; this.offset = 0; } /** * Write KeyValue format into a byte array. + * @param row row key + * @param roffset row offset + * @param rlength row length + * @param family family name + * @param foffset family offset + * @param flength family length + * @param qualifier column qualifier + * @param qoffset qualifier offset + * @param qlength qualifier length + * @param timestamp version timestamp + * @param type key type + * @param value column value + * @param voffset value offset + * @param vlength value length + * @return + */ + static byte [] createByteArray(final byte [] row, final int roffset, + final int rlength, final byte [] family, final int foffset, int flength, + final byte [] qualifier, final int qoffset, int qlength, + final long timestamp, final Type type, + final byte [] value, final int voffset, int vlength) { + if (rlength > Short.MAX_VALUE) { + throw new IllegalArgumentException("Row > " + Short.MAX_VALUE); + } + if (row == null) { + throw new IllegalArgumentException("Row is null"); + } + // Family length + flength = family == null ? 0 : flength; + if (flength > Byte.MAX_VALUE) { + throw new IllegalArgumentException("Family > " + Byte.MAX_VALUE); + } + // Qualifier length + qlength = qualifier == null ? 0 : qlength; + if (qlength > Integer.MAX_VALUE - rlength - flength) { + throw new IllegalArgumentException("Qualifier > " + Integer.MAX_VALUE); + } + // Key length + 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; + // Value length + vlength = value == null? 0 : vlength; + if (vlength > HConstants.MAXIMUM_VALUE_LENGTH) { + throw new IllegalArgumentException("Valuer > " + + HConstants.MAXIMUM_VALUE_LENGTH); + } + + // 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)); + if(flength != 0) { + pos = Bytes.putBytes(bytes, pos, family, foffset, flength); + } + if(qlength != 0) { + pos = Bytes.putBytes(bytes, pos, qualifier, qoffset, qlength); + } + pos = Bytes.putLong(bytes, pos, timestamp); + pos = Bytes.putByte(bytes, pos, type.getCode()); + if (value != null && value.length > 0) { + pos = Bytes.putBytes(bytes, pos, value, voffset, vlength); + } + return bytes; + } + + /** + * Write KeyValue format into a byte array. + *

+ * Takes column in the form family:qualifier * @param row - row key (arbitrary byte array) * @param roffset * @param rlength @@ -423,12 +525,6 @@ public class KeyValue implements Writable, HeapSize { final byte [] column, final int coffset, int clength, final long timestamp, final Type type, final byte [] value, final int voffset, int vlength) { - if (rlength > Short.MAX_VALUE) { - throw new IllegalArgumentException("Row > " + Short.MAX_VALUE); - } - if (row == null) { - throw new IllegalArgumentException("Row is null"); - } // If column is non-null, figure where the delimiter is at. int delimiteroffset = 0; if (column != null && column.length > 0) { @@ -436,41 +532,15 @@ public class KeyValue implements Writable, HeapSize { if (delimiteroffset > Byte.MAX_VALUE) { throw new IllegalArgumentException("Family > " + Byte.MAX_VALUE); } + } else { + return createByteArray(row,roffset,rlength,null,0,0,null,0,0,timestamp, + type,value,voffset,vlength); } - // Value length - vlength = value == null? 0: vlength; - // Column length - minus delimiter - clength = column == null || column.length == 0? 0: clength - 1; - long longkeylength = KEY_INFRASTRUCTURE_SIZE + rlength + clength; - 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); - // Write out column family length. - pos = Bytes.putByte(bytes, pos, (byte)(delimiteroffset & 0x0000ff)); - if (column != null && column.length != 0) { - // Write family. - pos = Bytes.putBytes(bytes, pos, column, coffset, delimiteroffset); - // Write qualifier. - delimiteroffset++; - pos = Bytes.putBytes(bytes, pos, column, coffset + delimiteroffset, - column.length - delimiteroffset); - } - pos = Bytes.putLong(bytes, pos, timestamp); - pos = Bytes.putByte(bytes, pos, type.getCode()); - if (value != null && value.length > 0) { - pos = Bytes.putBytes(bytes, pos, value, voffset, vlength); - } - return bytes; + int flength = delimiteroffset-coffset; + int qlength = clength - flength - 1; + return createByteArray(row, roffset, rlength, column, coffset, + flength, column, delimiteroffset+1, qlength, timestamp, type, + value, voffset, vlength); } // Needed doing 'contains' on List. Only compares the key portion, not the @@ -485,6 +555,12 @@ public class KeyValue implements Writable, HeapSize { return result; } + //--------------------------------------------------------------------------- + // + // KeyValue cloning + // + //--------------------------------------------------------------------------- + /** * @param timestamp * @return Clone of bb's key portion with only the row and timestamp filled in. @@ -492,15 +568,8 @@ public class KeyValue implements Writable, HeapSize { */ public KeyValue cloneRow(final long timestamp) { return new KeyValue(getBuffer(), getRowOffset(), getRowLength(), - null, 0, 0, timestamp, Type.codeToType(getType()), null, 0, 0); - } - - /** - * @return Clone of bb's key portion with type set to Type.Delete. - * @throws IOException - */ - public KeyValue cloneDelete() { - return createKey(Type.Delete); + null, 0, 0, null, 0, 0, + timestamp, Type.codeToType(getType()), null, 0, 0); } /** @@ -531,6 +600,12 @@ public class KeyValue implements Writable, HeapSize { return new KeyValue(other, 0, other.length); } + //--------------------------------------------------------------------------- + // + // String representation + // + //--------------------------------------------------------------------------- + public String toString() { return keyToString(this.bytes, this.offset + ROW_OFFSET, getKeyLength()) + "/vlen=" + getValueLength(); @@ -563,11 +638,20 @@ public class KeyValue implements Writable, HeapSize { columnlength - familylength); long timestamp = Bytes.toLong(b, o + (l - TIMESTAMP_TYPE_SIZE)); byte type = b[o + l - 1]; +// return row + "/" + family + +// (family != null && family.length() > 0? COLUMN_FAMILY_DELIMITER: "") + +// qualifier + "/" + timestamp + "/" + Type.codeToType(type); return row + "/" + family + - (family != null && family.length() > 0? COLUMN_FAMILY_DELIMITER: "") + + (family != null && family.length() > 0? ":" :"") + qualifier + "/" + timestamp + "/" + Type.codeToType(type); } + //--------------------------------------------------------------------------- + // + // Public Member Accessors + // + //--------------------------------------------------------------------------- + /** * @return The byte array backing this KeyValue. */ @@ -589,7 +673,13 @@ public class KeyValue implements Writable, HeapSize { return length; } - /* + //--------------------------------------------------------------------------- + // + // Length and Offset Calculators + // + //--------------------------------------------------------------------------- + + /** * Determines the total length of the KeyValue stored in the specified * byte array and offset. Includes all headers. * @param bytes byte array @@ -602,6 +692,162 @@ public class KeyValue implements Writable, HeapSize { Bytes.toInt(bytes, offset + Bytes.SIZEOF_INT); } + /** + * @return Key offset in backing buffer.. + */ + public int getKeyOffset() { + return this.offset + ROW_OFFSET; + } + + /** + * @return Length of key portion. + */ + public int getKeyLength() { + return Bytes.toInt(this.bytes, this.offset); + } + + /** + * @return Value offset + */ + public int getValueOffset() { + return getKeyOffset() + getKeyLength(); + } + + /** + * @return Value length + */ + public int getValueLength() { + return Bytes.toInt(this.bytes, this.offset + Bytes.SIZEOF_INT); + } + + /** + * @return Row offset + */ + public int getRowOffset() { + return getKeyOffset() + Bytes.SIZEOF_SHORT; + } + + /** + * @return Row length + */ + public short getRowLength() { + return Bytes.toShort(this.bytes, getKeyOffset()); + } + + /** + * @return Family offset + */ + public int getFamilyOffset() { + return getFamilyOffset(getRowLength()); + } + + /** + * @return Family offset + */ + public int getFamilyOffset(int rlength) { + return this.offset + ROW_OFFSET + Bytes.SIZEOF_SHORT + rlength + Bytes.SIZEOF_BYTE; + } + + /** + * @return Family length + */ + public byte getFamilyLength() { + return getFamilyLength(getFamilyOffset()); + } + + /** + * @return Family length + */ + public byte getFamilyLength(int foffset) { + return this.bytes[foffset-1]; + } + + /** + * @return Qualifier offset + */ + public int getQualifierOffset() { + return getQualifierOffset(getFamilyOffset()); + } + + /** + * @return Qualifier offset + */ + public int getQualifierOffset(int foffset) { + return foffset + getFamilyLength(foffset); + } + + /** + * @return Qualifier length + */ + public int getQualifierLength() { + return getQualifierLength(getRowLength(),getFamilyLength()); + } + + /** + * @return Qualifier length + */ + public int getQualifierLength(int rlength, int flength) { + return getKeyLength() - + (KEY_INFRASTRUCTURE_SIZE + rlength + flength); + } + + /** + * @return Column (family + qualifier) length + */ + public int getTotalColumnLength() { + int rlength = getRowLength(); + int foffset = getFamilyOffset(rlength); + return getTotalColumnLength(rlength,foffset); + } + + /** + * @return Column (family + qualifier) length + */ + public int getTotalColumnLength(int rlength, int foffset) { + int flength = getFamilyLength(foffset); + int qlength = getQualifierLength(rlength,flength); + return flength + qlength; + } + + /** + * @return Timestamp offset + */ + public int getTimestampOffset() { + return getTimestampOffset(getKeyLength()); + } + + /** + * @param keylength Pass if you have it to save on a int creation. + * @return Timestamp offset + */ + public int getTimestampOffset(final int keylength) { + return getKeyOffset() + keylength - TIMESTAMP_TYPE_SIZE; + } + + /** + * @return True if this KeyValue has a LATEST_TIMESTAMP timestamp. + */ + public boolean isLatestTimestamp() { + return Bytes.compareTo(getBuffer(), getTimestampOffset(), Bytes.SIZEOF_LONG, + HConstants.LATEST_TIMESTAMP_BYTES, 0, Bytes.SIZEOF_LONG) == 0; + } + + public boolean updateLatestStamp(final byte [] now) { + int tsOffset = getTimestampOffset(); + if(Bytes.compareTo(now, 0, Bytes.SIZEOF_LONG, + this.bytes, tsOffset, Bytes.SIZEOF_LONG) < 0) { + System.arraycopy(now, 0, this.bytes, tsOffset, Bytes.SIZEOF_LONG); + return true; + } + return false; + } + + //--------------------------------------------------------------------------- + // + // Methods that return copies of fields + // + //--------------------------------------------------------------------------- + /** * @return Copy of the key portion only. Used compacting and testing. */ @@ -611,32 +857,20 @@ public class KeyValue implements Writable, HeapSize { System.arraycopy(getBuffer(), getKeyOffset(), key, 0, keylength); return key; } - - public String getKeyString() { - return Bytes.toString(getBuffer(), getKeyOffset(), getKeyLength()); - } - + /** - * @return Key offset in backing buffer.. + * Do not use unless you have to. Use {@link #getBuffer()} with appropriate + * offset and lengths instead. + * @return Value in a new byte array. */ - public int getKeyOffset() { - return this.offset + ROW_OFFSET; + public byte [] getValue() { + int o = getValueOffset(); + int l = getValueLength(); + byte [] result = new byte[l]; + System.arraycopy(getBuffer(), o, result, 0, l); + return result; } - - /** - * @return Row length. - */ - public short getRowLength() { - return Bytes.toShort(this.bytes, getKeyOffset()); - } - - /** - * @return Offset into backing buffer at which row starts. - */ - public int getRowOffset() { - return getKeyOffset() + Bytes.SIZEOF_SHORT; - } - + /** * Do not use this unless you have to. * Use {@link #getBuffer()} with appropriate offsets and lengths instead. @@ -666,25 +900,10 @@ public class KeyValue implements Writable, HeapSize { return Bytes.toLong(this.bytes, tsOffset); } - /** - * @param keylength Pass if you have it to save on a int creation. - * @return Offset into backing buffer at which timestamp starts. - */ - int getTimestampOffset(final int keylength) { - return getKeyOffset() + keylength - TIMESTAMP_TYPE_SIZE; - } - - /** - * @return True if a {@link Type#Delete}. - */ - public boolean isDeleteType() { - return getType() == Type.Delete.getCode(); - } - /** * @return Type of this KeyValue. */ - byte getType() { + public byte getType() { return getType(getKeyLength()); } @@ -697,99 +916,170 @@ public class KeyValue implements Writable, HeapSize { } /** - * @return Length of key portion. + * @return True if Delete KeyValue type. */ - public int getKeyLength() { - return Bytes.toInt(this.bytes, this.offset); + public boolean isDeleteType() { + return getType() == Type.Delete.code; } /** - * @return Value length + * @return True if DeleteColumn KeyValue type. */ - public int getValueLength() { - return Bytes.toInt(this.bytes, this.offset + Bytes.SIZEOF_INT); + public boolean isDeleteColumnType() { + return getType() == Type.DeleteColumn.code; } /** - * @return Offset into backing buffer at which value starts. + * Do not use this unless you have to. + * Use {@link #getBuffer()} with appropriate offsets and lengths instead. + * @return Returns column. Makes a copy. Inserts delimiter. */ - public int getValueOffset() { - return getKeyOffset() + getKeyLength(); - } - - /** - * Do not use unless you have to. Use {@link #getBuffer()} with appropriate - * offset and lengths instead. - * @return Value in a new byte array. - */ - public byte [] getValue() { - int o = getValueOffset(); - int l = getValueLength(); - byte [] result = new byte[l]; - System.arraycopy(getBuffer(), o, result, 0, l); + public byte [] getColumn() { + int fo = getFamilyOffset(); + int fl = getFamilyLength(fo); + int ql = getQualifierLength(); + byte [] result = new byte[fl + 1 + ql]; + System.arraycopy(this.bytes, fo, result, 0, fl); + result[fl] = COLUMN_FAMILY_DELIMITER; + System.arraycopy(this.bytes, fo + fl, result, + fl + 1, ql); return result; } /** - * @return Offset into backing buffer at which the column begins + * Do not use this unless you have to. + * Use {@link #getBuffer()} with appropriate offsets and lengths instead. + * @return Returns family. Makes a copy. */ - public int getColumnOffset() { - return getColumnOffset(getRowLength()); + public byte [] getFamily() { + int o = getFamilyOffset(); + int l = getFamilyLength(o); + byte [] result = new byte[l]; + System.arraycopy(this.bytes, o, result, 0, l); + return result; } /** - * @param rowlength - length of row. - * @return Offset into backing buffer at which the column begins + * Do not use this unless you have to. + * Use {@link #getBuffer()} with appropriate offsets and lengths instead. + * @return Returns qualifier. Makes a copy. */ - public int getColumnOffset(final int rowlength) { - return getRowOffset() + rowlength + 1; + public byte [] getQualifier() { + int o = getQualifierOffset(); + int l = getQualifierLength(); + byte [] result = new byte[l]; + System.arraycopy(this.bytes, o, result, 0, l); + return result; } + //--------------------------------------------------------------------------- + // + // KeyValue splitter + // + //--------------------------------------------------------------------------- + /** - * @param columnoffset Pass if you have it to save on an int creation. - * @return Length of family portion of column. + * Utility class that splits a KeyValue buffer into separate byte arrays. + *

+ * Should get rid of this if we can, but is very useful for debugging. */ - int getFamilyLength(final int columnoffset) { - return this.bytes[columnoffset - 1]; + public static class SplitKeyValue { + private byte [][] split; + SplitKeyValue() { + this.split = new byte[6][]; + } + public void setRow(byte [] value) { this.split[0] = value; } + public void setFamily(byte [] value) { this.split[1] = value; } + public void setQualifier(byte [] value) { this.split[2] = value; } + public void setTimestamp(byte [] value) { this.split[3] = value; } + public void setType(byte [] value) { this.split[4] = value; } + public void setValue(byte [] value) { this.split[5] = value; } + public byte [] getRow() { return this.split[0]; } + public byte [] getFamily() { return this.split[1]; } + public byte [] getQualifier() { return this.split[2]; } + public byte [] getTimestamp() { return this.split[3]; } + public byte [] getType() { return this.split[4]; } + public byte [] getValue() { return this.split[5]; } } - - /** - * @param columnoffset Pass if you have it to save on an int creation. - * @return Length of column. - */ - public int getColumnLength(final int columnoffset) { - return getColumnLength(columnoffset, getKeyLength()); + + public SplitKeyValue split() { + SplitKeyValue split = new SplitKeyValue(); + int splitOffset = this.offset; + int keyLen = Bytes.toInt(bytes, splitOffset); + splitOffset += Bytes.SIZEOF_INT; + int valLen = Bytes.toInt(bytes, splitOffset); + splitOffset += Bytes.SIZEOF_INT; + short rowLen = Bytes.toShort(bytes, splitOffset); + splitOffset += Bytes.SIZEOF_SHORT; + byte [] row = new byte[rowLen]; + System.arraycopy(bytes, splitOffset, row, 0, rowLen); + splitOffset += rowLen; + split.setRow(row); + byte famLen = bytes[splitOffset]; + splitOffset += Bytes.SIZEOF_BYTE; + byte [] family = new byte[famLen]; + System.arraycopy(bytes, splitOffset, family, 0, famLen); + splitOffset += famLen; + split.setFamily(family); + int colLen = keyLen - + (rowLen + famLen + Bytes.SIZEOF_SHORT + Bytes.SIZEOF_BYTE + + Bytes.SIZEOF_LONG + Bytes.SIZEOF_BYTE); + byte [] qualifier = new byte[colLen]; + System.arraycopy(bytes, splitOffset, qualifier, 0, colLen); + splitOffset += colLen; + split.setQualifier(qualifier); + byte [] timestamp = new byte[Bytes.SIZEOF_LONG]; + System.arraycopy(bytes, splitOffset, timestamp, 0, Bytes.SIZEOF_LONG); + splitOffset += Bytes.SIZEOF_LONG; + split.setTimestamp(timestamp); + byte [] type = new byte[1]; + type[0] = bytes[splitOffset]; + splitOffset += Bytes.SIZEOF_BYTE; + split.setType(type); + byte [] value = new byte[valLen]; + System.arraycopy(bytes, splitOffset, value, 0, valLen); + split.setValue(value); + return split; } - - int getColumnLength(final int columnoffset, final int keylength) { - return (keylength + ROW_OFFSET) - (columnoffset - this.offset) - - TIMESTAMP_TYPE_SIZE; - } - + + //--------------------------------------------------------------------------- + // + // Compare specified fields against those contained in this KeyValue + // + //--------------------------------------------------------------------------- + /** * @param family * @return True if matching families. */ public boolean matchingFamily(final byte [] family) { - int o = getColumnOffset(); - // Family length byte is just before the column starts. - int l = this.bytes[o - 1]; + int o = getFamilyOffset(); + int l = getFamilyLength(o); return Bytes.compareTo(family, 0, family.length, this.bytes, o, l) == 0; } + /** + * @param qualifier + * @return True if matching qualifiers. + */ + public boolean matchingQualifier(final byte [] qualifier) { + int o = getQualifierOffset(); + int l = getQualifierLength(); + return Bytes.compareTo(qualifier, 0, qualifier.length, + this.bytes, o, l) == 0; + } + /** * @param column Column minus its delimiter - * @param familylength Length of family in passed column * @return True if column matches. * @see #matchingColumn(byte[]) */ - public boolean matchingColumnNoDelimiter(final byte [] column, - final int familylength) { - int o = getColumnOffset(); - int l = getColumnLength(o); - int f = getFamilyLength(o); - return compareColumns(getBuffer(), o, l, f, - column, 0, column.length, familylength) == 0; + public boolean matchingColumnNoDelimiter(final byte [] column) { + int rl = getRowLength(); + int o = getFamilyOffset(rl); + int fl = getFamilyLength(o); + int l = fl + getQualifierLength(rl,fl); + return Bytes.compareTo(column, 0, column.length, this.bytes, o, l) == 0; } /** @@ -798,14 +1088,40 @@ public class KeyValue implements Writable, HeapSize { */ public boolean matchingColumn(final byte [] column) { int index = getFamilyDelimiterIndex(column, 0, column.length); - int o = getColumnOffset(); - int l = getColumnLength(o); - int result = Bytes.compareTo(getBuffer(), o, index, column, 0, index); - if (result != 0) { + int rl = getRowLength(); + int o = getFamilyOffset(rl); + int fl = getFamilyLength(o); + int ql = getQualifierLength(rl,fl); + if(Bytes.compareTo(column, 0, index, this.bytes, o, fl) != 0) { return false; } - return Bytes.compareTo(getBuffer(), o + index, l - index, - column, index + 1, column.length - (index + 1)) == 0; + return Bytes.compareTo(column, index + 1, column.length - (index + 1), + this.bytes, o + fl, ql) == 0; + } + + /** + * + * @param family column family + * @param qualifier column qualifier + * @return True if column matches + */ + public boolean matchingColumn(final byte[] family, final byte[] qualifier) { + int rl = getRowLength(); + int o = getFamilyOffset(rl); + int fl = getFamilyLength(o); + int ql = getQualifierLength(rl,fl); + if(Bytes.compareTo(family, 0, family.length, this.bytes, o, family.length) + != 0) { + return false; + } + if(qualifier == null || qualifier.length == 0) { + if(ql == 0) { + return true; + } + return false; + } + return Bytes.compareTo(qualifier, 0, qualifier.length, + this.bytes, o + fl, ql) == 0; } /** @@ -842,42 +1158,35 @@ public class KeyValue implements Writable, HeapSize { return getRowLength() > 0 && !isEmptyColumn(); } - /** - * @return Returns column String with delimiter added back. Expensive! - */ - public String getColumnString() { - int o = getColumnOffset(); - int l = getColumnLength(o); - int familylength = getFamilyLength(o); - return Bytes.toString(this.bytes, o, familylength) + - COLUMN_FAMILY_DELIMITER + Bytes.toString(this.bytes, - o + familylength, l - familylength); - } - - /** - * Do not use this unless you have to. - * Use {@link #getBuffer()} with appropriate offsets and lengths instead. - * @return Returns column. Makes a copy. Inserts delimiter. - */ - public byte [] getColumn() { - int o = getColumnOffset(); - int l = getColumnLength(o); - int familylength = getFamilyLength(o); - byte [] result = new byte[l + 1]; - System.arraycopy(getBuffer(), o, result, 0, familylength); - result[familylength] = COLUMN_FAMILY_DELIMITER; - System.arraycopy(getBuffer(), o + familylength, result, - familylength + 1, l - familylength); - return result; - } - /** * @return True if column is empty. */ public boolean isEmptyColumn() { - return getColumnLength(getColumnOffset()) == 0; + return getQualifierLength() == 0; } + /** + * Splits a column in family:qualifier form into separate byte arrays. + *

+ * Catches + * @param c + * @return + */ + public static byte [][] parseColumn(byte [] c) { + final byte [][] result = new byte [2][]; + final int index = getFamilyDelimiterIndex(c, 0, c.length); + if (index == -1) { + throw new IllegalArgumentException("Impossible column name: " + c); + } + result[0] = new byte [index]; + System.arraycopy(c, 0, result[0], 0, index); + final int len = c.length - (index + 1); + result[1] = new byte[len]; + System.arraycopy(c, index + 1 /*Skip delimiter*/, result[1], 0, + len); + return result; + } + /** * @param b * @return Index of the family-qualifier colon delimiter character in passed @@ -1026,7 +1335,8 @@ public class KeyValue implements Writable, HeapSize { * @return Result comparing rows. */ public int compareRows(final KeyValue left, final KeyValue right) { - return compareRows(left, left.getRowLength(), right, right.getRowLength()); + return compareRows(left, left.getRowLength(), right, + right.getRowLength()); } /** @@ -1058,28 +1368,27 @@ public class KeyValue implements Writable, HeapSize { return getRawComparator().compareRows(left, loffset, llength, right, roffset, rlength); } - + public int compareColumns(final KeyValue left, final byte [] right, final int roffset, final int rlength, final int rfamilyoffset) { - int offset = left.getColumnOffset(); - int length = left.getColumnLength(offset); + int offset = left.getFamilyOffset(); + int length = left.getFamilyLength() + left.getQualifierLength(); return getRawComparator().compareColumns(left.getBuffer(), offset, length, left.getFamilyLength(offset), right, roffset, rlength, rfamilyoffset); } int compareColumns(final KeyValue left, final short lrowlength, - final int lkeylength, final KeyValue right, final short rrowlength, - final int rkeylength) { - int loffset = left.getColumnOffset(lrowlength); - int roffset = right.getColumnOffset(rrowlength); - int llength = left.getColumnLength(loffset, lkeylength); - int rlength = right.getColumnLength(roffset, rkeylength); - int lfamilylength = left.getFamilyLength(loffset); - int rfamilylength = right.getFamilyLength(roffset); - return getRawComparator().compareColumns(left.getBuffer(), loffset, - llength, lfamilylength, - right.getBuffer(), roffset, rlength, rfamilylength); + final KeyValue right, final short rrowlength) { + int lfoffset = left.getFamilyOffset(lrowlength); + int rfoffset = right.getFamilyOffset(rrowlength); + int lclength = left.getTotalColumnLength(lrowlength,lfoffset); + int rclength = right.getTotalColumnLength(rrowlength, rfoffset); + int lfamilylength = left.getFamilyLength(lfoffset); + int rfamilylength = right.getFamilyLength(rfoffset); + return getRawComparator().compareColumns(left.getBuffer(), lfoffset, + lclength, lfamilylength, + right.getBuffer(), rfoffset, rclength, rfamilylength); } /** @@ -1095,10 +1404,7 @@ public class KeyValue implements Writable, HeapSize { if (!matchingRows(left, lrowlength, right, rrowlength)) { return false; } - int lkeylength = left.getKeyLength(); - int rkeylength = right.getKeyLength(); - return compareColumns(left, lrowlength, lkeylength, - right, rrowlength, rkeylength) == 0; + return compareColumns(left, lrowlength, right, rrowlength) == 0; } /** @@ -1140,7 +1446,8 @@ public class KeyValue implements Writable, HeapSize { public boolean matchingRows(final byte [] left, final int loffset, final int llength, final byte [] right, final int roffset, final int rlength) { - int compare = compareRows(left, loffset, llength, right, roffset, rlength); + int compare = compareRows(left, loffset, llength, + right, roffset, rlength); if (compare != 0) { return false; } @@ -1214,19 +1521,43 @@ public class KeyValue implements Writable, HeapSize { */ public static KeyValue createFirstOnRow(final byte [] row, final long ts) { - return createFirstOnRow(row, null, ts); + return new KeyValue(row, null, null, ts, Type.Maximum); } /** * @param row - row key (arbitrary byte array) * @param ts - timestamp - * @return First possible key on passed row, column and timestamp. + * @return First possible key on passed row, column and timestamp */ public static KeyValue createFirstOnRow(final byte [] row, final byte [] c, final long ts) { - return new KeyValue(row, c, ts, Type.Maximum); + byte [][] split = parseColumn(c); + return new KeyValue(row, split[0], split[1], ts, Type.Maximum); } + /** + * @param row - row key (arbitrary byte array) + * @param f - family name + * @param q - column qualifier + * @return First possible key on passed row, and column. + */ + public static KeyValue createFirstOnRow(final byte [] row, final byte [] f, + final byte [] q) { + return new KeyValue(row, f, q, HConstants.LATEST_TIMESTAMP, Type.Maximum); + } + + /** + * @param row - row key (arbitrary byte array) + * @param f - family name + * @param q - column qualifier + * @param ts - timestamp + * @return First possible key on passed row, column and timestamp + */ + public static KeyValue createFirstOnRow(final byte [] row, final byte [] f, + final byte [] q, final long ts) { + return new KeyValue(row, f, q, ts, Type.Maximum); + } + /** * @param b * @param o @@ -1255,7 +1586,8 @@ public class KeyValue implements Writable, HeapSize { // "---" + Bytes.toString(right, roffset, rlength)); final int metalength = 7; // '.META.' length int lmetaOffsetPlusDelimiter = loffset + metalength; - int leftFarDelimiter = getDelimiterInReverse(left, lmetaOffsetPlusDelimiter, + int leftFarDelimiter = getDelimiterInReverse(left, + lmetaOffsetPlusDelimiter, llength - metalength, HRegionInfo.DELIMITER); int rmetaOffsetPlusDelimiter = roffset + metalength; int rightFarDelimiter = getDelimiterInReverse(right, @@ -1363,6 +1695,9 @@ public class KeyValue implements Writable, HeapSize { return compare; } + // if row matches, and no column in the 'left' AND put type is 'minimum', + // then return that left is larger than right. + // 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; @@ -1370,12 +1705,24 @@ public class KeyValue implements Writable, HeapSize { (lcolumnoffset - loffset); int rcolumnlength = rlength - TIMESTAMP_TYPE_SIZE - (rcolumnoffset - roffset); + + // This supports 'last key on a row' - the magic is if there is no column in the + // left operand, and the left operand has a type of '0' - magical value, + // then we say the left is bigger. This will let us seek to the last key in + // a row. + + byte ltype = left[loffset + (llength - 1)]; + + if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) { + return 1; // left is bigger. + } + compare = Bytes.compareTo(left, lcolumnoffset, lcolumnlength, right, rcolumnoffset, rcolumnlength); if (compare != 0) { return compare; } - + if (!this.ignoreTimestamp) { // Get timestamps. long ltimestamp = Bytes.toLong(left, @@ -1391,7 +1738,8 @@ public class KeyValue implements Writable, HeapSize { 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)]; + + // ltype is defined above byte rtype = right[roffset + (rlength - 1)]; return (0xff & rtype) - (0xff & ltype); } @@ -1402,7 +1750,7 @@ public class KeyValue implements Writable, HeapSize { return compare(left, 0, left.length, right, 0, right.length); } - protected int compareRows(byte [] left, int loffset, int llength, + public int compareRows(byte [] left, int loffset, int llength, byte [] right, int roffset, int rlength) { return Bytes.compareTo(left, loffset, llength, right, roffset, rlength); } @@ -1430,7 +1778,9 @@ public class KeyValue implements Writable, HeapSize { // HeapSize public long heapSize() { - return this.length; + int dataLen = bytes.length + (bytes.length % 8); + return HeapSize.OBJECT + HeapSize.BYTE_ARRAY + dataLen + + (2 * HeapSize.INT); } // Writable diff --git a/src/java/org/apache/hadoop/hbase/RegionHistorian.java b/src/java/org/apache/hadoop/hbase/RegionHistorian.java index df08ce74d09..2a1bdfa21db 100644 --- a/src/java/org/apache/hadoop/hbase/RegionHistorian.java +++ b/src/java/org/apache/hadoop/hbase/RegionHistorian.java @@ -29,8 +29,10 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.util.Bytes; @@ -57,17 +59,17 @@ public class RegionHistorian implements HConstants { "EEE, d MMM yyyy HH:mm:ss"); - private static enum HistorianColumnKey { - REGION_CREATION ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"creation")), - REGION_OPEN ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"open")), - REGION_SPLIT ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"split")), - REGION_COMPACTION ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"compaction")), - REGION_FLUSH ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"flush")), - REGION_ASSIGNMENT ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"assignment")); + private static enum HistorianQualifierKey { + REGION_CREATION ( Bytes.toBytes("creation")), + REGION_OPEN ( Bytes.toBytes("open")), + REGION_SPLIT ( Bytes.toBytes("split")), + REGION_COMPACTION ( Bytes.toBytes("compaction")), + REGION_FLUSH ( Bytes.toBytes("flush")), + REGION_ASSIGNMENT ( Bytes.toBytes("assignment")); - byte[] key; + byte[] key; - HistorianColumnKey(byte[] key) { + HistorianQualifierKey(byte[] key) { this.key = key; } } @@ -113,15 +115,17 @@ public class RegionHistorian implements HConstants { * moment to retrieve all version and to have the column key information. * To be changed when HTable.getRow handles versions. */ - for (HistorianColumnKey keyEnu : HistorianColumnKey.values()) { + for (HistorianQualifierKey keyEnu : HistorianQualifierKey.values()) { byte[] columnKey = keyEnu.key; - Cell[] cells = this.metaTable.get(Bytes.toBytes(regionName), - columnKey, ALL_VERSIONS); - if (cells != null) { - for (Cell cell : cells) { - informations.add(historian.new RegionHistoryInformation(cell - .getTimestamp(), Bytes.toString(columnKey).split(":")[1], Bytes - .toString(cell.getValue()))); + Get get = new Get(Bytes.toBytes(regionName)); + get.addColumn(CATALOG_HISTORIAN_FAMILY, columnKey); + get.setMaxVersions(ALL_VERSIONS); + Result result = this.metaTable.get(get); + + if (result != null) { + for(KeyValue kv : result.raw()) { + informations.add(historian.new RegionHistoryInformation( + kv.getTimestamp(), columnKey, kv.getValue())); } } } @@ -138,7 +142,7 @@ public class RegionHistorian implements HConstants { * @param serverName */ public void addRegionAssignment(HRegionInfo info, String serverName) { - add(HistorianColumnKey.REGION_ASSIGNMENT.key, "Region assigned to server " + add(HistorianQualifierKey.REGION_ASSIGNMENT.key, "Region assigned to server " + serverName, info); } @@ -147,7 +151,7 @@ public class RegionHistorian implements HConstants { * @param info */ public void addRegionCreation(HRegionInfo info) { - add(HistorianColumnKey.REGION_CREATION.key, "Region creation", info); + add(HistorianQualifierKey.REGION_CREATION.key, "Region creation", info); } /** @@ -156,7 +160,7 @@ public class RegionHistorian implements HConstants { * @param address */ public void addRegionOpen(HRegionInfo info, HServerAddress address) { - add(HistorianColumnKey.REGION_OPEN.key, "Region opened on server : " + add(HistorianQualifierKey.REGION_OPEN.key, "Region opened on server : " + address.getHostname(), info); } @@ -171,7 +175,7 @@ public class RegionHistorian implements HConstants { HRegionInfo newInfo2) { HRegionInfo[] infos = new HRegionInfo[] { newInfo1, newInfo2 }; for (HRegionInfo info : infos) { - add(HistorianColumnKey.REGION_SPLIT.key, SPLIT_PREFIX + + add(HistorianQualifierKey.REGION_SPLIT.key, SPLIT_PREFIX + oldInfo.getRegionNameAsString(), info); } } @@ -188,7 +192,7 @@ public class RegionHistorian implements HConstants { // such danger compacting; compactions are not allowed when // Flusher#flushSomeRegions is run. if (LOG.isDebugEnabled()) { - add(HistorianColumnKey.REGION_COMPACTION.key, + add(HistorianQualifierKey.REGION_COMPACTION.key, "Region compaction completed in " + timeTaken, info); } } @@ -211,9 +215,8 @@ public class RegionHistorian implements HConstants { * @param text * @param info */ - private void add(byte[] column, - String text, HRegionInfo info) { - add(column, text, info, LATEST_TIMESTAMP); + private void add(byte [] qualifier, String text, HRegionInfo info) { + add(qualifier, text, info, LATEST_TIMESTAMP); } /** @@ -223,18 +226,19 @@ public class RegionHistorian implements HConstants { * @param info * @param timestamp */ - private void add(byte[] column, - String text, HRegionInfo info, long timestamp) { + private void add(byte [] qualifier, String text, HRegionInfo info, + long timestamp) { if (!isOnline()) { // Its a noop return; } if (!info.isMetaRegion()) { - BatchUpdate batch = new BatchUpdate(info.getRegionName()); - batch.setTimestamp(timestamp); - batch.put(column, Bytes.toBytes(text)); + Put put = new Put(info.getRegionName()); + put.setTimeStamp(timestamp); + put.add(HConstants.CATALOG_HISTORIAN_FAMILY, qualifier, + Bytes.toBytes(text)); try { - this.metaTable.commit(batch); + this.metaTable.put(put); } catch (IOException ioe) { LOG.warn("Unable to '" + text + "'", ioe); } @@ -252,34 +256,35 @@ public class RegionHistorian implements HConstants { private long timestamp; - private String event; + private byte [] event = null; - private String description; + private byte [] description = null; /** * @param timestamp * @param event * @param description */ - public RegionHistoryInformation(long timestamp, String event, - String description) { + public RegionHistoryInformation(long timestamp, byte [] event, + byte [] description) { this.timestamp = timestamp; this.event = event; this.description = description; } - + + public int compareTo(RegionHistoryInformation otherInfo) { return -1 * Long.valueOf(timestamp).compareTo(otherInfo.getTimestamp()); } /** @return the event */ public String getEvent() { - return event; + return Bytes.toString(event); } /** @return the description */ public String getDescription() { - return description; + return Bytes.toString(description); } /** @return the timestamp */ diff --git a/src/java/org/apache/hadoop/hbase/WritableComparator.java b/src/java/org/apache/hadoop/hbase/WritableComparator.java index b765d681e8a..8faa1894e4c 100644 --- a/src/java/org/apache/hadoop/hbase/WritableComparator.java +++ b/src/java/org/apache/hadoop/hbase/WritableComparator.java @@ -23,6 +23,7 @@ import java.util.Comparator; import org.apache.hadoop.io.Writable; -public interface WritableComparator extends Writable, Comparator { -// No methods, just bring the two interfaces together -} +/** + * Interface that brings writable and comparable together + */ +public interface WritableComparator extends Writable, Comparator {} diff --git a/src/java/org/apache/hadoop/hbase/client/Delete.java b/src/java/org/apache/hadoop/hbase/client/Delete.java new file mode 100644 index 00000000000..f3028235aba --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/client/Delete.java @@ -0,0 +1,312 @@ +/* + * 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.client; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +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; + +/** + * Used to perform Delete operations on a single row. + *

+ * To delete an entire row, instantiate a Delete object with the row + * to delete. To further define the scope of what to delete, perform + * additional methods as outlined below. + *

+ * To delete specific families, execute {@link #deleteFamily(byte []) deleteFamily} + * for each family to delete. + *

+ * To delete multiple versions of specific columns, execute + * {@link #deleteColumns(byte [],byte []) deleteColumns} + * for each column to delete. + *

+ * To delete specific versions of specific columns, execute + * {@link #deleteColumn(byte [],byte [],long) deleteColumn} + * for each column version to delete. + *

+ * Specifying timestamps calling constructor, deleteFamily, and deleteColumns + * will delete all versions with a timestamp less than or equal to that + * specified. Specifying a timestamp to deleteColumn will delete versions + * only with a timestamp equal to that specified. + *

The timestamp passed to the constructor is only used ONLY for delete of + * rows. For anything less -- a deleteColumn, deleteColumns or + * deleteFamily -- then you need to use the method overrides that take a + * timestamp. The constructor timestamp is not referenced. + */ +public class Delete implements Writable { + private byte [] row = null; + // This ts is only used when doing a deleteRow. Anything less, + private long ts; + private long lockId = -1L; + private final Map> familyMap = + new TreeMap>(Bytes.BYTES_COMPARATOR); + + /** Constructor for Writable. DO NOT USE */ + public Delete() { + this(null); + } + + /** + * Create a Delete operation for the specified row. + *

+ * If no further operations are done, this will delete everything + * associated with the specified row (all versions of all columns in all + * families). + * @param row row key + */ + public Delete(byte [] row) { + this(row, HConstants.LATEST_TIMESTAMP, null); + } + + /** + * Create a Delete operation for the specified row and timestamp, using + * an optional row lock. + *

+ * If no further operations are done, this will delete all columns in all + * families of the specified row with a timestamp less than or equal to the + * specified timestamp. + * @param row row key + * @param timestamp maximum version timestamp + * @param rowLock previously acquired row lock, or null + */ + public Delete(byte [] row, long timestamp, RowLock rowLock) { + this.row = row; + this.ts = timestamp; + if (rowLock != null) { + this.lockId = rowLock.getLockId(); + } + } + + /** + * Delete all versions of all columns of the specified family. + *

+ * Overrides previous calls to deleteColumn and deleteColumns for the + * specified family. + * @param family family name + */ + public void deleteFamily(byte [] family) { + this.deleteFamily(family, HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete all columns of the specified family with a timestamp less than + * or equal to the specified timestamp. + *

+ * Overrides previous calls to deleteColumn and deleteColumns for the + * specified family. + * @param family family name + * @param timestamp maximum version timestamp + */ + public void deleteFamily(byte [] family, long timestamp) { + List list = familyMap.get(family); + if(list == null) { + list = new ArrayList(); + } else if(!list.isEmpty()) { + list.clear(); + } + list.add(new KeyValue(row, family, null, timestamp, KeyValue.Type.DeleteFamily)); + familyMap.put(family, list); + } + + /** + * Delete all versions of the specified column. + * @param family family name + * @param qualifier column qualifier + */ + public void deleteColumns(byte [] family, byte [] qualifier) { + this.deleteColumns(family, qualifier, HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete all versions of the specified column with a timestamp less than + * or equal to the specified timestamp. + * @param family family name + * @param qualifier column qualifier + * @param timestamp maximum version timestamp + */ + public void deleteColumns(byte [] family, byte [] qualifier, long timestamp) { + List list = familyMap.get(family); + if (list == null) { + list = new ArrayList(); + } + list.add(new KeyValue(this.row, family, qualifier, timestamp, + KeyValue.Type.DeleteColumn)); + familyMap.put(family, list); + } + + /** + * Delete the latest version of the specified column. + * This is an expensive call in that on the server-side, it first does a + * get to find the latest versions timestamp. Then it adds a delete using + * the fetched cells timestamp. + * @param family family name + * @param qualifier column qualifier + */ + public void deleteColumn(byte [] family, byte [] qualifier) { + this.deleteColumn(family, qualifier, HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete the specified version of the specified column. + * @param family family name + * @param qualifier column qualifier + * @param timestamp version timestamp + */ + public void deleteColumn(byte [] family, byte [] qualifier, long timestamp) { + List list = familyMap.get(family); + if(list == null) { + list = new ArrayList(); + } + list.add(new KeyValue( + this.row, family, qualifier, timestamp, KeyValue.Type.Delete)); + familyMap.put(family, list); + } + + /** + * Delete the latest version of the specified column, given in + * family:qualifier notation. + * @param column colon-delimited family and qualifier + */ + public void deleteColumn(byte [] column) { + byte [][] parts = KeyValue.parseColumn(column); + this.deleteColumn(parts[0], parts[1], HConstants.LATEST_TIMESTAMP); + } + + /** + * Method for retrieving the delete's familyMap + * @return familyMap + */ + public Map> getFamilyMap() { + return this.familyMap; + } + + /** + * Method for retrieving the delete's row + * @return row + */ + public byte [] getRow() { + return this.row; + } + + /** + * Method for retrieving the delete's RowLock + * @return RowLock + */ + public RowLock getRowLock() { + return new RowLock(this.row, this.lockId); + } + + /** + * Method for retrieving the delete's lockId + * @return + */ + public long getLockId() { + return this.lockId; + } + + /** + * Method for retrieving the delete's timestamp + * @return timestamp + */ + public long getTimeStamp() { + return this.ts; + } + + /** + * @return string + */ + @Override + public String toString() { + StringBuffer sb = new StringBuffer(); + sb.append("row="); + sb.append(Bytes.toString(this.row)); + sb.append(", ts="); + sb.append(this.ts); + sb.append(", families={"); + boolean moreThanOne = false; + for(Map.Entry> entry : this.familyMap.entrySet()) { + if(moreThanOne) { + sb.append(", "); + } else { + moreThanOne = true; + } + sb.append("(family="); + sb.append(Bytes.toString(entry.getKey())); + sb.append(", keyvalues=("); + boolean moreThanOneB = false; + for(KeyValue kv : entry.getValue()) { + if(moreThanOneB) { + sb.append(", "); + } else { + moreThanOneB = true; + } + sb.append(kv.toString()); + } + sb.append(")"); + } + sb.append("}"); + return sb.toString(); + } + + //Writable + public void readFields(final DataInput in) throws IOException { + this.row = Bytes.readByteArray(in); + this.ts = in.readLong(); + this.lockId = in.readLong(); + this.familyMap.clear(); + int numFamilies = in.readInt(); + for(int i=0;i list = new ArrayList(numColumns); + for(int j=0;j> entry : familyMap.entrySet()) { + Bytes.writeByteArray(out, entry.getKey()); + List list = entry.getValue(); + out.writeInt(list.size()); + for(KeyValue kv : list) { + kv.write(out); + } + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/client/Get.java b/src/java/org/apache/hadoop/hbase/client/Get.java new file mode 100644 index 00000000000..1c6000c1f9a --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/client/Get.java @@ -0,0 +1,398 @@ +/** + * 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.client; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.io.HbaseObjectWritable; +import org.apache.hadoop.hbase.io.TimeRange; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.Writable; + +/** + * Used to perform Get operations on a single row. + *

+ * To get everything for a row, instantiate a Get object with the row to get. + * To further define the scope of what to get, perform additional methods as + * outlined below. + *

+ * To get all columns from specific families, execute {@link #addFamily(byte[]) addFamily} + * for each family to retrieve. + *

+ * To get specific columns, execute {@link #addColumn(byte[], byte[]) addColumn} + * for each column to retrieve. + *

+ * To only retrieve columns within a specific range of version timestamps, + * execute {@link #setTimeRange(long, long) setTimeRange}. + *

+ * To only retrieve columns with a specific timestamp, execute + * {@link #setTimeStamp(long) setTimestamp}. + *

+ * To limit the number of versions of each column to be returned, execute + * {@link #setMaxVersions(int) setMaxVersions}. + *

+ * To add a filter, execute {@link #setFilter(RowFilterInterface) setFilter}. + */ +public class Get implements Writable { + private byte [] row = null; + private long lockId = -1L; + private int maxVersions = 1; + private RowFilterInterface filter = null; + private TimeRange tr = new TimeRange(); + private Map> familyMap = + new TreeMap>(Bytes.BYTES_COMPARATOR); + + /** Constructor for Writable. DO NOT USE */ + public Get() {} + + /** + * Create a Get operation for the specified row. + *

+ * If no further operations are done, this will get the latest version of + * all columns in all families of the specified row. + * @param row row key + */ + public Get(byte [] row) { + this(row, null); + } + + /** + * Create a Get operation for the specified row, using an existing row lock. + *

+ * If no further operations are done, this will get the latest version of + * all columns in all families of the specified row. + * @param row row key + * @param rowLock previously acquired row lock, or null + */ + public Get(byte [] row, RowLock rowLock) { + this.row = row; + if(rowLock != null) { + this.lockId = rowLock.getLockId(); + } + } + + /** + * Get all columns from the specified family. + *

+ * Overrides previous calls to addColumn for this family. + * @param family family name + * @return the Get object + */ + public Get addFamily(byte [] family) { + familyMap.remove(family); + familyMap.put(family, null); + return this; + } + + /** + * Get the column from the specific family with the specified qualifier. + *

+ * Overrides previous calls to addFamily for this family. + * @param family family name + * @param qualifier column qualifier + * @return the Get objec + */ + public Get addColumn(byte [] family, byte [] qualifier) { + NavigableSet set = familyMap.get(family); + if(set == null) { + set = new TreeSet(Bytes.BYTES_COMPARATOR); + } + set.add(qualifier); + familyMap.put(family, set); + return this; + } + + /** + * Adds an array of columns specified the old format, family:qualifier. + *

+ * Overrides previous calls to addFamily for any families in the input. + * @param columns array of columns, formatted as

family:qualifier
+ */ + public Get addColumns(byte [][] columns) { + if (columns == null) return this; + for(int i = 0; i < columns.length; i++) { + try { + addColumn(columns[i]); + } catch(Exception e) {} + } + return this; + } + + /** + * @param column Old format column. + * @return This. + */ + public Get addColumn(final byte [] column) { + if (column == null) return this; + byte [][] split = KeyValue.parseColumn(column); + addColumn(split[0], split[1]); + return this; + } + + /** + * Get versions of columns only within the specified timestamp range, + * [minStamp, maxStamp). + * @param minStamp minimum timestamp value, inclusive + * @param maxStamp maximum timestamp value, exclusive + * @throws IOException if invalid time range + */ + public Get setTimeRange(long minStamp, long maxStamp) + throws IOException { + tr = new TimeRange(minStamp, maxStamp); + return this; + } + + /** + * Get versions of columns with the specified timestamp. + * @param timestamp version timestamp + */ + public Get setTimeStamp(long timestamp) { + try { + tr = new TimeRange(timestamp, timestamp+1); + } catch(IOException e) { + // Will never happen + } + return this; + } + + /** + * Get all available versions. + */ + public Get setMaxVersions() { + this.maxVersions = Integer.MAX_VALUE; + return this; + } + + /** + * Get up to the specified number of versions of each column. + * @param maxVersions maximum versions for each column + * @throws IOException if invalid number of versions + */ + public Get setMaxVersions(int maxVersions) throws IOException { + if(maxVersions <= 0) { + throw new IOException("maxVersions must be positive"); + } + this.maxVersions = maxVersions; + return this; + } + + /** + * Apply the specified server-side filter when performing the Get. + * @param filter filter to run on the server + */ + public Get setFilter(RowFilterInterface filter) { + this.filter = filter; + return this; + } + + /** Accessors */ + + /** + * Method for retrieving the get's row + * @return row + */ + public byte [] getRow() { + return this.row; + } + + /** + * Method for retrieving the get's RowLock + * @return RowLock + */ + public RowLock getRowLock() { + return new RowLock(this.row, this.lockId); + } + + /** + * Method for retrieving the get's lockId + * @return lockId + */ + public long getLockId() { + return this.lockId; + } + + /** + * Method for retrieving the get's maximum number of version + * @return the maximum number of version to fetch for this get + */ + public int getMaxVersions() { + return this.maxVersions; + } + + /** + * Method for retrieving the get's TimeRange + * @return timeRange + */ + public TimeRange getTimeRange() { + return this.tr; + } + + /** + * Method for retrieving the keys in the familyMap + * @return keys in the current familyMap + */ + public Set familySet() { + return this.familyMap.keySet(); + } + + /** + * Method for retrieving the number of families to get from + * @return number of families + */ + public int numFamilies() { + return this.familyMap.size(); + } + + /** + * Method for checking if any families have been inserted into this Get + * @return true if familyMap is non empty false otherwise + */ + public boolean hasFamilies() { + return !this.familyMap.isEmpty(); + } + + /** + * Method for retrieving the get's familyMap + * @return familyMap + */ + public Map> getFamilyMap() { + return this.familyMap; + } + + /** + * @return String + */ + @Override + public String toString() { + StringBuffer sb = new StringBuffer(); + sb.append("row="); + sb.append(Bytes.toString(this.row)); + sb.append(", maxVersions="); + sb.append("" + this.maxVersions); + sb.append(", timeRange="); + sb.append("[" + this.tr.getMin() + "," + this.tr.getMax() + ")"); + sb.append(", families="); + if(this.familyMap.size() == 0) { + sb.append("ALL"); + return sb.toString(); + } + boolean moreThanOne = false; + for(Map.Entry> entry : + this.familyMap.entrySet()) { + if(moreThanOne) { + sb.append("), "); + } else { + moreThanOne = true; + sb.append("{"); + } + sb.append("(family="); + sb.append(Bytes.toString(entry.getKey())); + sb.append(", columns="); + if(entry.getValue() == null) { + sb.append("ALL"); + } else { + sb.append("{"); + boolean moreThanOneB = false; + for(byte [] column : entry.getValue()) { + if(moreThanOneB) { + sb.append(", "); + } else { + moreThanOneB = true; + } + sb.append(Bytes.toString(column)); + } + sb.append("}"); + } + } + sb.append("}"); + return sb.toString(); + } + + //Writable + public void readFields(final DataInput in) + throws IOException { + this.row = Bytes.readByteArray(in); + this.lockId = in.readLong(); + this.maxVersions = in.readInt(); + boolean hasFilter = in.readBoolean(); + if(hasFilter) { + this.filter = + (RowFilterInterface)HbaseObjectWritable.readObject(in, null); + } + this.tr = new TimeRange(); + tr.readFields(in); + int numFamilies = in.readInt(); + this.familyMap = + new TreeMap>(Bytes.BYTES_COMPARATOR); + for(int i=0; i set = null; + if(hasColumns) { + int numColumns = in.readInt(); + set = new TreeSet(Bytes.BYTES_COMPARATOR); + for(int j=0; j> entry : + familyMap.entrySet()) { + Bytes.writeByteArray(out, entry.getKey()); + NavigableSet columnSet = entry.getValue(); + if(columnSet == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeInt(columnSet.size()); + for(byte [] qualifier : columnSet) { + Bytes.writeByteArray(out, qualifier); + } + } + } + } +} diff --git a/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index c581a8815a7..5c8ae9e1bec 100644 --- a/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import java.util.Map; +import java.util.NavigableMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -34,8 +35,9 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.RegionException; import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.ipc.HMasterInterface; import org.apache.hadoop.hbase.ipc.HRegionInterface; @@ -51,7 +53,8 @@ import org.apache.hadoop.ipc.RemoteException; */ public class HBaseAdmin { private final Log LOG = LogFactory.getLog(this.getClass().getName()); - private final HConnection connection; +// private final HConnection connection; + final HConnection connection; private volatile HBaseConfiguration conf; private final long pause; private final int numRetries; @@ -121,11 +124,13 @@ public class HBaseAdmin { return this.connection.listTables(); } - public HTableDescriptor getTableDescriptor(final String tableName) - throws IOException { - return getTableDescriptor(Bytes.toBytes(tableName)); - } + /** + * Method for getting the tableDescriptor + * @param tableName as a byte [] + * @return the tableDescriptor + * @throws IOException + */ public HTableDescriptor getTableDescriptor(final byte [] tableName) throws IOException { return this.connection.getHTableDescriptor(tableName); @@ -238,19 +243,22 @@ public class HBaseAdmin { for (int tries = 0; tries < numRetries; tries++) { long scannerId = -1L; try { - scannerId = - server.openScanner(firstMetaServer.getRegionInfo().getRegionName(), - HConstants.COL_REGIONINFO_ARRAY, tableName, - HConstants.LATEST_TIMESTAMP, null); - RowResult values = server.next(scannerId); + Scan scan = new Scan().addColumn(HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER); + + scannerId = server.openScanner( + firstMetaServer.getRegionInfo().getRegionName(), + scan); + Result values = server.next(scannerId); if (values == null || values.size() == 0) { break; } boolean found = false; - for (Map.Entry e: values.entrySet()) { - if (Bytes.equals(e.getKey(), HConstants.COL_REGIONINFO)) { + NavigableMap infoValues = values.getFamilyMap(HConstants.CATALOG_FAMILY); + for (Map.Entry e: infoValues.entrySet()) { + if (Bytes.equals(e.getKey(), HConstants.REGIONINFO_QUALIFIER)) { info = (HRegionInfo) Writables.getWritable( - e.getValue().getValue(), info); + e.getValue(), info); if (Bytes.equals(info.getTableDesc().getName(), tableName)) { found = true; @@ -566,7 +574,7 @@ public class HBaseAdmin { newargs[i + xtraArgsCount] = args[i]; } } - modifyTable(HConstants.META_TABLE_NAME, HConstants.MODIFY_CLOSE_REGION, + modifyTable(HConstants.META_TABLE_NAME, HConstants.Modify.CLOSE_REGION, newargs); } @@ -589,7 +597,7 @@ public class HBaseAdmin { * @throws IOException */ public void flush(final byte [] tableNameOrRegionName) throws IOException { - modifyTable(tableNameOrRegionName, HConstants.MODIFY_TABLE_FLUSH); + modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_FLUSH); } /** @@ -611,7 +619,7 @@ public class HBaseAdmin { * @throws IOException */ public void compact(final byte [] tableNameOrRegionName) throws IOException { - modifyTable(tableNameOrRegionName, HConstants.MODIFY_TABLE_COMPACT); + modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_COMPACT); } /** @@ -635,7 +643,7 @@ public class HBaseAdmin { */ public void majorCompact(final byte [] tableNameOrRegionName) throws IOException { - modifyTable(tableNameOrRegionName, HConstants.MODIFY_TABLE_MAJOR_COMPACT); + modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_MAJOR_COMPACT); } /** @@ -657,7 +665,7 @@ public class HBaseAdmin { * @throws IOException */ public void split(final byte [] tableNameOrRegionName) throws IOException { - modifyTable(tableNameOrRegionName, HConstants.MODIFY_TABLE_SPLIT); + modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_SPLIT); } /* @@ -667,7 +675,8 @@ public class HBaseAdmin { * @param op * @throws IOException */ - private void modifyTable(final byte [] tableNameOrRegionName, final int op) + private void modifyTable(final byte [] tableNameOrRegionName, + final HConstants.Modify op) throws IOException { if (tableNameOrRegionName == null) { throw new IllegalArgumentException("Pass a table name or region name"); @@ -689,7 +698,7 @@ public class HBaseAdmin { */ public void modifyTable(final byte [] tableName, HTableDescriptor htd) throws IOException { - modifyTable(tableName, HConstants.MODIFY_TABLE_SET_HTD, htd); + modifyTable(tableName, HConstants.Modify.TABLE_SET_HTD, htd); } /** @@ -702,7 +711,8 @@ public class HBaseAdmin { * @param args operation specific arguments * @throws IOException */ - public void modifyTable(final byte [] tableName, int op, Object... args) + public void modifyTable(final byte [] tableName, HConstants.Modify op, + Object... args) throws IOException { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); @@ -715,7 +725,7 @@ public class HBaseAdmin { Writable[] arr = null; try { switch (op) { - case HConstants.MODIFY_TABLE_SET_HTD: + case TABLE_SET_HTD: if (args == null || args.length < 1 || !(args[0] instanceof HTableDescriptor)) { throw new IllegalArgumentException("SET_HTD requires a HTableDescriptor"); @@ -725,10 +735,10 @@ public class HBaseAdmin { this.master.modifyTable(tableName, op, arr); break; - case HConstants.MODIFY_TABLE_COMPACT: - case HConstants.MODIFY_TABLE_SPLIT: - case HConstants.MODIFY_TABLE_MAJOR_COMPACT: - case HConstants.MODIFY_TABLE_FLUSH: + case TABLE_COMPACT: + case TABLE_SPLIT: + case TABLE_MAJOR_COMPACT: + case TABLE_FLUSH: if (args != null && args.length > 0) { arr = new Writable[1]; if (args[0] instanceof byte[]) { @@ -745,7 +755,7 @@ public class HBaseAdmin { this.master.modifyTable(tableName, op, arr); break; - case HConstants.MODIFY_CLOSE_REGION: + case CLOSE_REGION: if (args == null || args.length < 1) { throw new IllegalArgumentException("Requires at least a region name"); } diff --git a/src/java/org/apache/hadoop/hbase/client/HConnection.java b/src/java/org/apache/hadoop/hbase/client/HConnection.java index 44033ad6d20..9d960f9221a 100644 --- a/src/java/org/apache/hadoop/hbase/client/HConnection.java +++ b/src/java/org/apache/hadoop/hbase/client/HConnection.java @@ -26,7 +26,6 @@ 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.ipc.HMasterInterface; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper; @@ -190,6 +189,6 @@ public interface HConnection { * @param tableName The name of the table * @throws IOException */ - public void processBatchOfRows(ArrayList list, byte[] tableName) + public void processBatchOfRows(ArrayList list, byte[] tableName) 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 d0255201477..a7d6093ddd7 100644 --- a/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java +++ b/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java @@ -1,5 +1,5 @@ /** - * Copyright 2007 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -38,15 +38,12 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; 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.MasterNotRunningException; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.TableNotFoundException; 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.ipc.HBaseRPC; import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion; import org.apache.hadoop.hbase.ipc.HMasterInterface; @@ -338,9 +335,9 @@ public class HConnectionManager implements HConstants { MetaScannerVisitor visitor = new MetaScannerVisitor() { - public boolean processRow(RowResult rowResult) throws IOException { + public boolean processRow(Result result) throws IOException { HRegionInfo info = Writables.getHRegionInfo( - rowResult.get(COL_REGIONINFO)); + result.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER)); // Only examine the rows where the startKey is zero length if (info != null && info.getStartKey().length == 0) { @@ -387,12 +384,13 @@ public class HConnectionManager implements HConstants { HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES); byte[] endKey = null; HRegionInfo currentRegion = null; + Scan scan = new Scan(startKey); + scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER); ScannerCallable s = new ScannerCallable(this, (Bytes.equals(tableName, HConstants.META_TABLE_NAME) ? HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME), - HConstants.COL_REGIONINFO_ARRAY, startKey, - HConstants.LATEST_TIMESTAMP, null - ); + scan.getStartRow(), + scan); try { // Open scanner getRegionServerWithRetries(s); @@ -402,27 +400,25 @@ public class HConnectionManager implements HConstants { startKey = oldRegion.getEndKey(); } currentRegion = s.getHRegionInfo(); - RowResult r = null; - RowResult[] rrs = null; + Result r = null; + Result [] rrs = null; while ((rrs = getRegionServerWithRetries(s)) != null) { r = rrs[0]; - Cell c = r.get(HConstants.COL_REGIONINFO); - if (c != null) { - byte[] value = c.getValue(); - if (value != null) { - HRegionInfo info = Writables.getHRegionInfoOrNull(value); - if (info != null) { - if (Bytes.equals(info.getTableDesc().getName(), tableName)) { - rowsScanned += 1; - rowsOffline += info.isOffline() ? 1 : 0; - } + byte [] value = r.getValue(HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER); + if (value != null) { + HRegionInfo info = Writables.getHRegionInfoOrNull(value); + if (info != null) { + if (Bytes.equals(info.getTableDesc().getName(), tableName)) { + rowsScanned += 1; + rowsOffline += info.isOffline() ? 1 : 0; } } } } endKey = currentRegion.getEndKey(); - } while (!(endKey == null || HStoreKey.equalsTwoRowKeys(endKey, - HConstants.EMPTY_BYTE_ARRAY))); + } while (!(endKey == null || + Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY))); } finally { s.setClose(); @@ -440,9 +436,9 @@ public class HConnectionManager implements HConstants { protected HTableDescriptorFinder(byte[] tableName) { this.tableName = tableName; } - public boolean processRow(RowResult rowResult) throws IOException { + public boolean processRow(Result rowResult) throws IOException { HRegionInfo info = Writables.getHRegionInfo( - rowResult.get(HConstants.COL_REGIONINFO)); + rowResult.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER)); HTableDescriptor desc = info.getTableDesc(); if (Bytes.compareTo(desc.getName(), tableName) == 0) { result = desc; @@ -554,21 +550,22 @@ public class HConnectionManager implements HConstants { getHRegionConnection(metaLocation.getServerAddress()); // Query the root or meta region for the location of the meta region - RowResult regionInfoRow = server.getClosestRowBefore( + Result regionInfoRow = server.getClosestRowBefore( metaLocation.getRegionInfo().getRegionName(), metaKey, - HConstants.COLUMN_FAMILY); + HConstants.CATALOG_FAMILY); if (regionInfoRow == null) { throw new TableNotFoundException(Bytes.toString(tableName)); } - Cell value = regionInfoRow.get(COL_REGIONINFO); - if (value == null || value.getValue().length == 0) { + byte [] value = regionInfoRow.getValue(CATALOG_FAMILY, + REGIONINFO_QUALIFIER); + if (value == null || value.length == 0) { throw new IOException("HRegionInfo was null or empty in " + Bytes.toString(parentTable)); } // convert the row result into the HRegionLocation we need! HRegionInfo regionInfo = (HRegionInfo) Writables.getWritable( - value.getValue(), new HRegionInfo()); + value, new HRegionInfo()); // possible we got a region of a different table... if (!Bytes.equals(regionInfo.getTableDesc().getName(), tableName)) { throw new TableNotFoundException( @@ -579,8 +576,11 @@ public class HConnectionManager implements HConstants { regionInfo.getRegionNameAsString()); } - String serverAddress = - Writables.cellToString(regionInfoRow.get(COL_SERVER)); + value = regionInfoRow.getValue(CATALOG_FAMILY, SERVER_QUALIFIER); + String serverAddress = ""; + if(value != null) { + serverAddress = Bytes.toString(value); + } if (serverAddress.equals("")) { throw new NoServerForRegionException("No server address listed " + "in " + Bytes.toString(parentTable) + " for region " + @@ -680,8 +680,8 @@ public class HConnectionManager implements HConstants { // this one. the exception case is when the endkey is EMPTY_START_ROW, // signifying that the region we're checking is actually the last // region in the table. - if (HStoreKey.equalsTwoRowKeys(endKey, HConstants.EMPTY_END_ROW) || - HStoreKey.getComparator(tableName).compareRows(endKey, row) > 0) { + if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) || + KeyValue.getRowComparator(tableName).compare(endKey, row) > 0) { return possibleRegion; } } @@ -718,7 +718,7 @@ public class HConnectionManager implements HConstants { // by nature of the map, we know that the start key has to be < // otherwise it wouldn't be in the headMap. - if (HStoreKey.getComparator(tableName).compareRows(endKey, row) <= 0) { + if (KeyValue.getRowComparator(tableName).compare(endKey, row) <= 0) { // delete any matching entry HRegionLocation rl = tableLocations.remove(matchingRegions.lastKey()); @@ -978,15 +978,15 @@ public class HConnectionManager implements HConstants { return location; } - public void processBatchOfRows(ArrayList list, byte[] tableName) + public void processBatchOfRows(ArrayList list, byte[] tableName) throws IOException { if (list.isEmpty()) { return; } boolean retryOnlyOne = false; int tries = 0; - Collections.sort(list); - List tempUpdates = new ArrayList(); + Collections.sort(list); + List currentPuts = new ArrayList(); HRegionLocation location = getRegionLocationForRowWithRetries(tableName, list.get(0).getRow(), false); @@ -994,8 +994,8 @@ public class HConnectionManager implements HConstants { byte [] region = currentRegion; boolean isLastRow = false; for (int i = 0; i < list.size() && tries < numRetries; i++) { - BatchUpdate batchUpdate = list.get(i); - tempUpdates.add(batchUpdate); + Put put = list.get(i); + currentPuts.add(put); isLastRow = (i + 1) == list.size(); if (!isLastRow) { location = getRegionLocationForRowWithRetries(tableName, @@ -1003,19 +1003,19 @@ public class HConnectionManager implements HConstants { region = location.getRegionInfo().getRegionName(); } if (!Bytes.equals(currentRegion, region) || isLastRow || retryOnlyOne) { - final BatchUpdate[] updates = tempUpdates.toArray(new BatchUpdate[0]); + final Put [] puts = currentPuts.toArray(new Put[0]); int index = getRegionServerWithRetries(new ServerCallable( - this, tableName, batchUpdate.getRow()) { + this, tableName, put.getRow()) { public Integer call() throws IOException { - int i = server.batchUpdates(location.getRegionInfo() - .getRegionName(), updates); + int i = server.put(location.getRegionInfo() + .getRegionName(), puts); return i; } }); if (index != -1) { if (tries == numRetries - 1) { throw new RetriesExhaustedException("Some server", - currentRegion, batchUpdate.getRow(), + currentRegion, put.getRow(), tries, new ArrayList()); } long sleepTime = getPauseTime(tries); @@ -1031,7 +1031,7 @@ public class HConnectionManager implements HConstants { } catch (InterruptedException e) { // continue } - i = i - updates.length + index; + i = i - puts.length + index; retryOnlyOne = true; location = getRegionLocationForRowWithRetries(tableName, list.get(i + 1).getRow(), true); @@ -1041,7 +1041,7 @@ public class HConnectionManager implements HConstants { retryOnlyOne = false; } currentRegion = region; - tempUpdates.clear(); + currentPuts.clear(); } } } diff --git a/src/java/org/apache/hadoop/hbase/client/HTable.java b/src/java/org/apache/hadoop/hbase/client/HTable.java index b71dbc2d23b..dfa8f4d2902 100644 --- a/src/java/org/apache/hadoop/hbase/client/HTable.java +++ b/src/java/org/apache/hadoop/hbase/client/HTable.java @@ -1,5 +1,5 @@ /** - * Copyright 2008 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -34,31 +34,40 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; 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.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.filter.StopRowFilter; 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.RowResult; import org.apache.hadoop.hbase.io.HbaseMapWritable; +import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Writables; + /** * Used to communicate with a single HBase table + * TODO: checkAndSave in oldAPI + * TODO: Converting filters + * TODO: Regex deletes. */ public class HTable { private final HConnection connection; private final byte [] tableName; protected final int scannerTimeout; private volatile HBaseConfiguration configuration; - private ArrayList writeBuffer; + private ArrayList writeBuffer; private long writeBufferSize; private boolean autoFlush; private long currentWriteBufferSize; @@ -113,7 +122,7 @@ public class HTable { conf.getInt("hbase.regionserver.lease.period", 60 * 1000); this.configuration = conf; this.connection.locateRegion(tableName, HConstants.EMPTY_START_ROW); - this.writeBuffer = new ArrayList(); + this.writeBuffer = new ArrayList(); this.writeBufferSize = this.configuration.getLong("hbase.client.write.buffer", 2097152); this.autoFlush = true; @@ -242,7 +251,8 @@ public class HTable { } /** - * Gets the starting and ending row keys for every region in the currently open table + * Gets the starting and ending row keys for every region in the currently + * open table * * @return Pair of arrays of region starting and ending row keys * @throws IOException @@ -252,9 +262,10 @@ public class HTable { final List startKeyList = new ArrayList(); final List endKeyList = new ArrayList(); MetaScannerVisitor visitor = new MetaScannerVisitor() { - public boolean processRow(RowResult rowResult) throws IOException { + public boolean processRow(Result rowResult) throws IOException { HRegionInfo info = Writables.getHRegionInfo( - rowResult.get(HConstants.COL_REGIONINFO)); + rowResult.getValue(HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER)); if (Bytes.equals(info.getTableDesc().getName(), getTableName())) { if (!(info.isOffline() || info.isSplit())) { startKeyList.add(info.getStartKey()); @@ -280,18 +291,20 @@ public class HTable { new TreeMap(); MetaScannerVisitor visitor = new MetaScannerVisitor() { - public boolean processRow(RowResult rowResult) throws IOException { + public boolean processRow(Result rowResult) throws IOException { HRegionInfo info = Writables.getHRegionInfo( - rowResult.get(HConstants.COL_REGIONINFO)); + rowResult.getValue(HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER)); if (!(Bytes.equals(info.getTableDesc().getName(), getTableName()))) { return false; } HServerAddress server = new HServerAddress(); - Cell c = rowResult.get(HConstants.COL_SERVER); - if (c != null && c.getValue() != null && c.getValue().length > 0) { - String address = Bytes.toString(c.getValue()); + byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY, + HConstants.SERVER_QUALIFIER); + if (value != null && value.length > 0) { + String address = Bytes.toString(value); server = new HServerAddress(address); } @@ -307,1089 +320,227 @@ public class HTable { } /** - * Get a single value for the specified row and column + * Return the row that matches row exactly, + * or the one that immediately preceeds it. * + * @param regionName region name * @param row row key - * @param column column name - * @return value for specified row/column + * @param family Column family to look for row in. + * @return map of values * @throws IOException + * @since 0.20.0 + */ + public Result getRowOrBefore(final byte[] row, final byte[] family) + throws IOException { + return connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, row) { + public Result call() throws IOException { + return server.getClosestRowBefore(location.getRegionInfo().getRegionName(), + row, family); + } + }); + } + + /** + * Return the row that matches row exactly, + * or the one that immediately preceeds it. + * + * @param regionName region name + * @param row row key + * @param family Column family to look for row in. + * @return map of values + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getRowOrBefore(byte[], byte[]} */ - public Cell get(final String row, final String column) + public RowResult getClosestRowBefore(final byte[] row, final byte[] family) throws IOException { - return get(Bytes.toBytes(row), Bytes.toBytes(column)); + Result r = getRowOrBefore(row, family); + return r == null || r.isEmpty()? null: r.getRowResult(); } /** - * Get a single value for the specified row and column + * Get a scanner on the current table as specified by the {@link Scan} object * - * @param row row key - * @param column column name - * @param numVersions - number of versions to retrieve - * @return value for specified row/column - * @throws IOException - */ - public Cell [] get(final String row, final String column, int numVersions) - throws IOException { - return get(Bytes.toBytes(row), Bytes.toBytes(column), numVersions); - } - - /** - * Get a single value for the specified row and column - * - * @param row row key - * @param column column name - * @return value for specified row/column - * @throws IOException - */ - public Cell get(final byte [] row, final byte [] column) - throws IOException { - return connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Cell call() throws IOException { - Cell[] result = server.get(location.getRegionInfo().getRegionName(), - row, column, -1, -1); - return (result == null)? null : result[0]; - } - } - ); - } - - /** - * Get the specified number of versions of the specified row and column - * @param row row key - * @param column column name - * @param numVersions number of versions to retrieve - * @return Array of Cells. - * @throws IOException - */ - public Cell [] get(final byte [] row, final byte [] column, - final int numVersions) - throws IOException { - return connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Cell[] call() throws IOException { - return server.get(location.getRegionInfo().getRegionName(), row, - column, -1, numVersions); - } - } - ); - } - - /** - * Get the specified number of versions of the specified row and column with - * the specified timestamp. - * - * @param row - row key - * @param column - column name - * @param timestamp - timestamp - * @param numVersions - number of versions to retrieve - * @return - array of values that match the above criteria - * @throws IOException - */ - public Cell[] get(final String row, final String column, - final long timestamp, final int numVersions) - throws IOException { - return get(Bytes.toBytes(row), Bytes.toBytes(column), timestamp, numVersions); - } - - /** - * Get the specified number of versions of the specified row and column with - * the specified timestamp. - * - * @param row - row key - * @param column - column name - * @param timestamp - timestamp - * @param numVersions - number of versions to retrieve - * @return - array of values that match the above criteria - * @throws IOException - */ - public Cell[] get(final byte [] row, final byte [] column, - final long timestamp, final int numVersions) - throws IOException { - Cell[] values = null; - values = connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Cell[] call() throws IOException { - return server.get(location.getRegionInfo().getRegionName(), row, - column, timestamp, numVersions); - } - } - ); - - if (values != null) { - ArrayList cellValues = new ArrayList(); - for (int i = 0 ; i < values.length; i++) { - cellValues.add(values[i]); - } - return cellValues.toArray(new Cell[values.length]); - } - return null; - } - - /** - * Get all the data for the specified row at the latest timestamp - * - * @param row row key - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final String row) throws IOException { - return getRow(Bytes.toBytes(row)); - } - - /** - * Get all the data for the specified row at the latest timestamp - * - * @param row row key - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte [] row) throws IOException { - return getRow(row, HConstants.LATEST_TIMESTAMP); - } - - /** - * Get more than one version of all columns for the specified row - * - * @param row row key - * @param numVersions number of versions to return - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final String row, final int numVersions) - throws IOException { - return getRow(Bytes.toBytes(row), null, - HConstants.LATEST_TIMESTAMP, numVersions, null); - } - - /** - * Get more than one version of all columns for the specified row - * - * @param row row key - * @param numVersions number of versions to return - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte[] row, final int numVersions) - throws IOException { - return getRow(row, null, HConstants.LATEST_TIMESTAMP, numVersions, null); - } - - /** - * Get all the data for the specified row at a specified timestamp - * - * @param row row key - * @param ts timestamp - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final String row, final long ts) - throws IOException { - return getRow(Bytes.toBytes(row), ts); - } - - /** - * Get all the data for the specified row at a specified timestamp - * - * @param row row key - * @param ts timestamp - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte [] row, final long ts) - throws IOException { - return getRow(row,null,ts); - } - - public RowResult getRow(final String row, final long ts, - final int numVersions) throws IOException { - return getRow(Bytes.toBytes(row), null, ts, numVersions, null); - } - - /** - * Get more than one version of all columns for the specified row - * at a specified timestamp - * - * @param row row key - * @param timestamp timestamp - * @param numVersions number of versions to return - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte[] row, final long timestamp, - final int numVersions) throws IOException { - return getRow(row, null, timestamp, numVersions, null); - } - - /** - * Get selected columns for the specified row at the latest timestamp - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final String row, final String [] columns) - throws IOException { - return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns)); - } - - /** - * Get selected columns for the specified row at the latest timestamp - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte [] row, final byte [][] columns) - throws IOException { - return getRow(row, columns, HConstants.LATEST_TIMESTAMP); - } - - /** - * Get more than one version of selected columns for the specified row - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @param numVersions number of versions to return - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final String row, final String[] columns, - final int numVersions) throws IOException { - return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns), - HConstants.LATEST_TIMESTAMP, numVersions, null); - } - - /** - * Get more than one version of selected columns for the specified row - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @param numVersions number of versions to return - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte[] row, final byte[][] columns, - final int numVersions) throws IOException { - return getRow(row, columns, HConstants.LATEST_TIMESTAMP, numVersions, null); - } - - /** - * Get selected columns for the specified row at a specified timestamp - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @param ts timestamp - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final String row, final String [] columns, - final long ts) - throws IOException { - return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns), ts); - } - - /** - * Get selected columns for the specified row at a specified timestamp - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @param ts timestamp - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte [] row, final byte [][] columns, - final long ts) - throws IOException { - return getRow(row,columns,ts,1,null); - } - - public RowResult getRow(final String row, final String[] columns, - final long timestamp, final int numVersions, final RowLock rowLock) - throws IOException { - return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns), timestamp, - numVersions, rowLock); - } - - - /** - * Get selected columns for the specified row at a specified timestamp - * using existing row lock. - * - * @param row row key - * @param columns Array of column names and families you want to retrieve. - * @param ts timestamp - * @param numVersions - * @param rl row lock - * @return RowResult is null if row does not exist. - * @throws IOException - */ - public RowResult getRow(final byte [] row, final byte [][] columns, - final long ts, final int numVersions, final RowLock rl) - throws IOException { - return connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public RowResult call() throws IOException { - long lockId = -1L; - if(rl != null) { - lockId = rl.getLockId(); - } - return server.getRow(location.getRegionInfo().getRegionName(), row, - columns, ts, numVersions, lockId); - } - } - ); - } - - public RowResult getClosestRowBefore(final byte[] row, final byte[] columnFamily) - throws IOException { - return connection.getRegionServerWithRetries( - new ServerCallable(connection,tableName,row) { - public RowResult call() throws IOException { - return server.getClosestRowBefore( - location.getRegionInfo().getRegionName(), row, columnFamily - ); - } - } - ); - } - - /** - * Get a scanner on the current table starting at first row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. + * @param scan a configured {@link Scan} object * @return scanner * @throws IOException + * @since 0.20.0 */ - public Scanner getScanner(final String [] columns) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), HConstants.EMPTY_START_ROW); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final String [] columns, final String startRow) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow)); - } - - /** - * Get a scanner on the current table starting at first row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte[][] columns) - throws IOException { - return getScanner(columns, HConstants.EMPTY_START_ROW, - HConstants.LATEST_TIMESTAMP, null); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte[][] columns, final byte [] startRow) - throws IOException { - return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, null); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param timestamp only return results whose timestamp <= this value - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte[][] columns, final byte [] startRow, - long timestamp) - throws IOException { - return getScanner(columns, startRow, timestamp, null); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param filter a row filter using row-key regexp and/or column data filter. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte[][] columns, final byte [] startRow, - RowFilterInterface filter) - throws IOException { - return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, filter); - } - - /** - * Get a scanner on the current table starting at the specified row and - * ending just before stopRow. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param stopRow Row to stop scanning on. Once we hit this row we stop - * returning values; i.e. we return the row before this one but not the - * stopRow itself. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte [][] columns, - final byte [] startRow, final byte [] stopRow) - throws IOException { - return getScanner(columns, startRow, stopRow, - HConstants.LATEST_TIMESTAMP); - } - - /** - * Get a scanner on the current table starting at the specified row and - * ending just before stopRow. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param stopRow Row to stop scanning on. Once we hit this row we stop - * returning values; i.e. we return the row before this one but not the - * stopRow itself. - * @param timestamp only return results whose timestamp <= this value - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final String [] columns, - final String startRow, final String stopRow, final long timestamp) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow), - Bytes.toBytes(stopRow), timestamp); - } - - /** - * Get a scanner on the current table starting at the specified row and - * ending just before stopRow. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param stopRow Row to stop scanning on. Once we hit this row we stop - * returning values; i.e. we return the row before this one but not the - * stopRow itself. - * @param timestamp only return results whose timestamp <= this value - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte [][] columns, - final byte [] startRow, final byte [] stopRow, final long timestamp) - throws IOException { - return getScanner(columns, startRow, timestamp, - new WhileMatchRowFilter(new StopRowFilter(stopRow))); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param timestamp only return results whose timestamp <= this value - * @param filter a row filter using row-key regexp and/or column data filter. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(String[] columns, - String startRow, long timestamp, RowFilterInterface filter) - throws IOException { - return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow), - timestamp, filter); - } - - /** - * Get a scanner on the current table starting at the specified row. - * Return the specified columns. - * - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param timestamp only return results whose timestamp <= this value - * @param filter a row filter using row-key regexp and/or column data filter. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final byte [][] columns, - final byte [] startRow, long timestamp, RowFilterInterface filter) - throws IOException { - ClientScanner s = new ClientScanner(columns, startRow, - timestamp, filter); + public ResultScanner getScanner(final Scan scan) throws IOException { + ClientScanner s = new ClientScanner(scan); s.initialize(); return s; } - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @throws IOException - */ - public void deleteAll(final byte [] row) throws IOException { - deleteAll(row, null); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @throws IOException - */ - public void deleteAll(final String row) throws IOException { - deleteAll(row, null); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @param column column to be deleted - * @throws IOException - */ - public void deleteAll(final byte [] row, final byte [] column) - throws IOException { - deleteAll(row, column, HConstants.LATEST_TIMESTAMP); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - */ - public void deleteAll(final byte [] row, final long ts) - throws IOException { - deleteAll(row, null, ts); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - */ - public void deleteAll(final String row, final long ts) - throws IOException { - deleteAll(row, null, ts); - } - - /** - * Delete all cells that match the passed row and column. - * @param row Row to update - * @param column name of column whose value is to be deleted - * @throws IOException - */ - public void deleteAll(final String row, final String column) - throws IOException { - deleteAll(row, column, HConstants.LATEST_TIMESTAMP); - } - - /** - * Delete all cells that match the passed row and column and whose - * timestamp is equal-to or older than the passed timestamp. - * @param row Row to update - * @param column name of column whose value is to be deleted - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - */ - public void deleteAll(final String row, final String column, final long ts) - throws IOException { - deleteAll(Bytes.toBytes(row), - column != null? Bytes.toBytes(column): null, ts); - } - - /** - * Delete all cells that match the passed row and column and whose - * timestamp is equal-to or older than the passed timestamp. - * @param row Row to update - * @param column name of column whose value is to be deleted - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - */ - public void deleteAll(final byte [] row, final byte [] column, final long ts) - throws IOException { - deleteAll(row,column,ts,null); - } - - /** - * Delete all cells that match the passed row and column and whose - * timestamp is equal-to or older than the passed timestamp, using an - * existing row lock. - * @param row Row to update - * @param column name of column whose value is to be deleted - * @param ts Delete all cells of the same timestamp or older. - * @param rl Existing row lock - * @throws IOException - */ - public void deleteAll(final byte [] row, final byte [] column, final long ts, - final RowLock rl) - throws IOException { - connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Boolean call() throws IOException { - long lockId = -1L; - if(rl != null) { - lockId = rl.getLockId(); - } - if (column != null) { - this.server.deleteAll(location.getRegionInfo().getRegionName(), - row, column, ts, lockId); - } else { - this.server.deleteAll(location.getRegionInfo().getRegionName(), - row, ts, lockId); - } - return null; - } - } - ); - } - - /** - * Delete all cells that match the passed row and column. - * @param row Row to update - * @param colRegex column regex expression - * @throws IOException - */ - public void deleteAllByRegex(final String row, final String colRegex) - throws IOException { - deleteAll(row, colRegex, HConstants.LATEST_TIMESTAMP); - } - - /** - * Delete all cells that match the passed row and column and whose - * timestamp is equal-to or older than the passed timestamp. - * @param row Row to update - * @param colRegex Column Regex expression - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - */ - public void deleteAllByRegex(final String row, final String colRegex, - final long ts) throws IOException { - deleteAllByRegex(Bytes.toBytes(row), colRegex, ts); - } - - /** - * Delete all cells that match the passed row and column and whose - * timestamp is equal-to or older than the passed timestamp. - * @param row Row to update - * @param colRegex Column Regex expression - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - */ - public void deleteAllByRegex(final byte [] row, final String colRegex, - final long ts) throws IOException { - deleteAllByRegex(row, colRegex, ts, null); - } - - /** - * Delete all cells that match the passed row and column and whose - * timestamp is equal-to or older than the passed timestamp, using an - * existing row lock. - * @param row Row to update - * @param colRegex Column regex expression - * @param ts Delete all cells of the same timestamp or older. - * @param rl Existing row lock - * @throws IOException - */ - public void deleteAllByRegex(final byte [] row, final String colRegex, - final long ts, final RowLock rl) - throws IOException { - connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Boolean call() throws IOException { - long lockId = -1L; - if(rl != null) { - lockId = rl.getLockId(); - } - this.server.deleteAllByRegex(location.getRegionInfo().getRegionName(), - row, colRegex, ts, lockId); - return null; - } - } - ); - } - - /** - * Delete all cells for a row with matching column family at all timestamps. - * - * @param row The row to operate on - * @param family The column family to match - * @throws IOException - */ - public void deleteFamily(final String row, final String family) - throws IOException { - deleteFamily(row, family, HConstants.LATEST_TIMESTAMP); - } - - /** - * Delete all cells for a row with matching column family at all timestamps. - * - * @param row The row to operate on - * @param family The column family to match - * @throws IOException - */ - public void deleteFamily(final byte[] row, final byte[] family) - throws IOException { - deleteFamily(row, family, HConstants.LATEST_TIMESTAMP); - } - - /** - * Delete all cells for a row with matching column family with timestamps - * less than or equal to timestamp. - * - * @param row The row to operate on - * @param family The column family to match - * @param timestamp Timestamp to match - * @throws IOException - */ - public void deleteFamily(final String row, final String family, - final long timestamp) - throws IOException{ - deleteFamily(Bytes.toBytes(row), Bytes.toBytes(family), timestamp); - } - - /** - * Delete all cells for a row with matching column family with timestamps - * less than or equal to timestamp. - * - * @param row The row to operate on - * @param family The column family to match - * @param timestamp Timestamp to match - * @throws IOException - */ - public void deleteFamily(final byte [] row, final byte [] family, - final long timestamp) - throws IOException { - deleteFamily(row,family,timestamp,null); - } - - /** - * Delete all cells for a row with matching column family with timestamps - * less than or equal to timestamp, using existing row lock. - * - * @param row The row to operate on - * @param family The column family to match - * @param timestamp Timestamp to match - * @param rl Existing row lock - * @throws IOException - */ - public void deleteFamily(final byte [] row, final byte [] family, - final long timestamp, final RowLock rl) - throws IOException { - connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Boolean call() throws IOException { - long lockId = -1L; - if(rl != null) { - lockId = rl.getLockId(); - } - server.deleteFamily(location.getRegionInfo().getRegionName(), row, - family, timestamp, lockId); - return null; - } - } - ); - } - - /** - * Delete all cells for a row with matching column family regex - * at all timestamps. - * - * @param row The row to operate on - * @param familyRegex Column family regex - * @throws IOException - */ - public void deleteFamilyByRegex(final String row, final String familyRegex) - throws IOException { - deleteFamilyByRegex(row, familyRegex, HConstants.LATEST_TIMESTAMP); - } - - /** - * Delete all cells for a row with matching column family regex - * at all timestamps. - * - * @param row The row to operate on - * @param familyRegex Column family regex - * @throws IOException - */ - public void deleteFamilyByRegex(final byte[] row, final String familyRegex) - throws IOException { - deleteFamilyByRegex(row, familyRegex, HConstants.LATEST_TIMESTAMP); - } - - /** - * Delete all cells for a row with matching column family regex - * with timestamps less than or equal to timestamp. - * - * @param row The row to operate on - * @param familyRegex Column family regex - * @param timestamp Timestamp to match - * @throws IOException - */ - public void deleteFamilyByRegex(final String row, final String familyRegex, - final long timestamp) - throws IOException{ - deleteFamilyByRegex(Bytes.toBytes(row), familyRegex, timestamp); - } - - /** - * Delete all cells for a row with matching column family regex - * with timestamps less than or equal to timestamp. - * - * @param row The row to operate on - * @param familyRegex Column family regex - * @param timestamp Timestamp to match - * @throws IOException - */ - public void deleteFamilyByRegex(final byte [] row, final String familyRegex, - final long timestamp) - throws IOException { - deleteFamilyByRegex(row,familyRegex,timestamp,null); - } - - /** - * Delete all cells for a row with matching column family regex with - * timestamps less than or equal to timestamp, using existing - * row lock. + * Get a scanner on the current table as specified by the {@link Scan} object * - * @param row The row to operate on - * @param familyRegex Column Family Regex - * @param timestamp Timestamp to match - * @param r1 Existing row lock + * @param family + * @return * @throws IOException + * @since 0.20.0 */ - public void deleteFamilyByRegex(final byte[] row, final String familyRegex, - final long timestamp, final RowLock r1) throws IOException { - connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Boolean call() throws IOException { - long lockId = -1L; - if(r1 != null) { - lockId = r1.getLockId(); - } - server.deleteFamilyByRegex(location.getRegionInfo().getRegionName(), - row, familyRegex, timestamp, lockId); - return null; - } - } - ); + public ResultScanner getScanner(byte [] family) throws IOException { + Scan scan = new Scan(); + scan.addFamily(family); + return getScanner(scan); } - + /** - * Test for the existence of a row in the table. + * Get a scanner on the current table as specified by the {@link Scan} object * - * @param row The row - * @return true if the row exists, false otherwise + * @param family + * @param qualifier + * @return * @throws IOException + * @since 0.20.0 */ - public boolean exists(final byte [] row) throws IOException { - return exists(row, null, HConstants.LATEST_TIMESTAMP, null); - } - - /** - * Test for the existence of a row and column in the table. - * - * @param row The row - * @param column The column - * @return true if the row exists, false otherwise - * @throws IOException - */ - public boolean exists(final byte [] row, final byte[] column) + public ResultScanner getScanner(byte [] family, byte [] qualifier) throws IOException { - return exists(row, column, HConstants.LATEST_TIMESTAMP, null); + Scan scan = new Scan(); + scan.addColumn(family, qualifier); + return getScanner(scan); } /** - * Test for the existence of a coordinate in the table. - * - * @param row The row - * @param column The column - * @param timestamp The timestamp - * @return true if the specified coordinate exists + * Method for getting data from a row + * @param get the Get to fetch + * @return the result * @throws IOException + * @since 0.20.0 */ - public boolean exists(final byte [] row, final byte [] column, - long timestamp) throws IOException { - return exists(row, column, timestamp, null); - } - - /** - * Test for the existence of a coordinate in the table. - * - * @param row The row - * @param column The column - * @param timestamp The timestamp - * @param rl Existing row lock - * @return true if the specified coordinate exists - * @throws IOException - */ - public boolean exists(final byte [] row, final byte [] column, - final long timestamp, final RowLock rl) throws IOException { + public Result get(final Get get) throws IOException { return connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Boolean call() throws IOException { - long lockId = -1L; - if (rl != null) { - lockId = rl.getLockId(); + new ServerCallable(connection, tableName, get.getRow()) { + public Result call() throws IOException { + return server.get(location.getRegionInfo().getRegionName(), get); } - return Boolean.valueOf(server. - exists(location.getRegionInfo().getRegionName(), row, - column, timestamp, lockId)); } - } - ).booleanValue(); - } - - /** - * Commit a BatchUpdate to the table. - * If autoFlush is false, the update is buffered - * @param batchUpdate - * @throws IOException - */ - public synchronized void commit(final BatchUpdate batchUpdate) - throws IOException { - commit(batchUpdate,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 + * + * @param delete * @throws IOException - */ - public synchronized void commit(final BatchUpdate batchUpdate, - final RowLock rl) + * @since 0.20.0 + */ + public void delete(final Delete delete) throws IOException { - checkRowAndColumns(batchUpdate); - if(rl != null) { - batchUpdate.setRowLock(rl.getLockId()); - } - writeBuffer.add(batchUpdate); - currentWriteBufferSize += batchUpdate.heapSize(); - if (autoFlush || currentWriteBufferSize > writeBufferSize) { + connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, delete.getRow()) { + public Boolean call() throws IOException { + System.out.println("IN HT.get.ServerCallable,"); + server.delete(location.getRegionInfo().getRegionName(), delete); + return null; + } + } + ); + } + + /** + * Commit a Put to the table. + *

+ * If autoFlush is false, the update is buffered. + * @param put + * @throws IOException + * @since 0.20.0 + */ + public synchronized void put(final Put put) throws IOException { + validatePut(put); + writeBuffer.add(put); + currentWriteBufferSize += put.heapSize(); + if(autoFlush || currentWriteBufferSize > writeBufferSize) { flushCommits(); } } /** - * Commit a List of BatchUpdate to the table. - * If autoFlush is false, the updates are buffered - * @param batchUpdates + * Commit a List of Puts to the table. + *

+ * If autoFlush is false, the update is buffered. + * @param puts * @throws IOException - */ - public synchronized void commit(final List batchUpdates) - throws IOException { - for (BatchUpdate bu : batchUpdates) { - checkRowAndColumns(bu); - writeBuffer.add(bu); - currentWriteBufferSize += bu.heapSize(); + * @since 0.20.0 + */ + public synchronized void put(final List puts) throws IOException { + for(Put put : puts) { + validatePut(put); + writeBuffer.add(put); + currentWriteBufferSize += put.heapSize(); } - if (autoFlush || currentWriteBufferSize > writeBufferSize) { + if(autoFlush || currentWriteBufferSize > writeBufferSize) { flushCommits(); } } /** - * Atomically checks if a row's values match - * the expectedValues. If it does, it uses the - * batchUpdate to update the row. - * @param batchUpdate batchupdate to apply if check is successful - * @param expectedValues values to check - * @param rl rowlock + * Atomically increments a column value. If the column value isn't long-like, + * this could throw an exception. + * + * @param row + * @param family + * @param qualifier + * @param amount + * @return * @throws IOException */ - public synchronized boolean checkAndSave(final BatchUpdate batchUpdate, - final HbaseMapWritable expectedValues, final RowLock rl) + public long incrementColumnValue(final byte [] row, final byte [] family, + final byte [] qualifier, final long amount) throws IOException { - checkRowAndColumns(batchUpdate); - if(rl != null) { - batchUpdate.setRowLock(rl.getLockId()); + NullPointerException npe = null; + if (row == null) { + npe = new NullPointerException("row is null"); + } else if (family == null) { + npe = new NullPointerException("column is null"); + } + if (npe != null) { + IOException io = new IOException( + "Invalid arguments to incrementColumnValue", npe); + throw io; } return connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, batchUpdate.getRow()) { - public Boolean call() throws IOException { - return server.checkAndSave(location.getRegionInfo().getRegionName(), - batchUpdate, expectedValues)? - Boolean.TRUE: Boolean.FALSE; + new ServerCallable(connection, tableName, row) { + public Long call() throws IOException { + Get get = new Get(row); + get.addColumn(family, qualifier); + return server.incrementColumnValue( + location.getRegionInfo().getRegionName(), row, family, + qualifier, amount); + } } - } - ).booleanValue(); + ); } + + /** + * Atomically checks if a row/family/qualifier value match the expectedValue. + * If it does, it adds the put. + * + * @param row + * @param family + * @param qualifier + * @param value the expected value + * @param put + * @throws IOException + * @return true if the new put was execute, false otherwise + */ + public synchronized boolean checkAndPut(final byte [] row, + final byte [] family, final byte [] qualifier, final byte [] value, + final Put put) + throws IOException { + return connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, row) { + public Boolean call() throws IOException { + return server.checkAndPut(location.getRegionInfo().getRegionName(), + row, family, qualifier, value, put)? Boolean.TRUE: Boolean.FALSE; + } + } + ).booleanValue(); + } + /** * Commit to the table the buffer of BatchUpdate. - * Called automaticaly in the commit methods when autoFlush is true. + * Called automatically in the commit methods when autoFlush is true. * @throws IOException */ public void flushCommits() throws IOException { @@ -1409,25 +560,19 @@ public class HTable { public void close() throws IOException{ flushCommits(); } - + /** - * Utility method that checks rows existence, length and columns well - * formedness. - * - * @param bu + * Utility method that verifies Put is well formed. + * @param put * @throws IllegalArgumentException * @throws IOException */ - private void checkRowAndColumns(BatchUpdate bu) - throws IllegalArgumentException, IOException { - if (bu.getRow() == null || bu.getRow().length > HConstants.MAX_ROW_LENGTH) { - throw new IllegalArgumentException("Row key is invalid"); - } - for (BatchOperation bo : bu) { - HStoreKey.getFamily(bo.getColumn()); + private void validatePut(final Put put) throws IllegalArgumentException{ + if(put.isEmpty()) { + throw new IllegalArgumentException("No columns to insert"); } } - + /** * Obtain a row lock * @param row The row to lock @@ -1502,20 +647,1068 @@ public class HTable { * Get the write buffer * @return the current write buffer */ - public ArrayList getWriteBuffer() { + public ArrayList getWriteBuffer() { return writeBuffer; } + + // Old API. Pre-hbase-880, hbase-1304. - public long incrementColumnValue(final byte [] row, final byte [] column, - final long amount) throws IOException { - return connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Long call() throws IOException { - return server.incrementColumnValue( - location.getRegionInfo().getRegionName(), row, column, amount); - } + /** + * Get a single value for the specified row and column + * + * @param row row key + * @param column column name + * @return value for specified row/column + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public Cell get(final String row, final String column) + throws IOException { + return get(Bytes.toBytes(row), Bytes.toBytes(column)); + } + + /** + * Get a single value for the specified row and column + * + * @param row row key + * @param column column name + * @param numVersions - number of versions to retrieve + * @return value for specified row/column + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public Cell [] get(final String row, final String column, int numVersions) + throws IOException { + return get(Bytes.toBytes(row), Bytes.toBytes(column), numVersions); + } + + /** + * Get a single value for the specified row and column + * + * @param row row key + * @param column column name + * @return value for specified row/column + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public Cell get(final byte [] row, final byte [] column) + throws IOException { + Get g = new Get(row); + byte [][] fq = KeyValue.parseColumn(column); + g.addColumn(fq[0], fq[1]); + Result r = get(g); + return r == null || r.size() <= 0? null: r.getCellValue(); + } + + /** + * Get the specified number of versions of the specified row and column + * @param row row key + * @param column column name + * @param numVersions number of versions to retrieve + * @return Array of Cells. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public Cell [] get(final byte [] row, final byte [] column, + final int numVersions) + throws IOException { + return get(row, column, HConstants.LATEST_TIMESTAMP, numVersions); + } + + /** + * Get the specified number of versions of the specified row and column with + * the specified timestamp. + * + * @param row - row key + * @param column - column name + * @param timestamp - timestamp + * @param numVersions - number of versions to retrieve + * @return - array of values that match the above criteria + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public Cell[] get(final String row, final String column, + final long timestamp, final int numVersions) + throws IOException { + return get(Bytes.toBytes(row), Bytes.toBytes(column), timestamp, numVersions); + } + + /** + * Get the specified number of versions of the specified row and column with + * the specified timestamp. + * + * @param row - row key + * @param column - column name + * @param timestamp - timestamp + * @param numVersions - number of versions to retrieve + * @return - array of values that match the above criteria + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public Cell[] get(final byte [] row, final byte [] column, + final long timestamp, final int numVersions) + throws IOException { + Get g = new Get(row); + byte [][] fq = KeyValue.parseColumn(column); + if (fq[1].length == 0) { + g.addFamily(fq[0]); + } else { + g.addColumn(fq[0], fq[1]); + } + g.setMaxVersions(numVersions); + if (timestamp != HConstants.LATEST_TIMESTAMP) { + g.setTimeStamp(timestamp); + } + Result r = get(g); + return r == null || r.size() <= 0? null: r.getCellValues(); + } + + /** + * Get all the data for the specified row at the latest timestamp + * + * @param row row key + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final String row) throws IOException { + return getRow(Bytes.toBytes(row)); + } + + /** + * Get all the data for the specified row at the latest timestamp + * + * @param row row key + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final byte [] row) throws IOException { + return getRow(row, HConstants.LATEST_TIMESTAMP); + } + + /** + * Get more than one version of all columns for the specified row + * + * @param row row key + * @param numVersions number of versions to return + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final String row, final int numVersions) + throws IOException { + return getRow(Bytes.toBytes(row), null, + HConstants.LATEST_TIMESTAMP, numVersions, null); + } + + /** + * Get more than one version of all columns for the specified row + * + * @param row row key + * @param numVersions number of versions to return + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final byte[] row, final int numVersions) + throws IOException { + return getRow(row, null, HConstants.LATEST_TIMESTAMP, numVersions, null); + } + + /** + * Get all the data for the specified row at a specified timestamp + * + * @param row row key + * @param ts timestamp + * @return RowResult is null if row does not exist. + * @throws IOException + */ + public RowResult getRow(final String row, final long ts) + throws IOException { + return getRow(Bytes.toBytes(row), ts); + } + + /** + * Get all the data for the specified row at a specified timestamp + * + * @param row row key + * @param ts timestamp + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final byte [] row, final long ts) + throws IOException { + return getRow(row,null,ts); + } + + public RowResult getRow(final String row, final long ts, + final int numVersions) throws IOException { + return getRow(Bytes.toBytes(row), null, ts, numVersions, null); + } + + /** + * Get more than one version of all columns for the specified row + * at a specified timestamp + * + * @param row row key + * @param timestamp timestamp + * @param numVersions number of versions to return + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final byte[] row, final long timestamp, + final int numVersions) throws IOException { + return getRow(row, null, timestamp, numVersions, null); + } + + /** + * Get selected columns for the specified row at the latest timestamp + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final String row, final String [] columns) + throws IOException { + return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns)); + } + + /** + * Get selected columns for the specified row at the latest timestamp + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final byte [] row, final byte [][] columns) + throws IOException { + return getRow(row, columns, HConstants.LATEST_TIMESTAMP); + } + + /** + * Get more than one version of selected columns for the specified row + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @param numVersions number of versions to return + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final String row, final String[] columns, + final int numVersions) throws IOException { + return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns), + HConstants.LATEST_TIMESTAMP, numVersions, null); + } + + /** + * Get more than one version of selected columns for the specified row + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @param numVersions number of versions to return + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final byte[] row, final byte[][] columns, + final int numVersions) throws IOException { + return getRow(row, columns, HConstants.LATEST_TIMESTAMP, numVersions, null); + } + + /** + * Get selected columns for the specified row at a specified timestamp + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @param ts timestamp + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final String row, final String [] columns, + final long ts) + throws IOException { + return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns), ts); + } + + /** + * Get selected columns for the specified row at a specified timestamp + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @param ts timestamp + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final byte [] row, final byte [][] columns, + final long ts) + throws IOException { + return getRow(row,columns,ts,1,null); + } + + public RowResult getRow(final String row, final String[] columns, + final long timestamp, final int numVersions, final RowLock rowLock) + throws IOException { + return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns), timestamp, + numVersions, rowLock); + } + + + /** + * Get selected columns for the specified row at a specified timestamp + * using existing row lock. + * + * @param row row key + * @param columns Array of column names and families you want to retrieve. + * @param ts timestamp + * @param numVersions + * @param rl row lock + * @return RowResult is null if row does not exist. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)} + */ + public RowResult getRow(final byte [] row, final byte [][] columns, + final long ts, final int numVersions, final RowLock rl) + throws IOException { + Get g = rl != null? new Get(row, rl): new Get(row); + if (columns != null) { + for (int i = 0; i < columns.length; i++) { + byte[][] splits = KeyValue.parseColumn(columns[i]); + if (splits[1].length == 0) { + g.addFamily(splits[0]); + } else { + g.addColumn(splits[0], splits[1]); } - ); + } + } + g.setMaxVersions(numVersions); + if (ts != HConstants.LATEST_TIMESTAMP) { + g.setTimeStamp(ts); + } + Result r = get(g); + return r == null || r.size() <= 0? null: r.getRowResult(); + } + + /** + * Get a scanner on the current table starting at first row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @return scanner + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getScanner(Scan)} + */ + public Scanner getScanner(final String [] columns) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), HConstants.EMPTY_START_ROW); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @return scanner + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getScanner(Scan)} + */ + public Scanner getScanner(final String [] columns, final String startRow) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow)); + } + + /** + * Get a scanner on the current table starting at first row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @return scanner + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getScanner(Scan)} + */ + public Scanner getScanner(final byte[][] columns) + throws IOException { + return getScanner(columns, HConstants.EMPTY_START_ROW, + HConstants.LATEST_TIMESTAMP, null); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @return scanner + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getScanner(Scan)} + */ + public Scanner getScanner(final byte[][] columns, final byte [] startRow) + throws IOException { + return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, null); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param timestamp only return results whose timestamp <= this value + * @return scanner + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getScanner(Scan)} + */ + public Scanner getScanner(final byte[][] columns, final byte [] startRow, + long timestamp) + throws IOException { + return getScanner(columns, startRow, timestamp, null); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param filter a row filter using row-key regexp and/or column data filter. + * @return scanner + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getScanner(Scan)} + */ + public Scanner getScanner(final byte[][] columns, final byte [] startRow, + RowFilterInterface filter) + throws IOException { + return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, filter); + } + + /** + * Get a scanner on the current table starting at the specified row and + * ending just before stopRow. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param stopRow Row to stop scanning on. Once we hit this row we stop + * returning values; i.e. we return the row before this one but not the + * stopRow itself. + * @return scanner + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getScanner(Scan)} + */ + public Scanner getScanner(final byte [][] columns, + final byte [] startRow, final byte [] stopRow) + throws IOException { + return getScanner(columns, startRow, stopRow, HConstants.LATEST_TIMESTAMP); + } + + /** + * Get a scanner on the current table starting at the specified row and + * ending just before stopRow. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param stopRow Row to stop scanning on. Once we hit this row we stop + * returning values; i.e. we return the row before this one but not the + * stopRow itself. + * @param timestamp only return results whose timestamp <= this value + * @return scanner + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getScanner(Scan)} + */ + public Scanner getScanner(final String [] columns, + final String startRow, final String stopRow, final long timestamp) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow), + Bytes.toBytes(stopRow), timestamp); + } + + /** + * Get a scanner on the current table starting at the specified row and + * ending just before stopRow. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param stopRow Row to stop scanning on. Once we hit this row we stop + * returning values; i.e. we return the row before this one but not the + * stopRow itself. + * @param timestamp only return results whose timestamp <= this value + * @return scanner + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getScanner(Scan)} + */ + public Scanner getScanner(final byte [][] columns, + final byte [] startRow, final byte [] stopRow, final long timestamp) + throws IOException { + return getScanner(columns, startRow, timestamp, + new WhileMatchRowFilter(new StopRowFilter(stopRow))); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param timestamp only return results whose timestamp <= this value + * @param filter a row filter using row-key regexp and/or column data filter. + * @return scanner + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getScanner(Scan)} + */ + public Scanner getScanner(String[] columns, + String startRow, long timestamp, RowFilterInterface filter) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow), + timestamp, filter); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param timestamp only return results whose timestamp <= this value + * @param filter a row filter using row-key regexp and/or column data filter. + * @return scanner + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #getScanner(Scan)} + */ + public Scanner getScanner(final byte [][] columns, + final byte [] startRow, long timestamp, RowFilterInterface filter) + throws IOException { + // Convert old-style filter to new. We only do a few types at moment. + // If a whilematchrowfilter and it has a stoprowfilter, handle that. + Scan scan = filter == null? new Scan(startRow): + filter instanceof WhileMatchRowFilter && ((WhileMatchRowFilter)filter).getInternalFilter() instanceof StopRowFilter? + new Scan(startRow, ((StopRowFilter)((WhileMatchRowFilter)filter).getInternalFilter()).getStopRowKey()): + null /*new UnsupportedOperationException("Not handled yet")*/; + for (int i = 0; i < columns.length; i++) { + byte [][] splits = KeyValue.parseColumn(columns[i]); + if (splits[1].length == 0) { + scan.addFamily(splits[0]); + } else { + scan.addColumn(splits[0], splits[1]); + } + } + OldClientScanner s = new OldClientScanner(new ClientScanner(scan)); + s.initialize(); + return s; + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @throws IOException + */ + public void deleteAll(final byte [] row) throws IOException { + deleteAll(row, null); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @throws IOException + */ + public void deleteAll(final String row) throws IOException { + deleteAll(row, null); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @param column column to be deleted + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteAll(final byte [] row, final byte [] column) + throws IOException { + deleteAll(row, column, HConstants.LATEST_TIMESTAMP); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteAll(final byte [] row, final long ts) + throws IOException { + deleteAll(row, null, ts); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteAll(final String row, final long ts) + throws IOException { + deleteAll(row, null, ts); + } + + /** + * Delete all cells that match the passed row and column. + * @param row Row to update + * @param column name of column whose value is to be deleted + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteAll(final String row, final String column) + throws IOException { + deleteAll(row, column, HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete all cells that match the passed row and column and whose + * timestamp is equal-to or older than the passed timestamp. + * @param row Row to update + * @param column name of column whose value is to be deleted + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteAll(final String row, final String column, final long ts) + throws IOException { + deleteAll(Bytes.toBytes(row), + column != null? Bytes.toBytes(column): null, ts); + } + + /** + * Delete all cells that match the passed row and column and whose + * timestamp is equal-to or older than the passed timestamp. + * @param row Row to update + * @param column name of column whose value is to be deleted + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteAll(final byte [] row, final byte [] column, final long ts) + throws IOException { + deleteAll(row,column,ts,null); + } + + /** + * Delete all cells that match the passed row and column and whose + * timestamp is equal-to or older than the passed timestamp, using an + * existing row lock. + * @param row Row to update + * @param column name of column whose value is to be deleted + * @param ts Delete all cells of the same timestamp or older. + * @param rl Existing row lock + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteAll(final byte [] row, final byte [] column, final long ts, + final RowLock rl) + throws IOException { + Delete d = new Delete(row, ts, rl); + d.deleteColumn(column); + delete(d); + } + + /** + * Delete all cells that match the passed row and column. + * @param row Row to update + * @param colRegex column regex expression + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteAllByRegex(final String row, final String colRegex) + throws IOException { + deleteAll(row, colRegex, HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete all cells that match the passed row and column and whose + * timestamp is equal-to or older than the passed timestamp. + * @param row Row to update + * @param colRegex Column Regex expression + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteAllByRegex(final String row, final String colRegex, + final long ts) throws IOException { + deleteAllByRegex(Bytes.toBytes(row), colRegex, ts); + } + + /** + * Delete all cells that match the passed row and column and whose + * timestamp is equal-to or older than the passed timestamp. + * @param row Row to update + * @param colRegex Column Regex expression + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteAllByRegex(final byte [] row, final String colRegex, + final long ts) throws IOException { + deleteAllByRegex(row, colRegex, ts, null); + } + + /** + * Delete all cells that match the passed row and column and whose + * timestamp is equal-to or older than the passed timestamp, using an + * existing row lock. + * @param row Row to update + * @param colRegex Column regex expression + * @param ts Delete all cells of the same timestamp or older. + * @param rl Existing row lock + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteAllByRegex(final byte [] row, final String colRegex, + final long ts, final RowLock rl) + throws IOException { + throw new UnsupportedOperationException("TODO: Not yet implemented"); + } + + /** + * Delete all cells for a row with matching column family at all timestamps. + * + * @param row The row to operate on + * @param family The column family to match + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteFamily(final String row, final String family) + throws IOException { + deleteFamily(row, family, HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete all cells for a row with matching column family at all timestamps. + * + * @param row The row to operate on + * @param family The column family to match + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteFamily(final byte[] row, final byte[] family) + throws IOException { + deleteFamily(row, family, HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete all cells for a row with matching column family with timestamps + * less than or equal to timestamp. + * + * @param row The row to operate on + * @param family The column family to match + * @param timestamp Timestamp to match + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteFamily(final String row, final String family, + final long timestamp) + throws IOException{ + deleteFamily(Bytes.toBytes(row), Bytes.toBytes(family), timestamp); + } + + /** + * Delete all cells for a row with matching column family with timestamps + * less than or equal to timestamp. + * + * @param row The row to operate on + * @param family The column family to match + * @param timestamp Timestamp to match + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteFamily(final byte [] row, final byte [] family, + final long timestamp) + throws IOException { + deleteFamily(row,family,timestamp,null); + } + + /** + * Delete all cells for a row with matching column family with timestamps + * less than or equal to timestamp, using existing row lock. + * + * @param row The row to operate on + * @param family The column family to match + * @param timestamp Timestamp to match + * @param rl Existing row lock + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteFamily(final byte [] row, final byte [] family, + final long timestamp, final RowLock rl) + throws IOException { + // Is this right? LATEST_TS? St.Ack + Delete d = new Delete(row, HConstants.LATEST_TIMESTAMP, rl); + d.deleteFamily(family); + delete(d); + } + + /** + * Delete all cells for a row with matching column family regex + * at all timestamps. + * + * @param row The row to operate on + * @param familyRegex Column family regex + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteFamilyByRegex(final String row, final String familyRegex) + throws IOException { + deleteFamilyByRegex(row, familyRegex, HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete all cells for a row with matching column family regex + * at all timestamps. + * + * @param row The row to operate on + * @param familyRegex Column family regex + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteFamilyByRegex(final byte[] row, final String familyRegex) + throws IOException { + deleteFamilyByRegex(row, familyRegex, HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete all cells for a row with matching column family regex + * with timestamps less than or equal to timestamp. + * + * @param row The row to operate on + * @param familyRegex Column family regex + * @param timestamp Timestamp to match + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteFamilyByRegex(final String row, final String familyRegex, + final long timestamp) + throws IOException{ + deleteFamilyByRegex(Bytes.toBytes(row), familyRegex, timestamp); + } + + /** + * Delete all cells for a row with matching column family regex + * with timestamps less than or equal to timestamp. + * + * @param row The row to operate on + * @param familyRegex Column family regex + * @param timestamp Timestamp to match + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteFamilyByRegex(final byte [] row, final String familyRegex, + final long timestamp) + throws IOException { + deleteFamilyByRegex(row,familyRegex,timestamp,null); + } + + /** + * Delete all cells for a row with matching column family regex with + * timestamps less than or equal to timestamp, using existing + * row lock. + * + * @param row The row to operate on + * @param familyRegex Column Family Regex + * @param timestamp Timestamp to match + * @param r1 Existing row lock + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} + */ + public void deleteFamilyByRegex(final byte[] row, final String familyRegex, + final long timestamp, final RowLock r1) + throws IOException { + throw new UnsupportedOperationException("TODO: Not yet implemented"); + } + + /** + * Test for the existence of a row in the table. + * + * @param row The row + * @return true if the row exists, false otherwise + * @throws IOException + */ + public boolean exists(final byte [] row) throws IOException { + return exists(row, null, HConstants.LATEST_TIMESTAMP, null); + } + + /** + * Test for the existence of a row and column in the table. + * + * @param row The row + * @param column The column + * @return true if the row exists, false otherwise + * @throws IOException + */ + public boolean exists(final byte [] row, final byte[] column) + throws IOException { + return exists(row, column, HConstants.LATEST_TIMESTAMP, null); + } + + /** + * Test for the existence of a coordinate in the table. + * + * @param row The row + * @param column The column + * @param timestamp The timestamp + * @return true if the specified coordinate exists + * @throws IOException + */ + public boolean exists(final byte [] row, final byte [] column, + long timestamp) throws IOException { + return exists(row, column, timestamp, null); + } + + /** + * Test for the existence of a coordinate in the table. + * + * @param row The row + * @param column The column + * @param timestamp The timestamp + * @param rl Existing row lock + * @return true if the specified coordinate exists + * @throws IOException + */ + public boolean exists(final byte [] row, final byte [] column, + final long timestamp, final RowLock rl) throws IOException { + final Get g = new Get(row, rl); + g.addColumn(column); + g.setTimeStamp(timestamp); + return connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, row) { + public Boolean call() throws IOException { + return Boolean.valueOf(server. + exists(location.getRegionInfo().getRegionName(), g)); + } + } + ).booleanValue(); + } + + /** + * Commit a BatchUpdate to the table. + * If autoFlush is false, the update is buffered + * @param batchUpdate + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} or + * {@link #put(Put) + */ + public synchronized void commit(final BatchUpdate batchUpdate) + throws IOException { + commit(batchUpdate, 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 + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} or + * {@link #put(Put) + */ + public synchronized void commit(final BatchUpdate batchUpdate, + final RowLock rl) + throws IOException { + for (BatchOperation bo: batchUpdate) { + if (!bo.isPut()) throw new IOException("Only Puts in BU as of 0.20.0"); + Put p = new Put(batchUpdate.getRow(), rl); + p.add(bo.getColumn(),batchUpdate.getTimestamp(), bo.getValue()); + put(p); + } + } + + /** + * Commit a List of BatchUpdate to the table. + * If autoFlush is false, the updates are buffered + * @param batchUpdates + * @throws IOException + * @deprecated As of hbase 0.20.0, replaced by {@link #delete(List)} or + * {@link #put(List) + */ + public synchronized void commit(final List batchUpdates) + throws IOException { + // Am I breaking something here in old API by doing this? + for (BatchUpdate bu : batchUpdates) { + commit(bu); + } + } + + /** + * Atomically checks if a row's values match + * the expectedValues. If it does, it uses the + * batchUpdate to update the row. + * @param batchUpdate batchupdate to apply if check is successful + * @param expectedValues values to check + * @param rl rowlock + * @throws IOException + */ + public synchronized boolean checkAndSave(final BatchUpdate batchUpdate, + final HbaseMapWritable expectedValues, final RowLock rl) + throws IOException { + throw new UnsupportedOperationException("TODO: Not yet implemented"); } /** @@ -1523,59 +1716,45 @@ public class HTable { * If there are multiple regions in a table, this scanner will iterate * through them all. */ - protected class ClientScanner implements Scanner { + protected class ClientScanner implements ResultScanner { private final Log CLIENT_LOG = LogFactory.getLog(this.getClass()); - private byte[][] columns; - private byte [] startRow; - protected long scanTime; + private Scan scan; private boolean closed = false; private HRegionInfo currentRegion = null; private ScannerCallable callable = null; - protected RowFilterInterface filter; - private final LinkedList cache = new LinkedList(); - @SuppressWarnings("hiding") + private final LinkedList cache = new LinkedList(); private final int scannerCaching = HTable.this.scannerCaching; private long lastNext; - - protected ClientScanner(final byte[][] columns, final byte [] startRow, - final long timestamp, final RowFilterInterface filter) { + + protected ClientScanner(final Scan scan) { if (CLIENT_LOG.isDebugEnabled()) { CLIENT_LOG.debug("Creating scanner over " + Bytes.toString(getTableName()) - + " starting at key '" + Bytes.toString(startRow) + "'"); - } - // save off the simple parameters - this.columns = columns; - this.startRow = startRow; - this.scanTime = timestamp; - - // save the filter, and make sure that the filter applies to the data - // we're expecting to pull back - this.filter = filter; - if (filter != null) { - filter.validate(columns); + + " starting at key '" + Bytes.toString(scan.getStartRow()) + "'"); } + this.scan = scan; this.lastNext = System.currentTimeMillis(); + + // Removed filter validation. We have a new format now, only one of all + // the current filters has a validate() method. We can add it back, + // need to decide on what we're going to do re: filter redesign. + // Need, at the least, to break up family from qualifier as separate + // checks, I think it's important server-side filters are optimal in that + // respect. } - //TODO: change visibility to protected - - public void initialize() throws IOException { + protected void initialize() throws IOException { nextScanner(this.scannerCaching); } - - protected byte[][] getColumns() { - return columns; + + protected Scan getScan() { + return scan; } protected long getTimestamp() { - return scanTime; + return lastNext; } - protected RowFilterInterface getFilter() { - return filter; - } - /* * Gets a scanner for the next region. * Returns false if there are no more scanners. @@ -1603,9 +1782,10 @@ public class HTable { return false; } } - + HRegionInfo oldRegion = this.currentRegion; - byte [] localStartKey = oldRegion == null? startRow: oldRegion.getEndKey(); + byte [] localStartKey = + oldRegion == null ? scan.getStartRow() : oldRegion.getEndKey(); if (CLIENT_LOG.isDebugEnabled()) { CLIENT_LOG.debug("Advancing internal scanner to startKey at '" + @@ -1628,8 +1808,7 @@ public class HTable { protected ScannerCallable getScannerCallable(byte [] localStartKey, int nbRows) { ScannerCallable s = new ScannerCallable(getConnection(), - getTableName(), columns, - localStartKey, scanTime, filter); + getTableName(), localStartKey, scan); s.setCaching(nbRows); return s; } @@ -1640,22 +1819,22 @@ public class HTable { * filter. */ private boolean filterSaysStop(final byte [] endKey) { - if (this.filter == null) { + if(!scan.hasFilter()) { return false; } // Let the filter see current row. - this.filter.filterRowKey(endKey, 0, endKey.length); - return this.filter.filterAllRemaining(); + scan.getFilter().filterRowKey(endKey, 0, endKey.length); + return scan.getFilter().filterAllRemaining(); } - public RowResult next() throws IOException { + public Result next() throws IOException { // If the scanner is closed but there is some rows left in the cache, // it will first empty it before returning null if (cache.size() == 0 && this.closed) { return null; } if (cache.size() == 0) { - RowResult[] values = null; + Result [] values = null; int countdown = this.scannerCaching; // We need to reset it if it's a new callable that was created // with a countdown in nextScanner @@ -1674,7 +1853,7 @@ public class HTable { } lastNext = System.currentTimeMillis(); if (values != null && values.length > 0) { - for (RowResult rs : values) { + for (Result rs : values) { cache.add(rs); countdown--; } @@ -1693,18 +1872,18 @@ public class HTable { * @return Between zero and nbRows RowResults * @throws IOException */ - public RowResult[] next(int nbRows) throws IOException { + public Result [] next(int nbRows) throws IOException { // Collect values to be returned here - ArrayList resultSets = new ArrayList(nbRows); + ArrayList resultSets = new ArrayList(nbRows); for(int i = 0; i < nbRows; i++) { - RowResult next = next(); + Result next = next(); if (next != null) { resultSets.add(next); } else { break; } } - return resultSets.toArray(new RowResult[resultSets.size()]); + return resultSets.toArray(new Result[resultSets.size()]); } public void close() { @@ -1723,6 +1902,88 @@ public class HTable { closed = true; } + public Iterator iterator() { + return new Iterator() { + // The next RowResult, possibly pre-read + Result next = null; + + // return true if there is another item pending, false if there isn't. + // this method is where the actual advancing takes place, but you need + // to call next() to consume it. hasNext() will only advance if there + // isn't a pending next(). + public boolean hasNext() { + if (next == null) { + try { + next = ClientScanner.this.next(); + return next != null; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return true; + } + + // get the pending next item and advance the iterator. returns null if + // there is no next item. + public Result next() { + // since hasNext() does the real advancing, we call this to determine + // if there is a next before proceeding. + if (!hasNext()) { + return null; + } + + // if we get to here, then hasNext() has given us an item to return. + // we want to return the item and then null out the next pointer, so + // we use a temporary variable. + Result temp = next; + next = null; + return temp; + } + + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + } + + /** + * {@link Scanner} implementation made on top of a {@link ResultScanner}. + */ + protected class OldClientScanner implements Scanner { + private final ClientScanner cs; + + OldClientScanner(final ClientScanner cs) { + this.cs = cs; + } + + protected void initialize() throws IOException { + this.cs.initialize(); + } + + @Override + public void close() { + this.cs.close(); + } + + @Override + public RowResult next() throws IOException { + Result r = this.cs.next(); + return r == null || r.isEmpty()? null: r.getRowResult(); + } + + @Override + public RowResult [] next(int nbRows) throws IOException { + Result [] rr = this.cs.next(nbRows); + if (rr == null || rr.length == 0) return null; + RowResult [] results = new RowResult[rr.length]; + for (int i = 0; i < rr.length; i++) { + results[i] = rr[i].getRowResult(); + } + return results; + } + + @Override public Iterator iterator() { return new Iterator() { // The next RowResult, possibly pre-read @@ -1735,7 +1996,7 @@ public class HTable { public boolean hasNext() { if (next == null) { try { - next = ClientScanner.this.next(); + next = OldClientScanner.this.next(); return next != null; } catch (IOException e) { throw new RuntimeException(e); @@ -1767,4 +2028,4 @@ public class HTable { }; } } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/client/HTablePool.java b/src/java/org/apache/hadoop/hbase/client/HTablePool.java index e8676c5d90b..7116d56b17f 100755 --- a/src/java/org/apache/hadoop/hbase/client/HTablePool.java +++ b/src/java/org/apache/hadoop/hbase/client/HTablePool.java @@ -47,6 +47,7 @@ public class HTablePool { /** * Get a shared table pool. + * @param config * @param tableName the table name * @return the table pool */ diff --git a/src/java/org/apache/hadoop/hbase/client/MetaScanner.java b/src/java/org/apache/hadoop/hbase/client/MetaScanner.java index 689ca01db14..19eb6c4809b 100644 --- a/src/java/org/apache/hadoop/hbase/client/MetaScanner.java +++ b/src/java/org/apache/hadoop/hbase/client/MetaScanner.java @@ -5,7 +5,6 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.util.Bytes; /** @@ -49,14 +48,14 @@ class MetaScanner implements HConstants { // Scan over each meta region ScannerCallable callable = null; do { - callable = new ScannerCallable(connection, META_TABLE_NAME, - COLUMN_FAMILY_ARRAY, startRow, LATEST_TIMESTAMP, null); + Scan scan = new Scan(startRow).addFamily(CATALOG_FAMILY); + callable = new ScannerCallable(connection, META_TABLE_NAME, scan.getStartRow(), scan); // Open scanner connection.getRegionServerWithRetries(callable); try { - RowResult r = null; + Result r = null; do { - RowResult [] rrs = connection.getRegionServerWithRetries(callable); + Result [] rrs = connection.getRegionServerWithRetries(callable); if (rrs == null || rrs.length == 0 || rrs[0].size() == 0) { break; } @@ -85,6 +84,6 @@ class MetaScanner implements HConstants { * @return A boolean to know if it should continue to loop in the region * @throws IOException */ - public boolean processRow(RowResult rowResult) throws IOException; + public boolean processRow(Result rowResult) throws IOException; } } diff --git a/src/java/org/apache/hadoop/hbase/client/Put.java b/src/java/org/apache/hadoop/hbase/client/Put.java new file mode 100644 index 00000000000..b3225ef15b7 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/client/Put.java @@ -0,0 +1,305 @@ +/* + * 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.client; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +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.io.HeapSize; +import org.apache.hadoop.hbase.util.Bytes; + + +/** + * Used to perform Put operations for a single row. + *

+ * To perform a Put, instantiate a Put object with the row to insert to and + * for each column to be inserted, execute {@link #add(byte[], byte[], byte[]) add} or + * {@link #add(byte[], byte[], long, byte[]) add} if setting the timestamp. + */ +public class Put implements HeapSize, Writable, Comparable { + private byte [] row = null; + private long timestamp = HConstants.LATEST_TIMESTAMP; + private long lockId = -1L; + private Map> familyMap = + new TreeMap>(Bytes.BYTES_COMPARATOR); + + /** Constructor for Writable. DO NOT USE */ + public Put() {} + + /** + * Create a Put operation for the specified row. + * @param row row key + */ + public Put(byte [] row) { + this(row, null); + } + + /** + * Create a Put operation for the specified row, using an existing row lock. + * @param row row key + * @param rowLock previously acquired row lock, or null + */ + public Put(byte [] row, RowLock rowLock) { + if(row == null || row.length > HConstants.MAX_ROW_LENGTH) { + throw new IllegalArgumentException("Row key is invalid"); + } + this.row = row; + if(rowLock != null) { + this.lockId = rowLock.getLockId(); + } + } + + /** + * Copy constructor. Creates a Put operation cloned from the specified Put. + * @param putToCopy put to copy + */ + public Put(Put putToCopy) { + this(putToCopy.getRow(), putToCopy.getRowLock()); + this.familyMap = + new TreeMap>(Bytes.BYTES_COMPARATOR); + for(Map.Entry> entry : + putToCopy.getFamilyMap().entrySet()) { + this.familyMap.put(entry.getKey(), entry.getValue()); + } + } + + /** + * Add the specified column and value to this Put operation. + * @param family family name + * @param qualifier column qualifier + * @param value column value + */ + public void add(byte [] family, byte [] qualifier, byte [] value) { + add(family, qualifier, this.timestamp, value); + } + + /** + * Add the specified column and value, with the specified timestamp as + * its version to this Put operation. + * @param column Old style column name with family and qualifier put together + * with a colon. + * @param timestamp version timestamp + * @param value column value + */ + public void add(byte [] column, long timestamp, byte [] value) { + byte [][] parts = KeyValue.parseColumn(column); + add(parts[0], parts[1], timestamp, value); + } + + /** + * Add the specified column and value, with the specified timestamp as + * its version to this Put operation. + * @param family family name + * @param qualifier column qualifier + * @param timestamp version timestamp + * @param value column value + */ + public void add(byte [] family, byte [] qualifier, long timestamp, byte [] value) { + List list = familyMap.get(family); + if(list == null) { + list = new ArrayList(); + } + KeyValue kv = new KeyValue(this.row, family, qualifier, timestamp, + KeyValue.Type.Put, value); + list.add(kv); + familyMap.put(family, list); + } + + /** + * Add the specified KeyValue to this Put operation. + * @param kv + */ + public void add(KeyValue kv) { + byte [] family = kv.getFamily(); + List list = familyMap.get(family); + if(list == null) { + list = new ArrayList(); + } + list.add(kv); + familyMap.put(family, list); + } + + + /** + * Method for retrieving the put's familyMap + * @return familyMap + */ + public Map> getFamilyMap() { + return this.familyMap; + } + + /** + * Method for retrieving the put's row + * @return row + */ + public byte [] getRow() { + return this.row; + } + + /** + * Method for retrieving the put's RowLock + * @return RowLock + */ + public RowLock getRowLock() { + return new RowLock(this.row, this.lockId); + } + + /** + * Method for retrieving the put's lockId + * @return lockId + */ + public long getLockId() { + return this.lockId; + } + + /** + * Method to check if the familyMap is empty + * @return true if empty, false otherwise + */ + public boolean isEmpty() { + return familyMap.isEmpty(); + } + + /** + * Method for setting the timestamp + * @param timestamp + */ + public void setTimeStamp(long timestamp) { + this.timestamp = timestamp; + } + + public int numFamilies() { + return familyMap.size(); + } + + public int size() { + int size = 0; + for(List kvList : this.familyMap.values()) { + size += kvList.size(); + } + return size; + } + + /** + * @return String + */ + @Override + public String toString() { + StringBuffer sb = new StringBuffer(); + sb.append("row="); + sb.append(Bytes.toString(this.row)); + sb.append(", families={"); + boolean moreThanOne = false; + for(Map.Entry> entry : this.familyMap.entrySet()) { + if(moreThanOne) { + sb.append(", "); + } else { + moreThanOne = true; + } + sb.append("(family="); + sb.append(Bytes.toString(entry.getKey())); + sb.append(", keyvalues=("); + boolean moreThanOneB = false; + for(KeyValue kv : entry.getValue()) { + if(moreThanOneB) { + sb.append(", "); + } else { + moreThanOneB = true; + } + sb.append(kv.toString()); + } + sb.append(")"); + } + sb.append("}"); + return sb.toString(); + } + + public int compareTo(Put p) { + return Bytes.compareTo(this.getRow(), p.getRow()); + } + + //HeapSize + public long heapSize() { + long totalSize = 0; + for(Map.Entry> entry : this.familyMap.entrySet()) { + for(KeyValue kv : entry.getValue()) { + totalSize += kv.heapSize(); + } + } + return totalSize; + } + + //Writable + public void readFields(final DataInput in) + throws IOException { + this.row = Bytes.readByteArray(in); + this.timestamp = in.readLong(); + this.lockId = in.readLong(); + int numFamilies = in.readInt(); + this.familyMap = + new TreeMap>(Bytes.BYTES_COMPARATOR); + for(int i=0;i keys = new ArrayList(numKeys); + int totalLen = in.readInt(); + byte [] buf = new byte[totalLen]; + int offset = 0; + for(int j=0;j> entry : familyMap.entrySet()) { + Bytes.writeByteArray(out, entry.getKey()); + List keys = entry.getValue(); + out.writeInt(keys.size()); + int totalLen = 0; + for(KeyValue kv : keys) { + totalLen += kv.getLength(); + } + out.writeInt(totalLen); + for(KeyValue kv : keys) { + out.writeInt(kv.getLength()); + out.write(kv.getBuffer(), kv.getOffset(), kv.getLength()); + } + } + } +} diff --git a/src/java/org/apache/hadoop/hbase/client/Result.java b/src/java/org/apache/hadoop/hbase/client/Result.java new file mode 100644 index 00000000000..ba700dfa050 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/client/Result.java @@ -0,0 +1,479 @@ +/* + * 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.client; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValue.SplitKeyValue; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; + +import org.apache.hadoop.io.Writable; + +/** + * Single row result of a {@link Get} or {@link Scan} query. + */ +public class Result implements Writable { + private KeyValue [] kvs = null; + private NavigableMap>> familyMap = null; + // We're not using java serialization. Transient here is just a marker to say + // that this is where we cache row if we're ever asked for it. + private transient byte [] row = null; + + /** + * Constructor used for Writable. + */ + public Result() {} + + /** + * Instantiate a Result with the specified array of KeyValues. + * @param kvs array of KeyValues + */ + public Result(KeyValue [] kvs) { + if(kvs != null && kvs.length > 0) { + this.kvs = kvs; + } + } + + /** + * Instantiate a Result with the specified List of KeyValues. + * @param kvs List of KeyValues + */ + public Result(List kvs) { + this(kvs.toArray(new KeyValue[0])); + } + + /** + * Method for retrieving the row that this result is for + * @return row + */ + public synchronized byte [] getRow() { + if (this.row == null) { + this.row = + this.kvs == null || this.kvs.length == 0? null: this.kvs[0].getRow(); + } + return this.row; + } + + /** + * Directly return the unsorted array of KeyValues in this Result. + * @return unsorted array of KeyValues + */ + public KeyValue[] raw() { + return kvs; + } + + /** + * Return a sorted list of the KeyValues in this result. + * @return + */ + public List list() { + return Arrays.asList(sorted()); + } + + /** + * Returns a sorted array of KeyValues in this Result. + *

+ * Note: Sorting is done in place, so the backing array will be sorted + * after calling this method. + * @return sorted array of KeyValues + */ + public KeyValue[] sorted() { + if(isEmpty()) { + return null; + } + Arrays.sort(kvs, (Comparator)KeyValue.COMPARATOR); + return kvs; + } + + /** + * Map of families to all versions of its qualifiers and values. + *

+ * Returns a three level Map of the form: + * Map>> + *

+ * Note: All other map returning methods make use of this map internally. + * @return map from families to qualifiers to versions + */ + public NavigableMap>> getMap() { + if(this.familyMap != null) { + return this.familyMap; + } + if(isEmpty()) { + return null; + } + this.familyMap = + new TreeMap>> + (Bytes.BYTES_COMPARATOR); + for(KeyValue kv : this.kvs) { + SplitKeyValue splitKV = kv.split(); + byte [] family = splitKV.getFamily(); + NavigableMap> columnMap = + familyMap.get(family); + if(columnMap == null) { + columnMap = new TreeMap> + (Bytes.BYTES_COMPARATOR); + familyMap.put(family, columnMap); + } + byte [] qualifier = splitKV.getQualifier(); + NavigableMap versionMap = columnMap.get(qualifier); + if(versionMap == null) { + versionMap = new TreeMap(new Comparator() { + public int compare(Long l1, Long l2) { + return l2.compareTo(l1); + } + }); + columnMap.put(qualifier, versionMap); + } + Long timestamp = Bytes.toLong(splitKV.getTimestamp()); + byte [] value = splitKV.getValue(); + versionMap.put(timestamp, value); + } + return this.familyMap; + } + + /** + * Map of families to their most recent qualifiers and values. + *

+ * Returns a two level Map of the form: Map> + *

+ * The most recent version of each qualifier will be used. + * @return map from families to qualifiers and value + */ + public NavigableMap> getNoVersionMap() { + if(this.familyMap == null) { + getMap(); + } + if(isEmpty()) { + return null; + } + NavigableMap> returnMap = + new TreeMap>(Bytes.BYTES_COMPARATOR); + for(Map.Entry>> + familyEntry : familyMap.entrySet()) { + NavigableMap qualifierMap = + new TreeMap(Bytes.BYTES_COMPARATOR); + for(Map.Entry> qualifierEntry : + familyEntry.getValue().entrySet()) { + byte [] value = + qualifierEntry.getValue().get(qualifierEntry.getValue().firstKey()); + qualifierMap.put(qualifierEntry.getKey(), value); + } + returnMap.put(familyEntry.getKey(), qualifierMap); + } + return returnMap; + } + + /** + * Map of qualifiers to values. + *

+ * Returns a Map of the form: Map + * @return map of qualifiers to values + */ + public NavigableMap getFamilyMap(byte [] family) { + if(this.familyMap == null) { + getMap(); + } + if(isEmpty()) { + return null; + } + NavigableMap returnMap = + new TreeMap(Bytes.BYTES_COMPARATOR); + NavigableMap> qualifierMap = + familyMap.get(family); + if(qualifierMap == null) { + return returnMap; + } + for(Map.Entry> entry : + qualifierMap.entrySet()) { + byte [] value = + entry.getValue().get(entry.getValue().firstKey()); + returnMap.put(entry.getKey(), value); + } + return returnMap; + } + + /** + * Get the latest version of the specified column. + * @param family family name + * @param qualifier column qualifier + * @return value of latest version of column, null if none found + */ + public byte [] getValue(byte [] family, byte [] qualifier) { + Map.Entry entry = getKeyValue(family, qualifier); + return entry == null ?null :entry.getValue(); + } + + public Cell getCellValue(byte[] family, byte[] qualifier) { + Map.Entry val = getKeyValue(family, qualifier); + return new Cell(val.getValue(), val.getKey()); + } + + /** + * @return First KeyValue in this Result as a Cell or null if empty. + */ + public Cell getCellValue() { + return isEmpty()? null: new Cell(kvs[0].getValue(), kvs[0].getTimestamp()); + } + + /** + * @return This Result as array of Cells or null if empty. + */ + public Cell [] getCellValues() { + if (isEmpty()) return null; + Cell [] results = new Cell[kvs.length]; + for (int i = 0; i < kvs.length; i++) { + results[i] = new Cell(kvs[i].getValue(), kvs[i].getTimestamp()); + } + return results; + } + + private Map.Entry getKeyValue(byte[] family, byte[] qualifier) { + if(this.familyMap == null) { + getMap(); + } + if(isEmpty()) { + return null; + } + NavigableMap> qualifierMap = + familyMap.get(family); + if(qualifierMap == null) { + return null; + } + NavigableMap versionMap = + getVersionMap(qualifierMap, qualifier); + if(versionMap == null) { + return null; + } + return versionMap.firstEntry(); + } + + private NavigableMap getVersionMap( + NavigableMap> qualifierMap, byte [] qualifier) { + if(qualifier != null) { + return qualifierMap.get(qualifier); + } else { + return qualifierMap.get(new byte[0]); + } + } + + /** + * Get the latest version of the specified column, + * using

family:qualifier
notation. + * @param column column in family:qualifier notation + * @return value of latest version of column, null if none found + */ + public byte [] getValue(byte [] column) { + try { + byte [][] split = KeyValue.parseColumn(column); + return getValue(split[0], split[1]); + } catch(Exception e) { + return null; + } + } + + /** + * Checks for existence of the specified column. + * @param family family name + * @param qualifier column qualifier + * @return true if at least one value exists in the result, false if not + */ + public boolean containsColumn(byte [] family, byte [] qualifier) { + if(this.familyMap == null) { + getMap(); + } + if(isEmpty()) { + return false; + } + NavigableMap> qualifierMap = + familyMap.get(family); + if(qualifierMap == null) { + return false; + } + NavigableMap versionMap = getVersionMap(qualifierMap, qualifier); + if(versionMap == null) { + return false; + } + return true; + } + + /** + * Returns this Result in the old return format, {@link RowResult}. + * @return a RowResult + */ + public RowResult getRowResult() { + return RowResult.createRowResult(Arrays.asList(kvs)); + } + + /** + * Returns the value of the first column in the Result. + * @return value of the first column + */ + public byte [] value() { + if(isEmpty()) { + return null; + } + return kvs[0].getValue(); + } + + /** + * Check if the underlying KeyValue [] is empty or not + * @return true if empty + */ + public boolean isEmpty() { + return (this.kvs == null || this.kvs.length == 0); + } + + /** + * + * @return the size of the underlying KeyValue [] + */ + public int size() { + return (this.kvs == null ? 0 : this.kvs.length); + } + + /** + * @return String + */ + @Override + public String toString() { + StringBuffer sb = new StringBuffer(); + sb.append("keyvalues="); + if(isEmpty()) { + sb.append("NONE"); + return sb.toString(); + } + sb.append("{"); + boolean moreThanOne = false; + for(KeyValue kv : this.kvs) { + if(moreThanOne) { + sb.append(", "); + } else { + moreThanOne = true; + } + sb.append(kv.toString()); + } + sb.append("}"); + return sb.toString(); + } + + //Writable + public void readFields(final DataInput in) + throws IOException { + int numKeys = in.readInt(); + this.kvs = new KeyValue[numKeys]; + if(numKeys == 0) { + return; + } + int totalBuffer = in.readInt(); + byte [] buf = new byte[totalBuffer]; + int offset = 0; + for(int i=0; i { + + /** + * Grab the next row's worth of values. The scanner will return a Result. + * @return Result object if there is another row, null if the scanner is + * exhausted. + * @throws IOException + */ + public Result next() throws IOException; + + /** + * @param nbRows number of rows to return + * @return Between zero and nbRows Results + * @throws IOException + */ + public Result [] next(int nbRows) throws IOException; + + /** + * Closes the scanner and releases any resources it has allocated + */ + public void close(); +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/client/Scan.java b/src/java/org/apache/hadoop/hbase/client/Scan.java new file mode 100644 index 00000000000..e84a0d68096 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/client/Scan.java @@ -0,0 +1,456 @@ +/* + * 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.client; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.io.HbaseObjectWritable; +import org.apache.hadoop.hbase.io.TimeRange; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.Writable; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Map; +import java.util.NavigableSet; +import java.util.TreeMap; +import java.util.TreeSet; + +/** + * Used to perform Scan operations. + *

+ * All operations are identical to {@link Get} with the exception of + * instantiation. Rather than specifying a single row, an optional startRow + * and stopRow may be defined. If rows are not specified, the Scanner will + * iterate over all rows. + *

+ * To scan everything for each row, instantiate a Scan object. + * To further define the scope of what to get when scanning, perform additional + * methods as outlined below. + *

+ * To get all columns from specific families, execute {@link #addFamily(byte[]) addFamily} + * for each family to retrieve. + *

+ * To get specific columns, execute {@link #addColumn(byte[], byte[]) addColumn} + * for each column to retrieve. + *

+ * To only retrieve columns within a specific range of version timestamps, + * execute {@link #setTimeRange(long, long) setTimeRange}. + *

+ * To only retrieve columns with a specific timestamp, execute + * {@link #setTimeStamp(long) setTimestamp}. + *

+ * To limit the number of versions of each column to be returned, execute + * {@link #setMaxVersions(int) setMaxVersions}. + *

+ * To add a filter, execute {@link #setFilter(org.apache.hadoop.hbase.filter.Filter) setFilter}. + */ +public class Scan implements Writable { + private byte [] startRow = HConstants.EMPTY_START_ROW; + private byte [] stopRow = HConstants.EMPTY_END_ROW; + private int maxVersions = 1; + private Filter filter = null; + private RowFilterInterface oldFilter = null; + private TimeRange tr = new TimeRange(); + private Map> familyMap = + new TreeMap>(Bytes.BYTES_COMPARATOR); + + /** + * Create a Scan operation across all rows. + */ + public Scan() {} + + public Scan(byte [] startRow, Filter filter) { + this(startRow); + this.filter = filter; + + } + + /** + * Create a Scan operation starting at the specified row. + *

+ * If the specified row does not exist, the Scanner will start from the + * next closest row after the specified row. + * @param startRow row to start scanner at or after + */ + public Scan(byte [] startRow) { + this.startRow = startRow; + } + + /** + * Create a Scan operation for the range of rows specified. + * @param startRow row to start scanner at or after (inclusive) + * @param stopRow row to stop scanner before (exclusive) + */ + public Scan(byte [] startRow, byte [] stopRow) { + this.startRow = startRow; + this.stopRow = stopRow; + } + + /** + * Get all columns from the specified family. + *

+ * Overrides previous calls to addColumn for this family. + * @param family family name + */ + public Scan addFamily(byte [] family) { + familyMap.remove(family); + familyMap.put(family, null); + + return this; + } + + /** + * Get the column from the specified family with the specified qualifier. + *

+ * Overrides previous calls to addFamily for this family. + * @param family family name + * @param qualifier column qualifier + */ + public Scan addColumn(byte [] family, byte [] qualifier) { + NavigableSet set = familyMap.get(family); + if(set == null) { + set = new TreeSet(Bytes.BYTES_COMPARATOR); + } + set.add(qualifier); + familyMap.put(family, set); + + return this; + } + + /** + * Adds an array of columns specified the old format, family:qualifier. + *

+ * Overrides previous calls to addFamily for any families in the input. + * @param columns array of columns, formatted as

family:qualifier
+ */ + public Scan addColumns(byte [][] columns) { + for(int i=0; i> familyMap) { + this.familyMap = familyMap; + + return this; + } + + /** + * Getting the familyMap + * @return familyMap + */ + public Map> getFamilyMap() { + return this.familyMap; + } + + /** + * @return the number of families in familyMap + */ + public int numFamilies() { + if(hasFamilies()) { + return this.familyMap.size(); + } + return 0; + } + + /** + * @return true if familyMap is non empty, false otherwise + */ + public boolean hasFamilies() { + return !this.familyMap.isEmpty(); + } + + /** + * @return the keys of the familyMap + */ + public byte[][] getFamilies() { + if(hasFamilies()) { + return this.familyMap.keySet().toArray(new byte[0][0]); + } + return null; + } + + /** + * @return the startrow + */ + public byte [] getStartRow() { + return this.startRow; + } + + /** + * @return the stoprow + */ + public byte [] getStopRow() { + return this.stopRow; + } + + /** + * @return the max number of versions to fetch + */ + public int getMaxVersions() { + return this.maxVersions; + } + + /** + * @return TimeRange + */ + public TimeRange getTimeRange() { + return this.tr; + } + + /** + * @return RowFilter + */ + public Filter getFilter() { + return filter; + } + + /** + * Get the old style filter, if there is one. + * @deprecated + * @return null or instance + */ + public RowFilterInterface getOldFilter() { + return oldFilter; + } + + /** + * @return true is a filter has been specified, false if not + */ + public boolean hasFilter() { + return filter != null || oldFilter != null; + } + + /** + * @return String + */ + @Override + public String toString() { + StringBuffer sb = new StringBuffer(); + sb.append("startRow="); + sb.append(Bytes.toString(this.startRow)); + sb.append(", stopRow="); + sb.append(Bytes.toString(this.stopRow)); + sb.append(", maxVersions="); + sb.append("" + this.maxVersions); + sb.append(", timeRange="); + sb.append("[" + this.tr.getMin() + "," + this.tr.getMax() + ")"); + sb.append(", families="); + if(this.familyMap.size() == 0) { + sb.append("ALL"); + return sb.toString(); + } + boolean moreThanOne = false; + for(Map.Entry> entry : this.familyMap.entrySet()) { + if(moreThanOne) { + sb.append("), "); + } else { + moreThanOne = true; + sb.append("{"); + } + sb.append("(family="); + sb.append(Bytes.toString(entry.getKey())); + sb.append(", columns="); + if(entry.getValue() == null) { + sb.append("ALL"); + } else { + sb.append("{"); + boolean moreThanOneB = false; + for(byte [] column : entry.getValue()) { + if(moreThanOneB) { + sb.append(", "); + } else { + moreThanOneB = true; + } + sb.append(Bytes.toString(column)); + } + sb.append("}"); + } + } + sb.append("}"); + return sb.toString(); + } + + //Writable + public void readFields(final DataInput in) + throws IOException { + this.startRow = Bytes.readByteArray(in); + this.stopRow = Bytes.readByteArray(in); + this.maxVersions = in.readInt(); + boolean hasFilter = in.readBoolean(); + if(hasFilter) { + this.filter = (Filter)HbaseObjectWritable.readObject(in, + new Configuration()); + } + boolean hasOldFilter = in.readBoolean(); + if (hasOldFilter) { + this.oldFilter = (RowFilterInterface)HbaseObjectWritable.readObject(in, + new Configuration()); + } + this.tr = new TimeRange(); + tr.readFields(in); + int numFamilies = in.readInt(); + this.familyMap = + new TreeMap>(Bytes.BYTES_COMPARATOR); + for(int i=0; i set = new TreeSet(Bytes.BYTES_COMPARATOR); + for(int j=0; j> entry : familyMap.entrySet()) { + Bytes.writeByteArray(out, entry.getKey()); + NavigableSet columnSet = entry.getValue(); + if(columnSet != null){ + out.writeInt(columnSet.size()); + for(byte [] qualifier : columnSet) { + Bytes.writeByteArray(out, qualifier); + } + } else { + out.writeInt(0); + } + } + } +} diff --git a/src/java/org/apache/hadoop/hbase/client/Scanner.java b/src/java/org/apache/hadoop/hbase/client/Scanner.java index 5f50f424632..42b0ddbba40 100644 --- a/src/java/org/apache/hadoop/hbase/client/Scanner.java +++ b/src/java/org/apache/hadoop/hbase/client/Scanner.java @@ -1,5 +1,5 @@ /** - * Copyright 2008 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -21,34 +21,32 @@ package org.apache.hadoop.hbase.client; import java.io.Closeable; import java.io.IOException; + import org.apache.hadoop.hbase.io.RowResult; /** * Interface for client-side scanning. * Go to {@link HTable} to obtain instances. + * @deprecated See {@link ResultScanner} */ public interface Scanner extends Closeable, Iterable { /** - * Grab the next row's worth of values. The scanner will return a RowResult - * that contains both the row's key and a map of byte[] column names to Cell - * value objects. The data returned will only contain the most recent data - * value for each row that is not newer than the target time passed when the - * scanner was created. + * Grab the next row's worth of values. * @return RowResult object if there is another row, null if the scanner is * exhausted. * @throws IOException */ public RowResult next() throws IOException; - + /** * @param nbRows number of rows to return - * @return Between zero and nbRows RowResults + * @return Between zero and nbRows Results * @throws IOException */ public RowResult [] next(int nbRows) throws IOException; - + /** * Closes the scanner and releases any resources it has allocated */ - public void close(); + public void close(); } \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java index c43598d035a..8b5d26013c6 100644 --- a/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java +++ b/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java @@ -23,37 +23,31 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.filter.RowFilterInterface; -import org.apache.hadoop.hbase.io.RowResult; /** * Retries scanner operations such as create, next, etc. - * Used by {@link Scanner}s made by {@link HTable}. + * Used by {@link ResultScanner}s made by {@link HTable}. */ -public class ScannerCallable extends ServerCallable { +public class ScannerCallable extends ServerCallable { private long scannerId = -1L; private boolean instantiated = false; private boolean closed = false; - private final byte [][] columns; - private final long timestamp; - private final RowFilterInterface filter; + private Scan scan; + private byte [] startRow; private int caching = 1; /** * @param connection * @param tableName - * @param columns * @param startRow - * @param timestamp - * @param filter + * @param scan */ - public ScannerCallable (HConnection connection, byte [] tableName, byte [][] columns, - byte [] startRow, long timestamp, RowFilterInterface filter) { + public ScannerCallable (HConnection connection, byte [] tableName, + byte [] startRow, Scan scan) { super(connection, tableName, startRow); - this.columns = columns; - this.timestamp = timestamp; - this.filter = filter; + this.scan = scan; + this.startRow = startRow; } /** @@ -71,7 +65,7 @@ public class ScannerCallable extends ServerCallable { /** * @see java.util.concurrent.Callable#call() */ - public RowResult[] call() throws IOException { + public Result [] call() throws IOException { if (scannerId != -1L && closed) { server.close(scannerId); scannerId = -1L; @@ -79,28 +73,19 @@ public class ScannerCallable extends ServerCallable { // open the scanner scannerId = openScanner(); } else { - RowResult [] rrs = server.next(scannerId, caching); - return rrs.length == 0 ? null : rrs; + Result [] rrs = server.next(scannerId, caching); + return rrs == null || rrs.length == 0? null: rrs; } return null; } protected long openScanner() throws IOException { return server.openScanner( - this.location.getRegionInfo().getRegionName(), columns, row, - timestamp, filter); + this.location.getRegionInfo().getRegionName(), scan); } - protected byte [][] getColumns() { - return columns; - } - - protected long getTimestamp() { - return timestamp; - } - - protected RowFilterInterface getFilter() { - return filter; + protected Scan getScan() { + return scan; } /** diff --git a/src/java/org/apache/hadoop/hbase/client/UnmodifyableHColumnDescriptor.java b/src/java/org/apache/hadoop/hbase/client/UnmodifyableHColumnDescriptor.java index eee609c4286..9bc569a0fe8 100644 --- a/src/java/org/apache/hadoop/hbase/client/UnmodifyableHColumnDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/client/UnmodifyableHColumnDescriptor.java @@ -55,14 +55,6 @@ public class UnmodifyableHColumnDescriptor extends HColumnDescriptor { throw new UnsupportedOperationException("HColumnDescriptor is read-only"); } - /** - * @see org.apache.hadoop.hbase.HColumnDescriptor#setMaxValueLength(int) - */ - @Override - public void setMaxValueLength(int maxLength) { - throw new UnsupportedOperationException("HColumnDescriptor is read-only"); - } - /** * @see org.apache.hadoop.hbase.HColumnDescriptor#setTimeToLive(int) */ diff --git a/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java b/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java index 8d3e002ed1b..8dd4c6fe4cb 100644 --- a/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java @@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification; +//import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification; /** * Read-only table descriptor. @@ -37,10 +37,14 @@ public class UnmodifyableHTableDescriptor extends HTableDescriptor { * Create an unmodifyable copy of an HTableDescriptor * @param desc */ +// UnmodifyableHTableDescriptor(final HTableDescriptor desc) { +// super(desc.getName(), getUnmodifyableFamilies(desc), desc.getIndexes(), desc.getValues()); +// } UnmodifyableHTableDescriptor(final HTableDescriptor desc) { - super(desc.getName(), getUnmodifyableFamilies(desc), desc.getIndexes(), desc.getValues()); + super(desc.getName(), getUnmodifyableFamilies(desc), desc.getValues()); } + /* * @param desc * @return Families as unmodifiable array. @@ -122,11 +126,11 @@ public class UnmodifyableHTableDescriptor extends HTableDescriptor { throw new UnsupportedOperationException("HTableDescriptor is read-only"); } - /** - * @see org.apache.hadoop.hbase.HTableDescriptor#addIndex(org.apache.hadoop.hbase.client.tableindexed.IndexSpecification) - */ - @Override - public void addIndex(IndexSpecification index) { - throw new UnsupportedOperationException("HTableDescriptor is read-only"); - } +// /** +// * @see org.apache.hadoop.hbase.HTableDescriptor#addIndex(org.apache.hadoop.hbase.client.tableindexed.IndexSpecification) +// */ +// @Override +// public void addIndex(IndexSpecification index) { +// throw new UnsupportedOperationException("HTableDescriptor is read-only"); +// } } diff --git a/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexKeyGenerator.java b/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexKeyGenerator.java deleted file mode 100644 index dae811ee3b0..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexKeyGenerator.java +++ /dev/null @@ -1,29 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.tableindexed; - -import java.util.Map; - -import org.apache.hadoop.io.Writable; - -public interface IndexKeyGenerator extends Writable { - - byte [] createIndexKey(byte [] rowKey, Map columns); -} diff --git a/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexNotFoundException.java b/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexNotFoundException.java deleted file mode 100644 index 3e6169c4b6b..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexNotFoundException.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.tableindexed; - -import java.io.IOException; - -/** - * Thrown when asking for an index that does not exist. - */ -public class IndexNotFoundException extends IOException { - - private static final long serialVersionUID = 6533971528557000965L; - - public IndexNotFoundException() { - super(); - } - - public IndexNotFoundException(String arg0) { - super(arg0); - } - - public IndexNotFoundException(Throwable arg0) { - super(arg0.getMessage()); - } - - public IndexNotFoundException(String arg0, Throwable arg1) { - super(arg0+arg1.getMessage()); - } - -} diff --git a/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexSpecification.java b/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexSpecification.java deleted file mode 100644 index 54f8c62cf55..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexSpecification.java +++ /dev/null @@ -1,190 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.tableindexed; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.ObjectWritable; -import org.apache.hadoop.io.Writable; - -/** Holds the specification for a single secondary index. */ -public class IndexSpecification implements Writable { - - // Columns that are indexed (part of the indexRowKey) - private byte[][] indexedColumns; - - // Constructs the - private IndexKeyGenerator keyGenerator; - - // Additional columns mapped into the indexed row. These will be available for - // filters when scanning the index. - private byte[][] additionalColumns; - - private byte[][] allColumns; - - // Id of this index, unique within a table. - private String indexId; - - /** Construct an "simple" index spec for a single column. - * @param indexId - * @param indexedColumn - */ - public IndexSpecification(String indexId, byte[] indexedColumn) { - this(indexId, new byte[][] { indexedColumn }, null, - new SimpleIndexKeyGenerator(indexedColumn)); - } - - /** - * Construct an index spec by specifying everything. - * - * @param indexId - * @param indexedColumns - * @param additionalColumns - * @param keyGenerator - */ - public IndexSpecification(String indexId, byte[][] indexedColumns, - byte[][] additionalColumns, IndexKeyGenerator keyGenerator) { - this.indexId = indexId; - this.indexedColumns = indexedColumns; - this.additionalColumns = additionalColumns; - this.keyGenerator = keyGenerator; - this.makeAllColumns(); - } - - public IndexSpecification() { - // For writable - } - - private void makeAllColumns() { - this.allColumns = new byte[indexedColumns.length - + (additionalColumns == null ? 0 : additionalColumns.length)][]; - System.arraycopy(indexedColumns, 0, allColumns, 0, indexedColumns.length); - if (additionalColumns != null) { - System.arraycopy(additionalColumns, 0, allColumns, indexedColumns.length, - additionalColumns.length); - } - } - - /** - * Get the indexedColumns. - * - * @return Return the indexedColumns. - */ - public byte[][] getIndexedColumns() { - return indexedColumns; - } - - /** - * Get the keyGenerator. - * - * @return Return the keyGenerator. - */ - public IndexKeyGenerator getKeyGenerator() { - return keyGenerator; - } - - /** - * Get the additionalColumns. - * - * @return Return the additionalColumns. - */ - public byte[][] getAdditionalColumns() { - return additionalColumns; - } - - /** - * Get the indexId. - * - * @return Return the indexId. - */ - public String getIndexId() { - return indexId; - } - - public byte[][] getAllColumns() { - return allColumns; - } - - public boolean containsColumn(byte[] column) { - for (byte[] col : allColumns) { - if (Bytes.equals(column, col)) { - return true; - } - } - return false; - } - - public byte[] getIndexedTableName(byte[] baseTableName) { - return Bytes.add(baseTableName, Bytes.toBytes("-" + indexId)); - } - - /** {@inheritDoc} */ - public void readFields(DataInput in) throws IOException { - indexId = in.readUTF(); - int numIndexedCols = in.readInt(); - indexedColumns = new byte[numIndexedCols][]; - for (int i = 0; i < numIndexedCols; i++) { - indexedColumns[i] = Bytes.readByteArray(in); - } - int numAdditionalCols = in.readInt(); - additionalColumns = new byte[numAdditionalCols][]; - for (int i = 0; i < numAdditionalCols; i++) { - additionalColumns[i] = Bytes.readByteArray(in); - } - makeAllColumns(); - HBaseConfiguration conf = new HBaseConfiguration(); - keyGenerator = (IndexKeyGenerator) ObjectWritable.readObject(in, conf); - } - - /** {@inheritDoc} */ - public void write(DataOutput out) throws IOException { - out.writeUTF(indexId); - out.writeInt(indexedColumns.length); - for (byte[] col : indexedColumns) { - Bytes.writeByteArray(out, col); - } - if (additionalColumns != null) { - out.writeInt(additionalColumns.length); - for (byte[] col : additionalColumns) { - Bytes.writeByteArray(out, col); - } - } else { - out.writeInt(0); - } - HBaseConfiguration conf = new HBaseConfiguration(); - ObjectWritable - .writeObject(out, keyGenerator, IndexKeyGenerator.class, conf); - } - - /** {@inheritDoc} */ - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("ID => "); - sb.append(indexId); - return sb.toString(); - } - - -} diff --git a/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexedTable.java b/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexedTable.java deleted file mode 100644 index 1cfa0ffb46c..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexedTable.java +++ /dev/null @@ -1,224 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.tableindexed; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.client.transactional.TransactionalTable; -import org.apache.hadoop.hbase.filter.RowFilterInterface; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.HbaseMapWritable; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.util.Bytes; - -/** HTable extended with indexed support. */ -public class IndexedTable extends TransactionalTable { - - // FIXME, these belong elsewhere - static final byte[] INDEX_COL_FAMILY_NAME = Bytes.toBytes("__INDEX__"); - static final byte[] INDEX_COL_FAMILY = Bytes.add( - INDEX_COL_FAMILY_NAME, new byte[] { HStoreKey.COLUMN_FAMILY_DELIMITER }); - public static final byte[] INDEX_BASE_ROW_COLUMN = Bytes.add( - INDEX_COL_FAMILY, Bytes.toBytes("ROW")); - - static final Log LOG = LogFactory.getLog(IndexedTable.class); - - private Map indexIdToTable = new HashMap(); - - public IndexedTable(final HBaseConfiguration conf, final byte[] tableName) - throws IOException { - super(conf, tableName); - - for (IndexSpecification spec : super.getTableDescriptor().getIndexes()) { - indexIdToTable.put(spec.getIndexId(), new HTable(conf, spec - .getIndexedTableName(tableName))); - } - } - - /** - * Open up an indexed scanner. Results will come back in the indexed order, - * but will contain RowResults from the original table. - * - * @param indexId the id of the index to use - * @param indexStartRow (created from the IndexKeyGenerator) - * @param indexColumns in the index table - * @param indexFilter filter to run on the index'ed table. This can only use - * columns that have been added to the index. - * @param baseColumns from the original table - * @return scanner - * @throws IOException - * @throws IndexNotFoundException - */ - public Scanner getIndexedScanner(String indexId, final byte[] indexStartRow, - byte[][] indexColumns, final RowFilterInterface indexFilter, - final byte[][] baseColumns) throws IOException, IndexNotFoundException { - IndexSpecification indexSpec = super.getTableDescriptor().getIndex(indexId); - if (indexSpec == null) { - throw new IndexNotFoundException("Index " + indexId - + " not defined in table " - + super.getTableDescriptor().getNameAsString()); - } - verifyIndexColumns(indexColumns, indexSpec); - // TODO, verify/remove index columns from baseColumns - - HTable indexTable = indexIdToTable.get(indexId); - - byte[][] allIndexColumns; - if (indexColumns != null) { - allIndexColumns = new byte[indexColumns.length + 1][]; - System - .arraycopy(indexColumns, 0, allIndexColumns, 0, indexColumns.length); - allIndexColumns[indexColumns.length] = INDEX_BASE_ROW_COLUMN; - } else { - byte[][] allColumns = indexSpec.getAllColumns(); - allIndexColumns = new byte[allColumns.length + 1][]; - System.arraycopy(allColumns, 0, allIndexColumns, 0, allColumns.length); - allIndexColumns[allColumns.length] = INDEX_BASE_ROW_COLUMN; - } - - Scanner indexScanner = indexTable.getScanner(allIndexColumns, - indexStartRow, indexFilter); - - return new ScannerWrapper(indexScanner, baseColumns); - } - - private void verifyIndexColumns(byte[][] requestedColumns, - IndexSpecification indexSpec) { - if (requestedColumns == null) { - return; - } - for (byte[] requestedColumn : requestedColumns) { - boolean found = false; - for (byte[] indexColumn : indexSpec.getAllColumns()) { - if (Bytes.equals(requestedColumn, indexColumn)) { - found = true; - break; - } - } - if (!found) { - throw new RuntimeException("Column [" + Bytes.toString(requestedColumn) - + "] not in index " + indexSpec.getIndexId()); - } - } - } - - private class ScannerWrapper implements Scanner { - - private Scanner indexScanner; - private byte[][] columns; - - public ScannerWrapper(Scanner indexScanner, byte[][] columns) { - this.indexScanner = indexScanner; - this.columns = columns; - } - - /** {@inheritDoc} */ - public RowResult next() throws IOException { - RowResult[] result = next(1); - if (result == null || result.length < 1) - return null; - return result[0]; - } - - /** {@inheritDoc} */ - public RowResult[] next(int nbRows) throws IOException { - RowResult[] indexResult = indexScanner.next(nbRows); - if (indexResult == null) { - return null; - } - RowResult[] result = new RowResult[indexResult.length]; - for (int i = 0; i < indexResult.length; i++) { - RowResult row = indexResult[i]; - byte[] baseRow = row.get(INDEX_BASE_ROW_COLUMN).getValue(); - LOG.debug("next index row [" + Bytes.toString(row.getRow()) - + "] -> base row [" + Bytes.toString(baseRow) + "]"); - HbaseMapWritable colValues = - new HbaseMapWritable(); - if (columns != null && columns.length > 0) { - LOG.debug("Going to base table for remaining columns"); - RowResult baseResult = IndexedTable.this.getRow(baseRow, columns); - - if (baseResult != null) { - colValues.putAll(baseResult); - } - } - for (Entry entry : row.entrySet()) { - byte[] col = entry.getKey(); - if (HStoreKey.matchingFamily(INDEX_COL_FAMILY_NAME, col)) { - continue; - } - colValues.put(col, entry.getValue()); - } - result[i] = new RowResult(baseRow, colValues); - } - return result; - } - - /** {@inheritDoc} */ - public void close() { - indexScanner.close(); - } - - /** {@inheritDoc} */ - public Iterator iterator() { - // FIXME, copied from HTable.ClientScanner. Extract this to common base - // class? - return new Iterator() { - RowResult next = null; - - public boolean hasNext() { - if (next == null) { - try { - next = ScannerWrapper.this.next(); - return next != null; - } catch (IOException e) { - throw new RuntimeException(e); - } - } - return true; - } - - public RowResult next() { - if (!hasNext()) { - return null; - } - RowResult temp = next; - next = null; - return temp; - } - - public void remove() { - throw new UnsupportedOperationException(); - } - }; - } - - } -} diff --git a/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexedTableAdmin.java b/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexedTableAdmin.java deleted file mode 100644 index 987622594e9..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/tableindexed/IndexedTableAdmin.java +++ /dev/null @@ -1,154 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.tableindexed; - -import java.io.IOException; -import java.util.Set; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.TreeSet; -import java.util.Map.Entry; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.ColumnNameParseException; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.TableExistsException; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.regionserver.tableindexed.IndexMaintenanceUtils; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Extension of HBaseAdmin that creates indexed tables. - * - */ -public class IndexedTableAdmin extends HBaseAdmin { - - private static final Log LOG = LogFactory.getLog(IndexedTableAdmin.class); - - /** - * Constructor - * - * @param conf Configuration object - * @throws MasterNotRunningException - */ - public IndexedTableAdmin(HBaseConfiguration conf) - throws MasterNotRunningException { - super(conf); - } - - /** - * Creates a new table - * - * @param desc table descriptor for table - * - * @throws IllegalArgumentException if the table name is reserved - * @throws MasterNotRunningException if master is not running - * @throws TableExistsException if table already exists (If concurrent - * threads, the table may have been created between test-for-existence and - * attempt-at-creation). - * @throws IOException - */ - @Override - public void createTable(HTableDescriptor desc) throws IOException { - super.createTable(desc); - this.createIndexTables(desc); - } - - private void createIndexTables(HTableDescriptor tableDesc) throws IOException { - byte[] baseTableName = tableDesc.getName(); - for (IndexSpecification indexSpec : tableDesc.getIndexes()) { - HTableDescriptor indexTableDesc = createIndexTableDesc(baseTableName, - indexSpec); - super.createTable(indexTableDesc); - } - } - - private HTableDescriptor createIndexTableDesc(byte[] baseTableName, - IndexSpecification indexSpec) throws ColumnNameParseException { - HTableDescriptor indexTableDesc = new HTableDescriptor(indexSpec - .getIndexedTableName(baseTableName)); - Set families = new TreeSet(Bytes.BYTES_COMPARATOR); - families.add(IndexedTable.INDEX_COL_FAMILY); - for (byte[] column : indexSpec.getAllColumns()) { - families.add(Bytes.add(HStoreKey.getFamily(column), - new byte[] { HStoreKey.COLUMN_FAMILY_DELIMITER })); - } - - for (byte[] colFamily : families) { - indexTableDesc.addFamily(new HColumnDescriptor(colFamily)); - } - - return indexTableDesc; - } - - /** Remove an index for a table. - * @throws IOException - * - */ - public void removeIndex(byte[] baseTableName, String indexId) throws IOException { - super.disableTable(baseTableName); - HTableDescriptor desc = super.getTableDescriptor(baseTableName); - IndexSpecification spec = desc.getIndex(indexId); - desc.removeIndex(indexId); - this.disableTable(spec.getIndexedTableName(baseTableName)); - this.deleteTable(spec.getIndexedTableName(baseTableName)); - super.modifyTable(baseTableName, desc); - super.enableTable(baseTableName); - } - - /** Add an index to a table. */ - public void addIndex(byte []baseTableName, IndexSpecification indexSpec) throws IOException { - LOG.warn("Adding index to existing table ["+Bytes.toString(baseTableName)+"], this may take a long time"); - // TODO, make table read-only - LOG.warn("Not putting table in readonly, if its being written to, the index may get out of sync"); - HTableDescriptor indexTableDesc = createIndexTableDesc(baseTableName, indexSpec); - super.createTable(indexTableDesc); - super.disableTable(baseTableName); - HTableDescriptor desc = super.getTableDescriptor(baseTableName); - desc.addIndex(indexSpec); - super.modifyTable(baseTableName, desc); - super.enableTable(baseTableName); - reIndexTable(baseTableName, indexSpec); - } - - private void reIndexTable(byte[] baseTableName, IndexSpecification indexSpec) throws IOException { - HTable baseTable = new HTable(baseTableName); - HTable indexTable = new HTable(indexSpec.getIndexedTableName(baseTableName)); - for (RowResult rowResult : baseTable.getScanner(indexSpec.getAllColumns())) { - SortedMap columnValues = new TreeMap(Bytes.BYTES_COMPARATOR); - for (Entry entry : rowResult.entrySet()) { - columnValues.put(entry.getKey(), entry.getValue().getValue()); - } - if (IndexMaintenanceUtils.doesApplyToIndex(indexSpec, columnValues)) { - BatchUpdate indexUpdate = IndexMaintenanceUtils.createIndexUpdate(indexSpec, rowResult.getRow(), columnValues); - indexTable.commit(indexUpdate); - } - } - } -} diff --git a/src/java/org/apache/hadoop/hbase/client/tableindexed/package.html b/src/java/org/apache/hadoop/hbase/client/tableindexed/package.html deleted file mode 100644 index 36214f7ed25..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/tableindexed/package.html +++ /dev/null @@ -1,46 +0,0 @@ - - - - - - - - -This package provides support for secondary indexing by maintaining a separate, "index", table for each index. - -The IndexSpecification class provides the metadata for the index. This includes: -
  • the columns that contribute to the index key, -
  • additional columns to put in the index table (and are thus made available to filters on the index table), -
    and -
  • an IndexKeyGenerator which constructs the index-row-key from the indexed column(s) and the original row. - -IndexesSpecifications can be added to a table's metadata (HTableDescriptor) before the table is constructed. -Afterwards, updates and deletes to the original table will trigger the updates in the index, and -the indexes can be scanned using the API on IndexedTable. - -For a simple example, look at the unit test in org.apache.hadoop.hbase.client.tableIndexed. - -

    To enable the indexing, modify hbase-site.xml to turn on the -IndexedRegionServer. This is done by setting -hbase.regionserver.class to -org.apache.hadoop.hbase.ipc.IndexedRegionInterface and -hbase.regionserver.impl to -org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegionServer - - - diff --git a/src/java/org/apache/hadoop/hbase/client/transactional/LocalTransactionLogger.java b/src/java/org/apache/hadoop/hbase/client/transactional/LocalTransactionLogger.java deleted file mode 100644 index 1738315b5c3..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/transactional/LocalTransactionLogger.java +++ /dev/null @@ -1,71 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.transactional; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Random; - -/** - * A local, in-memory implementation of the transaction logger. Does not provide a global view, so - * it can't be relighed on by - * - */ -public class LocalTransactionLogger implements TransactionLogger { - - private static LocalTransactionLogger instance; - - /** - * Creates singleton if it does not exist - * - * @return reference to singleton - */ - public synchronized static LocalTransactionLogger getInstance() { - if (instance == null) { - instance = new LocalTransactionLogger(); - } - return instance; - } - - private Random random = new Random(); - private Map transactionIdToStatusMap = Collections - .synchronizedMap(new HashMap()); - - private LocalTransactionLogger() { - // Enforce singlton - } - - /** @return random longs to minimize possibility of collision */ - public long createNewTransactionLog() { - long id = random.nextLong(); - transactionIdToStatusMap.put(id, TransactionStatus.PENDING); - return id; - } - - public TransactionStatus getStatusForTransaction(final long transactionId) { - return transactionIdToStatusMap.get(transactionId); - } - - public void setStatusForTransaction(final long transactionId, - final TransactionStatus status) { - transactionIdToStatusMap.put(transactionId, status); - } -} diff --git a/src/java/org/apache/hadoop/hbase/client/transactional/TransactionLogger.java b/src/java/org/apache/hadoop/hbase/client/transactional/TransactionLogger.java deleted file mode 100644 index 5ea321aec74..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/transactional/TransactionLogger.java +++ /dev/null @@ -1,59 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.transactional; - -/** - * Simple interface used to provide a log about transaction status. Written to - * by the client, and read by regionservers in case of failure. - * - */ -public interface TransactionLogger { - - /** Transaction status values */ - enum TransactionStatus { - /** Transaction is pending */ - PENDING, - /** Transaction was committed */ - COMMITTED, - /** Transaction was aborted */ - ABORTED - } - - /** - * Create a new transaction log. Return the transaction's globally unique id. - * Log's initial value should be PENDING - * - * @return transaction id - */ - long createNewTransactionLog(); - - /** - * @param transactionId - * @return transaction status - */ - TransactionStatus getStatusForTransaction(long transactionId); - - /** - * @param transactionId - * @param status - */ - void setStatusForTransaction(long transactionId, TransactionStatus status); - -} diff --git a/src/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java b/src/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java index 766e5064d7e..e69de29bb2d 100644 --- a/src/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java +++ b/src/java/org/apache/hadoop/hbase/client/transactional/TransactionManager.java @@ -1,152 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.transactional; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.HConnectionManager; -import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface; -import org.apache.hadoop.ipc.RemoteException; - -/** - * Transaction Manager. Responsible for committing transactions. - * - */ -public class TransactionManager { - static final Log LOG = LogFactory.getLog(TransactionManager.class); - - private final HConnection connection; - private final TransactionLogger transactionLogger; - - /** - * @param conf - */ - public TransactionManager(final HBaseConfiguration conf) { - this(LocalTransactionLogger.getInstance(), conf); - } - - /** - * @param transactionLogger - * @param conf - */ - public TransactionManager(final TransactionLogger transactionLogger, - final HBaseConfiguration conf) { - this.transactionLogger = transactionLogger; - connection = HConnectionManager.getConnection(conf); - } - - /** - * Called to start a transaction. - * - * @return new transaction state - */ - public TransactionState beginTransaction() { - long transactionId = transactionLogger.createNewTransactionLog(); - LOG.debug("Begining transaction " + transactionId); - return new TransactionState(transactionId); - } - - /** - * Try and commit a transaction. - * - * @param transactionState - * @throws IOException - * @throws CommitUnsuccessfulException - */ - public void tryCommit(final TransactionState transactionState) - throws CommitUnsuccessfulException, IOException { - LOG.debug("atempting to commit trasaction: " + transactionState.toString()); - - try { - for (HRegionLocation location : transactionState - .getParticipatingRegions()) { - TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection - .getHRegionConnection(location.getServerAddress()); - boolean canCommit = transactionalRegionServer.commitRequest(location - .getRegionInfo().getRegionName(), transactionState - .getTransactionId()); - if (LOG.isTraceEnabled()) { - LOG.trace("Region [" - + location.getRegionInfo().getRegionNameAsString() + "] votes " - + (canCommit ? "to commit" : "to abort") + " transaction " - + transactionState.getTransactionId()); - } - - if (!canCommit) { - LOG.debug("Aborting [" + transactionState.getTransactionId() + "]"); - abort(transactionState, location); - throw new CommitUnsuccessfulException(); - } - } - - LOG.debug("Commiting [" + transactionState.getTransactionId() + "]"); - - transactionLogger.setStatusForTransaction(transactionState - .getTransactionId(), TransactionLogger.TransactionStatus.COMMITTED); - - for (HRegionLocation location : transactionState - .getParticipatingRegions()) { - TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection - .getHRegionConnection(location.getServerAddress()); - transactionalRegionServer.commit(location.getRegionInfo() - .getRegionName(), transactionState.getTransactionId()); - } - } catch (RemoteException e) { - LOG.debug("Commit of transaction [" + transactionState.getTransactionId() - + "] was unsucsessful", e); - // FIXME, think about the what ifs - throw new CommitUnsuccessfulException(e); - } - // Tran log can be deleted now ... - } - - /** - * Abort a s transaction. - * - * @param transactionState - * @throws IOException - */ - public void abort(final TransactionState transactionState) throws IOException { - abort(transactionState, null); - } - - private void abort(final TransactionState transactionState, - final HRegionLocation locationToIgnore) throws IOException { - transactionLogger.setStatusForTransaction(transactionState - .getTransactionId(), TransactionLogger.TransactionStatus.ABORTED); - - for (HRegionLocation location : transactionState.getParticipatingRegions()) { - if (locationToIgnore != null && location.equals(locationToIgnore)) { - continue; - } - - TransactionalRegionInterface transactionalRegionServer = (TransactionalRegionInterface) connection - .getHRegionConnection(location.getServerAddress()); - - transactionalRegionServer.abort(location.getRegionInfo().getRegionName(), - transactionState.getTransactionId()); - } - } -} diff --git a/src/java/org/apache/hadoop/hbase/client/transactional/TransactionScannerCallable.java b/src/java/org/apache/hadoop/hbase/client/transactional/TransactionScannerCallable.java deleted file mode 100644 index 081068f3098..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/transactional/TransactionScannerCallable.java +++ /dev/null @@ -1,51 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.transactional; - -import java.io.IOException; - -import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.ScannerCallable; -import org.apache.hadoop.hbase.filter.RowFilterInterface; -import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface; - -class TransactionScannerCallable extends ScannerCallable { - - private TransactionState transactionState; - - TransactionScannerCallable(final TransactionState transactionState, - final HConnection connection, final byte[] tableName, - final byte[][] columns, final byte[] startRow, final long timestamp, - final RowFilterInterface filter) { - super(connection, tableName, columns, startRow, timestamp, filter); - this.transactionState = transactionState; - } - - @Override - protected long openScanner() throws IOException { - if (transactionState.addRegion(location)) { - ((TransactionalRegionInterface) server).beginTransaction(transactionState - .getTransactionId(), location.getRegionInfo().getRegionName()); - } - return ((TransactionalRegionInterface) server).openScanner(transactionState - .getTransactionId(), this.location.getRegionInfo().getRegionName(), - getColumns(), row, getTimestamp(), getFilter()); - } -} diff --git a/src/java/org/apache/hadoop/hbase/client/transactional/TransactionState.java b/src/java/org/apache/hadoop/hbase/client/transactional/TransactionState.java deleted file mode 100644 index 8c2f9805953..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/transactional/TransactionState.java +++ /dev/null @@ -1,78 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.transactional; - -import java.util.HashSet; -import java.util.Set; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HRegionLocation; - -/** - * Holds client-side transaction information. Client's use them as opaque - * objects passed around to transaction operations. - * - */ -public class TransactionState { - static final Log LOG = LogFactory.getLog(TransactionState.class); - - private final long transactionId; - - private Set participatingRegions = new HashSet(); - - TransactionState(final long transactionId) { - this.transactionId = transactionId; - } - - boolean addRegion(final HRegionLocation hregion) { - boolean added = participatingRegions.add(hregion); - - if (added) { - LOG.debug("Adding new hregion [" - + hregion.getRegionInfo().getRegionNameAsString() - + "] to transaction [" + transactionId + "]"); - } - - return added; - } - - Set getParticipatingRegions() { - return participatingRegions; - } - - /** - * Get the transactionId. - * - * @return Return the transactionId. - */ - public long getTransactionId() { - return transactionId; - } - - /** - * @see java.lang.Object#toString() - */ - @Override - public String toString() { - return "id: " + transactionId + ", particpants: " - + participatingRegions.size(); - } -} diff --git a/src/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java b/src/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java deleted file mode 100644 index fb5aae0aead..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java +++ /dev/null @@ -1,428 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.transactional; - -import java.io.IOException; - -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.client.ScannerCallable; -import org.apache.hadoop.hbase.client.ServerCallable; -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.RowResult; -import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface; - -/** - * Table with transactional support. - * - */ -public class TransactionalTable extends HTable { - - /** - * @param conf - * @param tableName - * @throws IOException - */ - public TransactionalTable(final HBaseConfiguration conf, - final String tableName) throws IOException { - super(conf, tableName); - } - - /** - * @param conf - * @param tableName - * @throws IOException - */ - public TransactionalTable(final HBaseConfiguration conf, - final byte[] tableName) throws IOException { - super(conf, tableName); - } - - private static abstract class TransactionalServerCallable extends - ServerCallable { - protected TransactionState transactionState; - - protected TransactionalRegionInterface getTransactionServer() { - return (TransactionalRegionInterface) server; - } - - protected void recordServer() throws IOException { - if (transactionState.addRegion(location)) { - getTransactionServer().beginTransaction( - transactionState.getTransactionId(), - location.getRegionInfo().getRegionName()); - } - } - - /** - * @param connection - * @param tableName - * @param row - * @param transactionState - */ - public TransactionalServerCallable(final HConnection connection, - final byte[] tableName, final byte[] row, - final TransactionState transactionState) { - super(connection, tableName, row); - this.transactionState = transactionState; - } - - } - - /** - * Get a single value for the specified row and column - * - * @param transactionState - * @param row row key - * @param column column name - * @return value for specified row/column - * @throws IOException - */ - public Cell get(final TransactionState transactionState, final byte[] row, - final byte[] column) throws IOException { - return super.getConnection().getRegionServerWithRetries( - new TransactionalServerCallable(super.getConnection(), super - .getTableName(), row, transactionState) { - public Cell call() throws IOException { - recordServer(); - return getTransactionServer().get( - transactionState.getTransactionId(), - location.getRegionInfo().getRegionName(), row, column); - } - }); - } - - /** - * Get the specified number of versions of the specified row and column - * - * @param transactionState - * @param row - row key - * @param column - column name - * @param numVersions - number of versions to retrieve - * @return - array byte values - * @throws IOException - */ - public Cell[] get(final TransactionState transactionState, final byte[] row, - final byte[] column, final int numVersions) throws IOException { - Cell[] values = null; - values = super.getConnection().getRegionServerWithRetries( - new TransactionalServerCallable(super.getConnection(), super - .getTableName(), row, transactionState) { - public Cell[] call() throws IOException { - recordServer(); - return getTransactionServer().get( - transactionState.getTransactionId(), - location.getRegionInfo().getRegionName(), row, column, - numVersions); - } - }); - - return values; - } - - /** - * Get the specified number of versions of the specified row and column with - * the specified timestamp. - * - * @param transactionState - * @param row - row key - * @param column - column name - * @param timestamp - timestamp - * @param numVersions - number of versions to retrieve - * @return - array of values that match the above criteria - * @throws IOException - */ - public Cell[] get(final TransactionState transactionState, final byte[] row, - final byte[] column, final long timestamp, final int numVersions) - throws IOException { - Cell[] values = null; - values = super.getConnection().getRegionServerWithRetries( - new TransactionalServerCallable(super.getConnection(), super - .getTableName(), row, transactionState) { - public Cell[] call() throws IOException { - recordServer(); - return getTransactionServer().get( - transactionState.getTransactionId(), - location.getRegionInfo().getRegionName(), row, column, - timestamp, numVersions); - } - }); - - return values; - } - - /** - * Get all the data for the specified row at the latest timestamp - * - * @param transactionState - * @param row row key - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final TransactionState transactionState, - final byte[] row) throws IOException { - return getRow(transactionState, row, HConstants.LATEST_TIMESTAMP); - } - - /** - * Get all the data for the specified row at a specified timestamp - * - * @param transactionState - * @param row row key - * @param ts timestamp - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final TransactionState transactionState, - final byte[] row, final long ts) throws IOException { - return super.getConnection().getRegionServerWithRetries( - new TransactionalServerCallable(super.getConnection(), super - .getTableName(), row, transactionState) { - public RowResult call() throws IOException { - recordServer(); - return getTransactionServer().getRow( - transactionState.getTransactionId(), - location.getRegionInfo().getRegionName(), row, ts); - } - }); - } - - /** - * Get selected columns for the specified row at the latest timestamp - * - * @param transactionState - * @param row row key - * @param columns Array of column names you want to retrieve. - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final TransactionState transactionState, - final byte[] row, final byte[][] columns) throws IOException { - return getRow(transactionState, row, columns, HConstants.LATEST_TIMESTAMP); - } - - /** - * Get selected columns for the specified row at a specified timestamp - * - * @param transactionState - * @param row row key - * @param columns Array of column names you want to retrieve. - * @param ts timestamp - * @return RowResult is empty if row does not exist. - * @throws IOException - */ - public RowResult getRow(final TransactionState transactionState, - final byte[] row, final byte[][] columns, final long ts) - throws IOException { - return super.getConnection().getRegionServerWithRetries( - new TransactionalServerCallable(super.getConnection(), super - .getTableName(), row, transactionState) { - public RowResult call() throws IOException { - recordServer(); - return getTransactionServer().getRow( - transactionState.getTransactionId(), - location.getRegionInfo().getRegionName(), row, columns, ts); - } - }); - } - - /** - * Delete all cells that match the passed row and whose timestamp is equal-to - * or older than the passed timestamp. - * - * @param transactionState - * @param row Row to update - * @param ts Delete all cells of the same timestamp or older. - * @throws IOException - */ - public void deleteAll(final TransactionState transactionState, - final byte[] row, final long ts) throws IOException { - super.getConnection().getRegionServerWithRetries( - new TransactionalServerCallable(super.getConnection(), super - .getTableName(), row, transactionState) { - public Boolean call() throws IOException { - recordServer(); - getTransactionServer().deleteAll( - transactionState.getTransactionId(), - location.getRegionInfo().getRegionName(), row, ts); - return null; - } - }); - } - - /** - * Get a scanner on the current table starting at first row. Return the - * specified columns. - * - * @param transactionState - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible to - * pass a regex in the column qualifier. A column qualifier is judged to be a - * regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final TransactionState transactionState, - final byte[][] columns) throws IOException { - return getScanner(transactionState, columns, HConstants.EMPTY_START_ROW, - HConstants.LATEST_TIMESTAMP, null); - } - - /** - * Get a scanner on the current table starting at the specified row. Return - * the specified columns. - * - * @param transactionState - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible to - * pass a regex in the column qualifier. A column qualifier is judged to be a - * regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final TransactionState transactionState, - final byte[][] columns, final byte[] startRow) throws IOException { - return getScanner(transactionState, columns, startRow, - HConstants.LATEST_TIMESTAMP, null); - } - - /** - * Get a scanner on the current table starting at the specified row. Return - * the specified columns. - * - * @param transactionState - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible to - * pass a regex in the column qualifier. A column qualifier is judged to be a - * regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param timestamp only return results whose timestamp <= this value - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final TransactionState transactionState, - final byte[][] columns, final byte[] startRow, final long timestamp) - throws IOException { - return getScanner(transactionState, columns, startRow, timestamp, null); - } - - /** - * Get a scanner on the current table starting at the specified row. Return - * the specified columns. - * - * @param transactionState - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible to - * pass a regex in the column qualifier. A column qualifier is judged to be a - * regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param filter a row filter using row-key regexp and/or column data filter. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final TransactionState transactionState, - final byte[][] columns, final byte[] startRow, - final RowFilterInterface filter) throws IOException { - return getScanner(transactionState, columns, startRow, - HConstants.LATEST_TIMESTAMP, filter); - } - - /** - * Get a scanner on the current table starting at the specified row. Return - * the specified columns. - * - * @param transactionState - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible to - * pass a regex in the column qualifier. A column qualifier is judged to be a - * regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row in table to scan - * @param timestamp only return results whose timestamp <= this value - * @param filter a row filter using row-key regexp and/or column data filter. - * @return scanner - * @throws IOException - */ - public Scanner getScanner(final TransactionState transactionState, - final byte[][] columns, final byte[] startRow, final long timestamp, - final RowFilterInterface filter) throws IOException { - ClientScanner scanner = new TransactionalClientScanner(transactionState, columns, startRow, - timestamp, filter); - scanner.initialize(); - return scanner; - } - - /** - * Commit a BatchUpdate to the table. - * - * @param transactionState - * @param batchUpdate - * @throws IOException - */ - public synchronized void commit(final TransactionState transactionState, - final BatchUpdate batchUpdate) throws IOException { - super.getConnection().getRegionServerWithRetries( - new TransactionalServerCallable(super.getConnection(), super - .getTableName(), batchUpdate.getRow(), transactionState) { - public Boolean call() throws IOException { - recordServer(); - getTransactionServer().batchUpdate( - transactionState.getTransactionId(), - location.getRegionInfo().getRegionName(), batchUpdate); - return null; - } - }); - } - - protected class TransactionalClientScanner extends HTable.ClientScanner { - - private TransactionState transactionState; - - protected TransactionalClientScanner( - final TransactionState transactionState, final byte[][] columns, - final byte[] startRow, final long timestamp, - final RowFilterInterface filter) { - super(columns, startRow, timestamp, filter); - this.transactionState = transactionState; - } - - @Override - protected ScannerCallable getScannerCallable( - final byte[] localStartKey, int caching) { - TransactionScannerCallable t = - new TransactionScannerCallable(transactionState, getConnection(), - getTableName(), getColumns(), localStartKey, getTimestamp(), - getFilter()); - t.setCaching(caching); - return t; - } - } - -} diff --git a/src/java/org/apache/hadoop/hbase/client/transactional/UnknownTransactionException.java b/src/java/org/apache/hadoop/hbase/client/transactional/UnknownTransactionException.java deleted file mode 100644 index 66f2bc50424..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/transactional/UnknownTransactionException.java +++ /dev/null @@ -1,43 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.transactional; - -import org.apache.hadoop.hbase.DoNotRetryIOException; - -/** - * Thrown if a region server is passed an unknown transaction id - */ -public class UnknownTransactionException extends DoNotRetryIOException { - - private static final long serialVersionUID = 698575374929591099L; - - /** constructor */ - public UnknownTransactionException() { - super(); - } - - /** - * Constructor - * @param s message - */ - public UnknownTransactionException(String s) { - super(s); - } -} diff --git a/src/java/org/apache/hadoop/hbase/client/transactional/package.html b/src/java/org/apache/hadoop/hbase/client/transactional/package.html deleted file mode 100644 index 357425c2f5d..00000000000 --- a/src/java/org/apache/hadoop/hbase/client/transactional/package.html +++ /dev/null @@ -1,61 +0,0 @@ - - - - - - - - -This package provides support for atomic transactions. Transactions can -span multiple regions. Transaction writes are applied when committing a -transaction. At commit time, the transaction is examined to see if it -can be applied while still maintaining atomicity. This is done by -looking for conflicts with the transactions that committed while the -current transaction was running. This technique is known as optimistic -concurrency control (OCC) because it relies on the assumption that -transactions will mostly not have conflicts with each other. - -

    -For more details on OCC, see the paper On Optimistic Methods for Concurrency Control -by Kung and Robinson available - here . - -

    To enable transactions, modify hbase-site.xml to turn on the -TransactionalRegionServer. This is done by setting -hbase.regionserver.class to -org.apache.hadoop.hbase.ipc.TransactionalRegionInterface and -hbase.regionserver.impl to -org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegionServer - -

    -The read set claimed by a transactional scanner is determined from the start and - end keys which the scanner is opened with. - - - -

    Known Issues

    - -Recovery in the face of hregion server failure -is not fully implemented. Thus, you cannot rely on the transactional -properties in the face of node failure. - - - - - - diff --git a/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java b/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java index ff9a2fc06d3..c8a9ae316e1 100644 --- a/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java @@ -33,12 +33,15 @@ import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.io.ObjectWritable; /** + * This filter is a no-op in HBase 0.20. Don't use it. + * * This filter is used to filter based on the value of a given column. It takes * an operator (equal, greater, not equal, etc) and either a byte [] value or a * byte [] comparator. If we have a byte [] value then we just do a * lexicographic compare. If this is not sufficient (eg you want to deserialize * a long and then compare it to a fixed long value), then you can pass in your * own comparator instead. + * @deprecated Use filters that are rooted on @{link Filter} instead */ public class ColumnValueFilter implements RowFilterInterface { /** Comparison operators. */ diff --git a/src/java/org/apache/hadoop/hbase/filter/Filter.java b/src/java/org/apache/hadoop/hbase/filter/Filter.java new file mode 100644 index 00000000000..d3a262b2720 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/filter/Filter.java @@ -0,0 +1,111 @@ +/* + * 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 org.apache.hadoop.io.Writable; +import org.apache.hadoop.hbase.KeyValue; + +/** + * Interface for row and column filters directly applied within the regionserver. + * A filter can expect the following call sequence: + *
      + *
    • {@link #reset()}
    • + *
    • {@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.
    • + *
    • {@link #filterRowKey(byte[],int,int)} -> true to drop this row, + * if false, we will also call
    • + *
    • {@link #filterKeyValue(KeyValue)} -> true to drop this key/value
    • + *
    • {@link #filterRow()} -> last chance to drop entire row based on the sequence of + * filterValue() calls. Eg: filter a row if it doesn't contain a specified column. + *
    • + *
    + * + * Filter instances are created one per region/scan. This interface replaces + * the old RowFilterInterface. + */ +public interface Filter extends Writable { + /** + * Reset the state of the filter between rows. + */ + public void reset(); + + /** + * Filters a row based on the row key. If this returns true, the entire + * row will be excluded. If false, each KeyValue in the row will be + * passed to {@link #filterKeyValue(KeyValue)} below. + * + * @param buffer buffer containing row key + * @param offset offset into buffer where row key starts + * @param length length of the row key + * @return true, remove entire row, false, include the row (maybe). + */ + public boolean filterRowKey(byte [] buffer, int offset, int length); + + /** + * If this returns true, the scan will terminate. + * + * @return true to end scan, false to continue. + */ + public boolean filterAllRemaining(); + + /** + * A way to filter based on the column family, column qualifier and/or the + * column value. Return code is described below. This allows filters to + * filter only certain number of columns, then terminate without matching ever + * column. + * + * If your filter returns ReturnCode.NEXT_ROW, it should return + * ReturnCode.NEXT_ROW until {@link #reset()} is called + * just in case the caller calls for the next row. + * + * @param v the KeyValue in question + * @return code as described below + * @see {@link Filter.ReturnCode} + */ + public ReturnCode filterKeyValue(KeyValue v); + + /** + * Return codes for filterValue(). + */ + public enum ReturnCode { + /** + * Include the KeyValue + */ + INCLUDE, + /** + * Skip this KeyValue + */ + SKIP, + /** + * Done with columns, skip to next row. Note that filterRow() will + * still be called. + */ + NEXT_ROW, + }; + + /** + * Last chance to veto row based on previous {@link #filterKeyValue(KeyValue)} + * calls. The filter needs to retain state then return a particular value for + * this call if they wish to exclude a row if a certain column is missing + * (for example). + * @return true to exclude row, false to include row. + */ + public boolean filterRow(); +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java index 1cb572e9ab3..4360b12db3e 100644 --- a/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java @@ -24,8 +24,10 @@ import org.apache.hadoop.hbase.util.Bytes; /** * Subclass of StopRowFilter that filters rows > the stop row, * making it include up to the last row but no further. + * + * @deprecated Use filters that are rooted on @{link Filter} instead */ -public class InclusiveStopRowFilter extends StopRowFilter{ +public class InclusiveStopRowFilter extends StopRowFilter { /** * Default constructor, filters nothing. Required though for RPC * deserialization. @@ -46,12 +48,17 @@ public class InclusiveStopRowFilter extends StopRowFilter{ */ @Override public boolean filterRowKey(final byte [] rowKey) { + return filterRowKey(rowKey, 0, rowKey.length); + } + + public boolean filterRowKey(byte []rowKey, int offset, int length) { if (rowKey == null) { if (getStopRowKey() == null) { return true; } return false; - } - return Bytes.compareTo(getStopRowKey(), rowKey) < 0; + } + return Bytes.compareTo(getStopRowKey(), 0, getStopRowKey().length, + rowKey, offset, length) < 0; } } diff --git a/src/java/org/apache/hadoop/hbase/filter/PageFilter.java b/src/java/org/apache/hadoop/hbase/filter/PageFilter.java new file mode 100644 index 00000000000..29b52e74785 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/filter/PageFilter.java @@ -0,0 +1,92 @@ +/** + * 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.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.hbase.KeyValue; + +/** + * Implementation of Filter interface that limits results to a specific page + * size. It terminates scanning once the number of filter-passed results is >= + * the given page size. + * + *

    + * Note that this filter cannot guarantee that the number of results returned + * to a client are <= page size. This is because the filter is applied + * separately on different region servers. It does however optimize the scan of + * individual HRegions by making sure that the page size is never exceeded + * locally. + *

    + */ +public class PageFilter implements Filter { + private long pageSize = Long.MAX_VALUE; + private int rowsAccepted = 0; + + /** + * Default constructor, filters nothing. Required though for RPC + * deserialization. + */ + public PageFilter() { + super(); + } + + /** + * Constructor that takes a maximum page size. + * + * @param pageSize Maximum result size. + */ + public PageFilter(final long pageSize) { + this.pageSize = pageSize; + } + + public void reset() { + rowsAccepted = 0; + } + + public boolean filterAllRemaining() { + return this.rowsAccepted >= this.pageSize; + } + + public boolean filterRowKey(byte[] rowKey, int offset, int length) { + return filterAllRemaining(); + } + + public void readFields(final DataInput in) throws IOException { + this.pageSize = in.readLong(); + } + + public void write(final DataOutput out) throws IOException { + out.writeLong(pageSize); + } + + @Override + public ReturnCode filterKeyValue(KeyValue v) { + this.rowsAccepted++; + return filterAllRemaining()? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE; + } + + @Override + public boolean filterRow() { + return filterAllRemaining(); + } +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java index a8e73d76d5b..5d9eca77b78 100644 --- a/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java @@ -40,6 +40,8 @@ import org.apache.hadoop.hbase.io.Cell; * individual HRegions by making sure that the page size is never exceeded * locally. *

    + * + * @deprecated Use filters that are rooted on @{link Filter} instead */ public class PageRowFilter implements RowFilterInterface { diff --git a/src/java/org/apache/hadoop/hbase/filter/PrefixRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/PrefixRowFilter.java index a4e3ece3b70..b7ef4158277 100644 --- a/src/java/org/apache/hadoop/hbase/filter/PrefixRowFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/PrefixRowFilter.java @@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.util.Bytes; /** * RowFilterInterface that filters everything that does not match a prefix + * + * @deprecated Use filters that are rooted on @{link Filter} instead */ public class PrefixRowFilter implements RowFilterInterface { protected byte[] prefix; diff --git a/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java b/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java index 0db5f455c60..76763d58a5b 100644 --- a/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java +++ b/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java @@ -30,7 +30,18 @@ import org.apache.hadoop.io.Writable; * * Interface used for row-level filters applied to HRegion.HScanner scan * results during calls to next(). - * TODO: Make Filters use proper comparator comparing rows. + * + * In HBase 0.20, not all of the functions will be called, thus filters which depend + * on them will not work as advertised! + * + * Specifically, you can only count on the following methods to be called: + * boolean filterRowKey(final byte [] rowKey, final int offset, final int length); + * boolean filterAllRemaining(); + * + * Complex filters that depend in more need to be rewritten to work with @{link Filter} + * + * Write new filters to use the @{link Filter} API instead. + * @deprecated Use filters that are rooted on @{link Filter} instead */ public interface RowFilterInterface extends Writable { /** diff --git a/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java b/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java index 6816845c6df..c9872ef11ff 100644 --- a/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java +++ b/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java @@ -38,6 +38,10 @@ import org.apache.hadoop.io.ObjectWritable; * which will be evaluated with a specified boolean operator MUST_PASS_ALL * (!AND) or MUST_PASS_ONE (!OR). Since you can use RowFilterSets as children * of RowFilterSet, you can create a hierarchy of filters to be evaluated. + * + * It is highly likely this construct will no longer work! + * + * @deprecated Use filters that are rooted on @{link Filter} instead */ public class RowFilterSet implements RowFilterInterface { diff --git a/src/java/org/apache/hadoop/hbase/filter/RowInclusiveStopFilter.java b/src/java/org/apache/hadoop/hbase/filter/RowInclusiveStopFilter.java new file mode 100644 index 00000000000..60bab74704d --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/filter/RowInclusiveStopFilter.java @@ -0,0 +1,89 @@ +/* + * 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 org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.DataOutput; +import java.io.IOException; +import java.io.DataInput; + +/** + * A Filter that stops after the given row. There is no "RowStopFilter" because the Scan + * spec allows you to specify a stop row. + * + * Use this filter to include the stop row, eg: [A,Z]. + */ +public class RowInclusiveStopFilter implements Filter { + private byte [] stopRowKey; + + public RowInclusiveStopFilter() { + super(); + } + + public RowInclusiveStopFilter(final byte [] stopRowKey) { + this.stopRowKey = stopRowKey; + } + + @Override + public void reset() { + // noop, no state + } + + @Override + public boolean filterRowKey(byte[] buffer, int offset, int length) { + if (buffer == null) { + if (this.stopRowKey == null) { + return true; //filter... + } + return false; + } + // if stopRowKey is <= buffer, then true, filter row. + return Bytes.compareTo(stopRowKey, 0, stopRowKey.length, buffer, offset, length) < 0; + } + + @Override + public boolean filterAllRemaining() { + return false; + } + + @Override + public ReturnCode filterKeyValue(KeyValue v) { + // include everything. + return ReturnCode.INCLUDE; + } + + @Override + public boolean filterRow() { + return false; + } + + @Override + public void write(DataOutput out) throws IOException { + Bytes.writeByteArray(out, this.stopRowKey); + } + + @Override + public void readFields(DataInput in) throws IOException { + this.stopRowKey = Bytes.readByteArray(in); + } +} diff --git a/src/java/org/apache/hadoop/hbase/client/tableindexed/SimpleIndexKeyGenerator.java b/src/java/org/apache/hadoop/hbase/filter/RowPrefixFilter.java similarity index 50% rename from src/java/org/apache/hadoop/hbase/client/tableindexed/SimpleIndexKeyGenerator.java rename to src/java/org/apache/hadoop/hbase/filter/RowPrefixFilter.java index 49694177a70..4947b7fe109 100644 --- a/src/java/org/apache/hadoop/hbase/client/tableindexed/SimpleIndexKeyGenerator.java +++ b/src/java/org/apache/hadoop/hbase/filter/RowPrefixFilter.java @@ -1,5 +1,5 @@ -/** - * Copyright 2008 The Apache Software Foundation +/* + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -17,43 +17,64 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.client.tableindexed; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.Map; +package org.apache.hadoop.hbase.filter; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.util.Bytes; -/** Creates indexed keys for a single column.... - * - */ -public class SimpleIndexKeyGenerator implements IndexKeyGenerator { +import java.io.DataOutput; +import java.io.IOException; +import java.io.DataInput; - private byte [] column; - - public SimpleIndexKeyGenerator(byte [] column) { - this.column = column; - } - - public SimpleIndexKeyGenerator() { - // For Writable - } - - /** {@inheritDoc} */ - public byte[] createIndexKey(byte[] rowKey, Map columns) { - return Bytes.add(columns.get(column), rowKey); +public class RowPrefixFilter implements Filter { + + protected byte [] prefix; + + public RowPrefixFilter(final byte [] prefix) { + this.prefix = prefix; } - /** {@inheritDoc} */ - public void readFields(DataInput in) throws IOException { - column = Bytes.readByteArray(in); + public RowPrefixFilter() { } - /** {@inheritDoc} */ + @Override + public void reset() { + } + + @Override + public boolean filterRowKey(byte[] buffer, int offset, int length) { + if (buffer == null) + return true; + if (length < prefix.length) + return true; + // if they are equal, return false => pass row + // else return true, filter row + return Bytes.compareTo(buffer, offset, prefix.length, prefix, 0, prefix.length) != 0; + } + + @Override + public boolean filterAllRemaining() { + return false; + } + + @Override + public ReturnCode filterKeyValue(KeyValue v) { + return ReturnCode.INCLUDE; + } + + @Override + public boolean filterRow() { + return false; + } + + @Override public void write(DataOutput out) throws IOException { - Bytes.writeByteArray(out, column); + Bytes.writeByteArray(out, prefix); } + @Override + public void readFields(DataInput in) throws IOException { + prefix = Bytes.readByteArray(in); + } } diff --git a/src/java/org/apache/hadoop/hbase/filter/RowWhileMatchFilter.java b/src/java/org/apache/hadoop/hbase/filter/RowWhileMatchFilter.java new file mode 100644 index 00000000000..749e93e7876 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/filter/RowWhileMatchFilter.java @@ -0,0 +1,96 @@ +/* + * 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 org.apache.hadoop.hbase.KeyValue; + +import java.io.DataOutput; +import java.io.IOException; +import java.io.DataInput; + +/** + * A wrapper filter that filters everything after the first filtered row. + */ +public class RowWhileMatchFilter implements Filter { + private boolean filterAllRemaining = false; + private Filter filter; + + public RowWhileMatchFilter() { + super(); + } + + public RowWhileMatchFilter(Filter filter) { + this.filter = filter; + } + + @Override + public void reset() { + // no state. + } + + private void changeFAR(boolean value) { + filterAllRemaining = filterAllRemaining || value; + } + + @Override + public boolean filterRowKey(byte[] buffer, int offset, int length) { + changeFAR(filter.filterRowKey(buffer, offset, length)); + return filterAllRemaining(); + } + + @Override + public boolean filterAllRemaining() { + return this.filterAllRemaining || this.filter.filterAllRemaining(); + } + + @Override + public ReturnCode filterKeyValue(KeyValue v) { + ReturnCode c = filter.filterKeyValue(v); + changeFAR(c != ReturnCode.INCLUDE); + return c; + } + + @Override + public boolean filterRow() { + return false; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeUTF(this.filter.getClass().getName()); + this.filter.write(out); + } + + @Override + public void readFields(DataInput in) throws IOException { + String className = in.readUTF(); + try { + this.filter = (Filter)(Class.forName(className).newInstance()); + this.filter.readFields(in); + } catch (InstantiationException e) { + throw new RuntimeException("Failed deserialize.", e); + } catch (IllegalAccessException e) { + throw new RuntimeException("Failed deserialize.", e); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Failed deserialize.", e); + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java index 5747178959d..38884a3e715 100644 --- a/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java @@ -32,6 +32,8 @@ import org.apache.hadoop.hbase.util.Bytes; /** * Implementation of RowFilterInterface that filters out rows greater than or * equal to a specified rowKey. + * + * @deprecated Use filters that are rooted on @{link Filter} instead */ public class StopRowFilter implements RowFilterInterface { private byte [] stopRowKey; diff --git a/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java index 81b18f8f651..9f0f937dce0 100644 --- a/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java @@ -34,6 +34,8 @@ import org.apache.hadoop.hbase.io.Cell; * filter(..) methods or filterNotNull(SortedMap), this wrapper's * filterAllRemaining() will return true. All filtering methods will * thereafter defer to the result of filterAllRemaining(). + * + * @deprecated Use filters that are rooted on @{link Filter} instead */ public class WhileMatchRowFilter implements RowFilterInterface { private boolean filterAllRemaining = false; diff --git a/src/java/org/apache/hadoop/hbase/filter/package-info.java b/src/java/org/apache/hadoop/hbase/filter/package-info.java index 81dc032f9e0..1ff9c2f37b2 100644 --- a/src/java/org/apache/hadoop/hbase/filter/package-info.java +++ b/src/java/org/apache/hadoop/hbase/filter/package-info.java @@ -17,7 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -/**Provides row-level filters applied to HRegion scan results during calls to {@link org.apache.hadoop.hbase.client.Scanner#next()}. +/**Provides row-level filters applied to HRegion scan results during calls to {@link org.apache.hadoop.hbase.client.ResultScanner#next()}.

    Use {@link org.apache.hadoop.hbase.filter.StopRowFilter} to stop the scan once rows exceed the supplied row key. Filters will not stop the scan unless hosted inside of a {@link org.apache.hadoop.hbase.filter.WhileMatchRowFilter}. diff --git a/src/java/org/apache/hadoop/hbase/io/BatchOperation.java b/src/java/org/apache/hadoop/hbase/io/BatchOperation.java index 66c07aa7f12..e52b0f69bd0 100644 --- a/src/java/org/apache/hadoop/hbase/io/BatchOperation.java +++ b/src/java/org/apache/hadoop/hbase/io/BatchOperation.java @@ -33,7 +33,8 @@ import org.apache.hadoop.io.Writable; * This object is purposely bare-bones because many instances are created * during bulk uploads. We have one class for DELETEs and PUTs rather than * a class per type because it makes the serialization easier. - * @see BatchUpdate + * @see BatchUpdate + * @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API. */ public class BatchOperation implements Writable, HeapSize { /** diff --git a/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java b/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java index 3c7ad0eeb9d..65d5c0eb22b 100644 --- a/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java +++ b/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java @@ -42,6 +42,7 @@ import org.apache.hadoop.io.WritableComparable; * There is one BatchUpdate object per server, so a series of batch operations * can result in multiple BatchUpdate objects if the batch contains rows that * are served by multiple region servers. + * @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API. */ public class BatchUpdate implements WritableComparable, Iterable, HeapSize { diff --git a/src/java/org/apache/hadoop/hbase/io/Cell.java b/src/java/org/apache/hadoop/hbase/io/Cell.java index e3daccb5bed..1f0813b673a 100644 --- a/src/java/org/apache/hadoop/hbase/io/Cell.java +++ b/src/java/org/apache/hadoop/hbase/io/Cell.java @@ -46,6 +46,7 @@ import agilejson.TOJSON; * stored with together as a result for get and getRow methods. This promotes * the timestamp of a cell to a first-class value, making it easy to take note * of temporal data. Cell is used all the way from HStore up to HTable. + * @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API. */ public class Cell implements Writable, Iterable>, ISerializable { @@ -228,7 +229,8 @@ public class Cell implements Writable, Iterable>, * TODO: This is the glue between old way of doing things and the new. * Herein we are converting our clean KeyValues to Map of Cells. */ - public static HbaseMapWritable createCells(final List results) { + public static HbaseMapWritable createCells( + final List results) { HbaseMapWritable cells = new HbaseMapWritable(); // Walking backward through the list of results though it has no effect @@ -275,4 +277,4 @@ public class Cell implements Writable, Iterable>, throws HBaseRestException { serializer.serializeCell(this); } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/io/CodeToClassAndBack.java b/src/java/org/apache/hadoop/hbase/io/CodeToClassAndBack.java index a9997a54edc..d7495e9f302 100644 --- a/src/java/org/apache/hadoop/hbase/io/CodeToClassAndBack.java +++ b/src/java/org/apache/hadoop/hbase/io/CodeToClassAndBack.java @@ -45,7 +45,7 @@ public interface CodeToClassAndBack { /** * Class list for supported classes */ - public Class[] classList = {byte[].class, Cell.class}; + public Class[] classList = {byte[].class, Cell.class}; /** * The static loader that is used instead of the static constructor in @@ -58,8 +58,8 @@ public interface CodeToClassAndBack { * Class that loads the static maps with their values. */ public class InternalStaticLoader{ - InternalStaticLoader(Class[] classList, Map> CODE_TO_CLASS, - Map, Byte> CLASS_TO_CODE){ + InternalStaticLoader(Class[] classList, + Map> CODE_TO_CLASS, Map, Byte> CLASS_TO_CODE){ byte code = 1; for(int i=0; i declaredClass; @@ -228,6 +233,12 @@ public class HbaseObjectWritable implements Writable, Configurable { Byte code = CLASS_TO_CODE.get(c); if (code == null) { LOG.error("Unsupported type " + c); + StackTraceElement[] els = new Exception().getStackTrace(); + for(StackTraceElement elem : els) { + LOG.error(elem.getMethodName()); + } +// new Exception().getStackTrace()[0].getMethodName()); +// throw new IOException(new Exception().getStackTrace()[0].getMethodName()); throw new UnsupportedOperationException("No code for unexpected " + c); } out.writeByte(code); @@ -261,6 +272,8 @@ public class HbaseObjectWritable implements Writable, Configurable { // byte-at-a-time we were previously doing. if (declClass.equals(byte [].class)) { Bytes.writeByteArray(out, (byte [])instanceObj); + } else if(declClass.equals(Result [].class)) { + Result.writeArray(out, (Result [])instanceObj); } else { int length = Array.getLength(instanceObj); out.writeInt(length); @@ -363,6 +376,8 @@ public class HbaseObjectWritable implements Writable, Configurable { } else if (declaredClass.isArray()) { // array if (declaredClass.equals(byte [].class)) { instance = Bytes.readByteArray(in); + } else if(declaredClass.equals(Result [].class)) { + instance = Result.readArray(in); } else { int length = in.readInt(); instance = Array.newInstance(declaredClass.getComponentType(), length); diff --git a/src/java/org/apache/hadoop/hbase/io/HeapSize.java b/src/java/org/apache/hadoop/hbase/io/HeapSize.java index e6f59e54bd2..91339076dfb 100644 --- a/src/java/org/apache/hadoop/hbase/io/HeapSize.java +++ b/src/java/org/apache/hadoop/hbase/io/HeapSize.java @@ -21,11 +21,24 @@ package org.apache.hadoop.hbase.io; /** * Implementations can be asked for an estimate of their size in bytes. + *

    * Useful for sizing caches. Its a given that implementation approximations - * probably do not account for 32 vs 64 bit nor for different VM implemenations. + * do not account for 32 vs 64 bit nor for different VM implementations. + *

    + * An Object's size is determined by the non-static data members in it, + * as well as the fixed {@link OBJECT} overhead. + *

    + * For example: + *

    + * public class SampleObject implements HeapSize {
    + *   
    + *   int [] numbers;
    + *   int x;
    + * }
    + * 
    */ public interface HeapSize { - + /** Reference size is 8 bytes on 64-bit, 4 bytes on 32-bit */ static final int REFERENCE = 8; @@ -49,10 +62,12 @@ public interface HeapSize { static final int LONG = 8; /** Array overhead */ - static final int BYTE_ARRAY = REFERENCE; static final int ARRAY = 3 * REFERENCE; static final int MULTI_ARRAY = (4 * REFERENCE) + ARRAY; + /** Byte arrays are fixed size below plus its length, 8 byte aligned */ + static final int BYTE_ARRAY = 3 * REFERENCE; + static final int BLOCK_SIZE_TAX = 8; static final int BYTE_BUFFER = 56; diff --git a/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java b/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java index 36235ee7535..1dbc2315347 100644 --- a/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java +++ b/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java @@ -122,10 +122,22 @@ implements WritableComparable { return this.length; } + /** + * @return the current length of the buffer. same as getSize() + */ + //Should probably deprecate getSize() so that we keep the same calls for all + //byte [] public int getLength() { - return getSize(); + if (this.bytes == null) { + throw new IllegalStateException("Uninitialiized. Null constructor " + + "called w/o accompaying readFields invocation"); + } + return this.length; } - + + /** + * @return offset + */ public int getOffset(){ return this.offset; } diff --git a/src/java/org/apache/hadoop/hbase/io/Reference.java b/src/java/org/apache/hadoop/hbase/io/Reference.java index d7d32cc79dd..ba9dcc98eba 100644 --- a/src/java/org/apache/hadoop/hbase/io/Reference.java +++ b/src/java/org/apache/hadoop/hbase/io/Reference.java @@ -65,10 +65,17 @@ public class Reference implements Writable { this(null, Range.bottom); } + /** + * + * @return Range + */ public Range getFileRegion() { return this.region; } + /** + * @return splitKey + */ public byte [] getSplitKey() { return splitkey; } diff --git a/src/java/org/apache/hadoop/hbase/io/RowResult.java b/src/java/org/apache/hadoop/hbase/io/RowResult.java index e61bd08b5b1..7be71a00851 100644 --- a/src/java/org/apache/hadoop/hbase/io/RowResult.java +++ b/src/java/org/apache/hadoop/hbase/io/RowResult.java @@ -32,7 +32,6 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeSet; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.rest.descriptors.RestCell; import org.apache.hadoop.hbase.rest.exception.HBaseRestException; @@ -46,11 +45,13 @@ import agilejson.TOJSON; /** * Holds row name and then a map of columns to cells. + * @deprecated As of hbase 0.20.0, replaced by new Get/Put/Delete/Result-based API. */ public class RowResult implements Writable, SortedMap, Comparable, ISerializable { private byte [] row = null; private final HbaseMapWritable cells; + private final byte [] COL_REGIONINFO = Bytes.toBytes("info:regioninfo"); /** default constructor for writable */ public RowResult() { @@ -102,6 +103,11 @@ public class RowResult implements Writable, SortedMap, return cells.containsKey(key); } + /** + * Check if the key can be found in this RowResult + * @param key + * @return true if key id found, false if not + */ public boolean containsKey(String key) { return cells.containsKey(Bytes.toBytes(key)); } @@ -175,6 +181,16 @@ public class RowResult implements Writable, SortedMap, public Cell get(String key) { return get(Bytes.toBytes(key)); } + + /** + * Get a cell using seperate family, columnQualifier arguments. + * @param family + * @param columnQualifier + * @return + */ + public Cell get(byte [] family, byte [] columnQualifier) { + return get(Bytes.add(family, KeyValue.COLUMN_FAMILY_DELIM_ARRAY, columnQualifier)); + } public Comparator comparator() { @@ -245,7 +261,7 @@ public class RowResult implements Writable, SortedMap, sb.append(Long.toString(e.getValue().getTimestamp())); sb.append(", value="); byte [] v = e.getValue().getValue(); - if (Bytes.equals(e.getKey(), HConstants.COL_REGIONINFO)) { + if (Bytes.equals(e.getKey(), this.COL_REGIONINFO)) { try { sb.append(Writables.getHRegionInfo(v).toString()); } catch (IOException ioe) { diff --git a/src/java/org/apache/hadoop/hbase/io/TimeRange.java b/src/java/org/apache/hadoop/hbase/io/TimeRange.java new file mode 100644 index 00000000000..404ddf92545 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/io/TimeRange.java @@ -0,0 +1,172 @@ +/* + * 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.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.io.Writable; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Represents an interval of version timestamps. + *

    + * Evaluated according to minStamp <= timestamp < maxStamp + * or [minStamp,maxStamp) in interval notation. + *

    + * Only used internally; should not be accessed directly by clients. + */ +public class TimeRange implements Writable { + private long minStamp = 0L; + private long maxStamp = Long.MAX_VALUE; + private boolean allTime = false; + + /** + * Default constructor. + * Represents interval [0, Long.MAX_VALUE) (allTime) + */ + public TimeRange() { + allTime = true; + } + + /** + * Represents interval [minStamp, Long.MAX_VALUE) + * @param minStamp the minimum timestamp value, inclusive + */ + public TimeRange(long minStamp) { + this.minStamp = minStamp; + } + + /** + * Represents interval [minStamp, Long.MAX_VALUE) + * @param minStamp the minimum timestamp value, inclusive + */ + public TimeRange(byte [] minStamp) { + this.minStamp = Bytes.toLong(minStamp); + } + + /** + * Represents interval [minStamp, maxStamp) + * @param minStamp the minimum timestamp, inclusive + * @param maxStamp the maximum timestamp, exclusive + * @throws IOException + */ + public TimeRange(long minStamp, long maxStamp) + throws IOException { + if(maxStamp < minStamp) { + throw new IOException("maxStamp is smaller than minStamp"); + } + this.minStamp = minStamp; + this.maxStamp = maxStamp; + } + + /** + * Represents interval [minStamp, maxStamp) + * @param minStamp the minimum timestamp, inclusive + * @param maxStamp the maximum timestamp, exclusive + * @throws IOException + */ + public TimeRange(byte [] minStamp, byte [] maxStamp) + throws IOException { + this(Bytes.toLong(minStamp), Bytes.toLong(maxStamp)); + } + + /** + * @return the smallest timestamp that should be considered + */ + public long getMin() { + return minStamp; + } + + /** + * @return the biggest timestamp that should be considered + */ + public long getMax() { + return maxStamp; + } + + /** + * Check if the specified timestamp is within this TimeRange. + *

    + * Returns true if within interval [minStamp, maxStamp), false + * if not. + * @param bytes timestamp to check + * @param offset offset into the bytes + * @return true if within TimeRange, false if not + */ + public boolean withinTimeRange(byte [] bytes, int offset) { + if(allTime) return true; + return withinTimeRange(Bytes.toLong(bytes, offset)); + } + + /** + * Check if the specified timestamp is within this TimeRange. + *

    + * Returns true if within interval [minStamp, maxStamp), false + * if not. + * @param timestamp timestamp to check + * @return true if within TimeRange, false if not + */ + public boolean withinTimeRange(long timestamp) { + if(allTime) return true; + // check if >= minStamp + return (minStamp <= timestamp && timestamp < maxStamp); + } + + /** + * Check if the specified timestamp is within this TimeRange. + *

    + * Returns true if within interval [minStamp, maxStamp), false + * if not. + * @param timestamp timestamp to check + * @return true if within TimeRange, false if not + */ + public boolean withinOrAfterTimeRange(long timestamp) { + if(allTime) return true; + // check if >= minStamp + return (timestamp >= minStamp); + } + + @Override + public String toString() { + StringBuffer sb = new StringBuffer(); + sb.append("maxStamp="); + sb.append(this.maxStamp); + sb.append(", minStamp="); + sb.append(this.minStamp); + return sb.toString(); + } + + //Writable + public void readFields(final DataInput in) throws IOException { + this.minStamp = in.readLong(); + this.maxStamp = in.readLong(); + this.allTime = in.readBoolean(); + } + + public void write(final DataOutput out) throws IOException { + out.writeLong(minStamp); + out.writeLong(maxStamp); + out.writeBoolean(this.allTime); + } +} diff --git a/src/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/src/java/org/apache/hadoop/hbase/io/hfile/HFile.java index 58f6cf9b6ee..0d8012131c3 100644 --- a/src/java/org/apache/hadoop/hbase/io/hfile/HFile.java +++ b/src/java/org/apache/hadoop/hbase/io/hfile/HFile.java @@ -104,11 +104,6 @@ import org.apache.hadoop.io.compress.Decompressor; *

    <fileinfo><trailer>
    . That is, there are not data nor meta * blocks present. *

    - * TODO: Bloomfilters. Need to add hadoop 0.20. first since it has bug fixes - * on the hadoop bf package. - * * TODO: USE memcmp by default? Write the keys out in an order that allows - * my using this -- reverse the timestamp. - * TODO: Add support for fast-gzip and for lzo. * TODO: Do scanners need to be able to take a start and end row? * TODO: Should BlockIndex know the name of its file? Should it have a Path * that points at its file say for the case where an index lives apart from @@ -465,8 +460,12 @@ public class HFile { * Add key/value to file. * Keys must be added in an order that agrees with the Comparator passed * on construction. - * @param key Key to add. Cannot be empty nor null. - * @param value Value to add. Cannot be empty nor null. + * @param key + * @param koffset + * @param klength + * @param value + * @param voffset + * @param vlength * @throws IOException */ public void append(final byte [] key, final int koffset, final int klength, @@ -1039,6 +1038,9 @@ public class HFile { } public KeyValue getKeyValue() { + if(this.block == null) { + return null; + } return new KeyValue(this.block.array(), this.block.arrayOffset() + this.block.position() - 8); } diff --git a/src/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java b/src/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java index 7f934e19e74..cdc0cb88ee9 100644 --- a/src/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java +++ b/src/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java @@ -24,6 +24,9 @@ public class SimpleBlockCache implements BlockCache { private ReferenceQueue q = new ReferenceQueue(); public int dumps = 0; + /** + * Constructor + */ public SimpleBlockCache() { super(); } @@ -36,6 +39,9 @@ public class SimpleBlockCache implements BlockCache { } } + /** + * @return the size + */ public synchronized int size() { processQueue(); return cache.size(); diff --git a/src/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java b/src/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java index 8b165559141..0819a8eea38 100644 --- a/src/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java +++ b/src/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java @@ -390,6 +390,7 @@ public class HBaseRPC { * @param addr * @param conf * @param maxAttempts + * @param timeout * @return proxy * @throws IOException */ diff --git a/src/java/org/apache/hadoop/hbase/ipc/HBaseServer.java b/src/java/org/apache/hadoop/hbase/ipc/HBaseServer.java index 01d20c1cc98..e884f81b5b9 100644 --- a/src/java/org/apache/hadoop/hbase/ipc/HBaseServer.java +++ b/src/java/org/apache/hadoop/hbase/ipc/HBaseServer.java @@ -86,7 +86,8 @@ public abstract class HBaseServer { public static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.HBaseServer"); - protected static final ThreadLocal SERVER = new ThreadLocal(); + protected static final ThreadLocal SERVER = + new ThreadLocal(); /** Returns the server instance called under or null. May be called under * {@link #call(Writable, long)} implementations, and under {@link Writable} @@ -128,10 +129,11 @@ public abstract class HBaseServer { private int handlerCount; // number of handler threads protected Class paramClass; // class of call parameters protected int maxIdleTime; // the maximum idle time after - // which a client may be disconnected - protected int thresholdIdleConnections; // the number of idle connections - // after which we will start - // cleaning up idle + // which a client may be + // disconnected + protected int thresholdIdleConnections; // the number of idle + // connections after which we + // will start cleaning up idle // connections int maxConnectionsToNuke; // the max number of // connections to nuke @@ -173,8 +175,9 @@ public abstract class HBaseServer { try { socket.bind(address, backlog); } catch (BindException e) { - BindException bindException = new BindException("Problem binding to " + address - + " : " + e.getMessage()); + BindException bindException = + new BindException("Problem binding to " + address + " : " + + e.getMessage()); bindException.initCause(e); throw bindException; } catch (SocketException e) { @@ -297,7 +300,6 @@ public abstract class HBaseServer { public void run() { LOG.info(getName() + ": starting"); SERVER.set(HBaseServer.this); - long lastPurgeTime = 0; // last check for old calls. while (running) { SelectionKey key = null; diff --git a/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java b/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java index ba32a843766..a514d6d9dcf 100644 --- a/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java +++ b/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java @@ -23,6 +23,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.io.Writable; @@ -109,7 +110,7 @@ public interface HMasterInterface extends HBaseRPCProtocolVersion { * @param args * @throws IOException */ - public void modifyTable(byte[] tableName, int op, Writable[] args) + public void modifyTable(byte[] tableName, HConstants.Modify op, Writable[] args) throws IOException; /** diff --git a/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java b/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java index 66885cfd267..5de6791dba0 100644 --- a/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java +++ b/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java @@ -1,5 +1,5 @@ /** - * Copyright 2007 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -21,15 +21,15 @@ package org.apache.hadoop.hbase.ipc; import java.io.IOException; -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.RowResult; -import org.apache.hadoop.hbase.io.HbaseMapWritable; - import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.regionserver.HRegion; /** * Clients interact with HRegionServers using a handle to the HRegionInterface. @@ -49,21 +49,6 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion { public HRegionInfo getRegionInfo(final byte [] regionName) throws NotServingRegionException; - /** - * Get the specified number of versions of the specified row and column with - * the specified timestamp. - * - * @param regionName region name - * @param row row key - * @param column column key - * @param timestamp timestamp - * @param numVersions number of versions to return - * @return array of values - * @throws IOException - */ - public Cell[] get(final byte [] regionName, final byte [] row, - final byte [] column, final long timestamp, final int numVersions) - throws IOException; /** * Return all the data for the row that matches row exactly, @@ -71,159 +56,104 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion { * * @param regionName region name * @param row row key - * @param columnFamily Column family to look for row in. + * @param family Column family to look for row in. * @return map of values * @throws IOException */ - public RowResult getClosestRowBefore(final byte [] regionName, - final byte [] row, final byte [] columnFamily) + public Result getClosestRowBefore(final byte [] regionName, + final byte [] row, final byte [] family) throws IOException; /** - * Get selected columns for the specified row at a given timestamp. * - * @param regionName region name - * @param row row key - * @param columns columns to get - * @param ts time stamp - * @param numVersions number of versions - * @param lockId lock id - * @return map of values + * @return the regions served by this regionserver + */ + public HRegion [] getOnlineRegionsAsArray(); + + /** + * Perform Get operation. + * @param regionName name of region to get from + * @param get Get operation + * @return Result * @throws IOException */ - public RowResult getRow(final byte [] regionName, final byte [] row, - final byte[][] columns, final long ts, - final int numVersions, final long lockId) - throws IOException; + public Result get(byte [] regionName, Get get) throws IOException; /** - * Applies a batch of updates via one RPC - * - * @param regionName name of the region to update - * @param b BatchUpdate - * @param lockId lock id + * Perform exists operation. + * @param regionName name of region to get from + * @param get Get operation describing cell to test + * @return true if exists * @throws IOException */ - public void batchUpdate(final byte [] regionName, final BatchUpdate b, - 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 batchUpdates(final byte[] regionName, final BatchUpdate[] b) - throws IOException; - - /** - * Applies a batch of updates to one row atomically via one RPC - * if the columns specified in expectedValues match - * the given values in expectedValues - * - * @param regionName name of the region to update - * @param b BatchUpdate - * @param expectedValues map of column names to expected data values. - * @return true if update was applied - * @throws IOException - */ - public boolean checkAndSave(final byte [] regionName, final BatchUpdate b, - final HbaseMapWritable expectedValues) - throws IOException; - + public boolean exists(byte [] regionName, Get get) throws IOException; /** - * Delete all cells that match the passed row and column and whose timestamp - * is equal-to or older than the passed timestamp. - * - * @param regionName region name - * @param row row key - * @param column column key - * @param timestamp Delete all entries that have this timestamp or older - * @param lockId lock id + * Put data into the specified region + * @param regionName + * @param put the data to be put * @throws IOException */ - public void deleteAll(byte [] regionName, byte [] row, byte [] column, - long timestamp, long lockId) - throws IOException; - - /** - * Delete all cells that match the passed row and whose - * timestamp is equal-to or older than the passed timestamp. - * - * @param regionName region name - * @param row row key - * @param timestamp Delete all entries that have this timestamp or older - * @param lockId lock id - * @throws IOException - */ - public void deleteAll(byte [] regionName, byte [] row, long timestamp, - long lockId) + public void put(final byte [] regionName, final Put put) throws IOException; /** - * Delete all cells that match the passed row & the column regex and whose - * timestamp is equal-to or older than the passed timestamp. + * Put an array of puts into the specified region + * @param regionName + * @param puts + * @return + * @throws IOException + */ + public int put(final byte[] regionName, final Put [] puts) + throws IOException; + + + /** + * Deletes all the KeyValues that match those found in the Delete object, + * if their ts <= to the Delete. In case of a delete with a specific ts it + * only deletes that specific KeyValue. + * @param regionName + * @param delete + * @throws IOException + */ + public void delete(final byte[] regionName, final Delete delete) + throws IOException; + + /** + * Atomically checks if a row/family/qualifier value match the expectedValue. + * If it does, it adds the put. * * @param regionName * @param row - * @param colRegex - * @param timestamp - * @param lockId + * @param family + * @param qualifier + * @param value the expected value + * @param put * @throws IOException + * @return true if the new put was execute, false otherwise */ - public void deleteAllByRegex(byte [] regionName, byte [] row, String colRegex, - long timestamp, long lockId) - throws IOException; - - /** - * Delete all cells for a row with matching column family with timestamps - * less than or equal to timestamp. - * - * @param regionName The name of the region to operate on - * @param row The row to operate on - * @param family The column family to match - * @param timestamp Timestamp to match - * @param lockId lock id - * @throws IOException - */ - public void deleteFamily(byte [] regionName, byte [] row, byte [] family, - long timestamp, long lockId) + public boolean checkAndPut(final byte[] regionName, final byte [] row, + final byte [] family, final byte [] qualifier, final byte [] value, + final Put put) throws IOException; /** - * Delete all cells for a row with matching column family regex with - * timestamps less than or equal to timestamp. + * Atomically increments a column value. If the column value isn't long-like, + * this could throw an exception. * - * @param regionName The name of the region to operate on - * @param row The row to operate on - * @param familyRegex column family regex - * @param timestamp Timestamp to match - * @param lockId lock id + * @param regionName + * @param row + * @param family + * @param qualifier + * @param amount + * @return new incremented column value * @throws IOException */ - public void deleteFamilyByRegex(byte [] regionName, byte [] row, String familyRegex, - long timestamp, long lockId) + public long incrementColumnValue(byte [] regionName, byte [] row, + byte [] family, byte [] qualifier, long amount) throws IOException; - - /** - * Returns true if any cells exist for the given coordinate. - * - * @param regionName The name of the region - * @param row The row - * @param column The column, or null for any - * @param timestamp The timestamp, or LATEST_TIMESTAMP for any - * @param lockID lock id - * @return true if the row exists, false otherwise - * @throws IOException - */ - public boolean exists(byte [] regionName, byte [] row, byte [] column, - long timestamp, long lockID) - throws IOException; - + + // // remote scanner interface // @@ -232,20 +162,11 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion { * Opens a remote scanner with a RowFilter. * * @param regionName name of region to scan - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex for column family name. A column name is judged to be - * regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row to scan - * @param timestamp only return values whose timestamp is <= this value - * @param filter RowFilter for filtering results at the row-level. - * + * @param scan configured scan object * @return scannerId scanner identifier used in other calls * @throws IOException */ - public long openScanner(final byte [] regionName, final byte [][] columns, - final byte [] startRow, long timestamp, RowFilterInterface filter) + public long openScanner(final byte [] regionName, final Scan scan) throws IOException; /** @@ -254,7 +175,7 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion { * @return map of values * @throws IOException */ - public RowResult next(long scannerId) throws IOException; + public Result next(long scannerId) throws IOException; /** * Get the next set of values @@ -263,7 +184,7 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion { * @return map of values * @throws IOException */ - public RowResult[] next(long scannerId, int numberOfRows) throws IOException; + public Result [] next(long scannerId, int numberOfRows) throws IOException; /** * Close a scanner @@ -272,7 +193,7 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion { * @throws IOException */ public void close(long scannerId) throws IOException; - + /** * Opens a remote row lock. * @@ -294,19 +215,6 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion { public void unlockRow(final byte [] regionName, final long lockId) throws IOException; - /** - * Atomically increments a column value. If the column value isn't long-like, this could - * throw an exception. - * - * @param regionName - * @param row - * @param column - * @param amount - * @return new incremented column value - * @throws IOException - */ - public long incrementColumnValue(byte [] regionName, byte [] row, - byte [] column, long amount) throws IOException; /** * Method used when a master is taking the place of another failed one. diff --git a/src/java/org/apache/hadoop/hbase/ipc/TransactionalRegionInterface.java b/src/java/org/apache/hadoop/hbase/ipc/TransactionalRegionInterface.java index 20e50c8706a..685f51296f0 100644 --- a/src/java/org/apache/hadoop/hbase/ipc/TransactionalRegionInterface.java +++ b/src/java/org/apache/hadoop/hbase/ipc/TransactionalRegionInterface.java @@ -19,7 +19,8 @@ package org.apache.hadoop.hbase.ipc; import java.io.IOException; -import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; @@ -136,33 +137,24 @@ public interface TransactionalRegionInterface extends HRegionInterface { * * @param transactionId * @param regionName region name - * @param row row key + * @param delete * @param timestamp Delete all entries that have this timestamp or older * @throws IOException */ - public void deleteAll(long transactionId, byte[] regionName, byte[] row, - long timestamp) throws IOException; - + public void delete(long transactionId, byte [] regionName, Delete delete) + throws IOException; + /** * Opens a remote scanner with a RowFilter. * * @param transactionId * @param regionName name of region to scan - * @param columns columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible to - * pass a regex for column family name. A column name is judged to be regex if - * it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param startRow starting row to scan - * @param timestamp only return values whose timestamp is <= this value - * @param filter RowFilter for filtering results at the row-level. - * + * @param scan * @return scannerId scanner identifier used in other calls * @throws IOException */ public long openScanner(final long transactionId, final byte[] regionName, - final byte[][] columns, final byte[] startRow, long timestamp, - RowFilterInterface filter) throws IOException; + Scan scan) throws IOException; /** * Applies a batch of updates via one RPC diff --git a/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java b/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java index ba82b7a9f9b..cf92960c3f4 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java +++ b/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java @@ -1,5 +1,5 @@ /** - * Copyright 2008 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -28,7 +28,9 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Scanner; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.filter.RowFilterSet; import org.apache.hadoop.hbase.filter.StopRowFilter; @@ -73,7 +75,7 @@ import org.apache.hadoop.util.StringUtils; * */ public abstract class TableInputFormatBase -implements InputFormat { +implements InputFormat { final Log LOG = LogFactory.getLog(TableInputFormatBase.class); private byte [][] inputColumns; private HTable table; @@ -84,12 +86,12 @@ implements InputFormat { * Iterate over an HBase table data, return (Text, RowResult) pairs */ protected class TableRecordReader - implements RecordReader { + implements RecordReader { private byte [] startRow; private byte [] endRow; private byte [] lastRow; private RowFilterInterface trrRowFilter; - private Scanner scanner; + private ResultScanner scanner; private HTable htable; private byte [][] trrInputColumns; @@ -106,16 +108,21 @@ implements InputFormat { new HashSet(); rowFiltersSet.add(new WhileMatchRowFilter(new StopRowFilter(endRow))); rowFiltersSet.add(trrRowFilter); - this.scanner = this.htable.getScanner(trrInputColumns, startRow, - new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL, - rowFiltersSet)); + Scan scan = new Scan(startRow); + scan.addColumns(trrInputColumns); +// scan.setFilter(new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL, +// rowFiltersSet)); + this.scanner = this.htable.getScanner(scan); } else { - this.scanner = - this.htable.getScanner(trrInputColumns, firstRow, endRow); + Scan scan = new Scan(firstRow, endRow); + scan.addColumns(trrInputColumns); + this.scanner = this.htable.getScanner(scan); } } else { - this.scanner = - this.htable.getScanner(trrInputColumns, firstRow, trrRowFilter); + Scan scan = new Scan(firstRow); + scan.addColumns(trrInputColumns); +// scan.setFilter(trrRowFilter); + this.scanner = this.htable.getScanner(scan); } } @@ -182,8 +189,8 @@ implements InputFormat { * * @see org.apache.hadoop.mapred.RecordReader#createValue() */ - public RowResult createValue() { - return new RowResult(); + public Result createValue() { + return new Result(); } public long getPos() { @@ -203,9 +210,9 @@ implements InputFormat { * @return true if there was more data * @throws IOException */ - public boolean next(ImmutableBytesWritable key, RowResult value) + public boolean next(ImmutableBytesWritable key, Result value) throws IOException { - RowResult result; + Result result; try { result = this.scanner.next(); } catch (UnknownScannerException e) { @@ -232,7 +239,7 @@ implements InputFormat { * @see org.apache.hadoop.mapred.InputFormat#getRecordReader(InputSplit, * JobConf, Reporter) */ - public RecordReader getRecordReader( + public RecordReader getRecordReader( InputSplit split, JobConf job, Reporter reporter) throws IOException { TableSplit tSplit = (TableSplit) split; diff --git a/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java b/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java index 9d6adb4bcaf..9bff33b1ab1 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java +++ b/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java @@ -1,5 +1,5 @@ /** - * Copyright 2007 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.mapred.FileAlreadyExistsException; @@ -40,7 +41,7 @@ import org.apache.hadoop.util.Progressable; * Convert Map/Reduce output and write it to an HBase table */ public class TableOutputFormat extends -FileOutputFormat { +FileOutputFormat { /** JobConf parameter that specifies the output table */ public static final String OUTPUT_TABLE = "hbase.mapred.outputtable"; @@ -51,7 +52,7 @@ FileOutputFormat { * and write to an HBase table */ protected static class TableRecordWriter - implements RecordWriter { + implements RecordWriter { private HTable m_table; /** @@ -69,8 +70,8 @@ FileOutputFormat { } public void write(ImmutableBytesWritable key, - BatchUpdate value) throws IOException { - m_table.commit(new BatchUpdate(value)); + Put value) throws IOException { + m_table.put(new Put(value)); } } diff --git a/src/java/org/apache/hadoop/hbase/master/BaseScanner.java b/src/java/org/apache/hadoop/hbase/master/BaseScanner.java index a2c5e8e8734..0297bd3802f 100644 --- a/src/java/org/apache/hadoop/hbase/master/BaseScanner.java +++ b/src/java/org/apache/hadoop/hbase/master/BaseScanner.java @@ -38,8 +38,9 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.UnknownScannerException; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.regionserver.HLog; import org.apache.hadoop.hbase.regionserver.HRegion; @@ -147,16 +148,18 @@ abstract class BaseScanner extends Chore implements HConstants { // Array to hold list of split parents found. Scan adds to list. After // scan we go check if parents can be removed. - Map splitParents = - new HashMap(); + Map splitParents = + new HashMap(); List emptyRows = new ArrayList(); int rows = 0; try { regionServer = master.connection.getHRegionConnection(region.getServer()); + scannerId = regionServer.openScanner(region.getRegionName(), - COLUMN_FAMILY_ARRAY, EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null); + new Scan().addFamily(HConstants.CATALOG_FAMILY)); while (true) { - RowResult values = regionServer.next(scannerId); + Result values = regionServer.next(scannerId); + if (values == null || values.size() == 0) { break; } @@ -165,8 +168,16 @@ abstract class BaseScanner extends Chore implements HConstants { emptyRows.add(values.getRow()); continue; } - String serverName = Writables.cellToString(values.get(COL_SERVER)); - long startCode = Writables.cellToLong(values.get(COL_STARTCODE)); + String serverName = ""; + byte [] val = values.getValue(CATALOG_FAMILY, SERVER_QUALIFIER); + if( val != null) { + serverName = Bytes.toString(val); + } + long startCode = 0L; + val = values.getValue(CATALOG_FAMILY, STARTCODE_QUALIFIER); + if(val != null) { + startCode = Bytes.toLong(val); + } // Note Region has been assigned. checkAssigned(info, serverName, startCode); @@ -213,7 +224,7 @@ abstract class BaseScanner extends Chore implements HConstants { // Take a look at split parents to see if any we can clean up. if (splitParents.size() > 0) { - for (Map.Entry e : splitParents.entrySet()) { + for (Map.Entry e : splitParents.entrySet()) { HRegionInfo hri = e.getKey(); cleanupSplits(region.getRegionName(), regionServer, hri, e.getValue()); } @@ -250,13 +261,13 @@ abstract class BaseScanner extends Chore implements HConstants { */ private boolean cleanupSplits(final byte [] metaRegionName, final HRegionInterface srvr, final HRegionInfo parent, - RowResult rowContent) + Result rowContent) throws IOException { boolean result = false; boolean hasReferencesA = hasReferences(metaRegionName, srvr, - parent.getRegionName(), rowContent, COL_SPLITA); + parent.getRegionName(), rowContent, CATALOG_FAMILY, SPLITA_QUALIFIER); boolean hasReferencesB = hasReferences(metaRegionName, srvr, - parent.getRegionName(), rowContent, COL_SPLITB); + parent.getRegionName(), rowContent, CATALOG_FAMILY, SPLITB_QUALIFIER); if (!hasReferencesA && !hasReferencesB) { LOG.info("Deleting region " + parent.getRegionNameAsString() + " (encoded=" + parent.getEncodedName() + @@ -283,15 +294,16 @@ abstract class BaseScanner extends Chore implements HConstants { */ private boolean hasReferences(final byte [] metaRegionName, final HRegionInterface srvr, final byte [] parent, - RowResult rowContent, final byte [] splitColumn) + Result rowContent, final byte [] splitFamily, byte [] splitQualifier) throws IOException { boolean result = false; HRegionInfo split = - Writables.getHRegionInfo(rowContent.get(splitColumn)); + Writables.getHRegionInfo(rowContent.getValue(splitFamily, splitQualifier)); if (split == null) { return result; } - Path tabledir = new Path(this.master.rootdir, split.getTableDesc().getNameAsString()); + Path tabledir = + new Path(this.master.rootdir, split.getTableDesc().getNameAsString()); for (HColumnDescriptor family: split.getTableDesc().getFamilies()) { Path p = Store.getStoreHomedir(tabledir, split.getEncodedName(), family.getName()); @@ -320,10 +332,10 @@ abstract class BaseScanner extends Chore implements HConstants { " no longer has references to " + Bytes.toString(parent)); } - BatchUpdate b = new BatchUpdate(parent); - b.delete(splitColumn); - srvr.batchUpdate(metaRegionName, b, -1L); - + Delete delete = new Delete(parent); + delete.deleteColumns(splitFamily, splitQualifier); + srvr.delete(metaRegionName, delete); + return result; } diff --git a/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java b/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java index b7c09f72678..ad99fb0a4c2 100644 --- a/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java +++ b/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java @@ -27,6 +27,8 @@ import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.util.Writables; @@ -78,11 +80,13 @@ class ChangeTableState extends TableOperation { } // Update meta table - BatchUpdate b = new BatchUpdate(i.getRegionName()); - updateRegionInfo(b, i); - b.delete(COL_SERVER); - b.delete(COL_STARTCODE); - server.batchUpdate(m.getRegionName(), b, -1L); + Put put = updateRegionInfo(i); + server.put(m.getRegionName(), put); + + Delete delete = new Delete(i.getRegionName()); + delete.deleteColumns(CATALOG_FAMILY, SERVER_QUALIFIER); + delete.deleteColumns(CATALOG_FAMILY, STARTCODE_QUALIFIER); + server.delete(m.getRegionName(), delete); if (LOG.isDebugEnabled()) { LOG.debug("Updated columns in row: " + i.getRegionNameAsString()); } @@ -125,9 +129,11 @@ class ChangeTableState extends TableOperation { servedRegions.clear(); } - protected void updateRegionInfo(final BatchUpdate b, final HRegionInfo i) + protected Put updateRegionInfo(final HRegionInfo i) throws IOException { i.setOffline(!online); - b.put(COL_REGIONINFO, Writables.getBytes(i)); + Put put = new Put(i.getRegionName()); + put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(i)); + return put; } } diff --git a/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java b/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java index bf5718e6a1d..a424b78f978 100644 --- a/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java +++ b/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java @@ -25,7 +25,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableNotDisabledException; -import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.util.Writables; @@ -47,9 +47,9 @@ abstract class ColumnOperation extends TableOperation { protected void updateRegionInfo(HRegionInterface server, byte [] regionName, HRegionInfo i) throws IOException { - BatchUpdate b = new BatchUpdate(i.getRegionName()); - b.put(COL_REGIONINFO, Writables.getBytes(i)); - server.batchUpdate(regionName, b, -1L); + Put put = new Put(i.getRegionName()); + put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(i)); + server.put(regionName, put); if (LOG.isDebugEnabled()) { LOG.debug("updated columns in row: " + i.getRegionNameAsString()); } diff --git a/src/java/org/apache/hadoop/hbase/master/HMaster.java b/src/java/org/apache/hadoop/hbase/master/HMaster.java index 5b24cded2cc..cb9295ee069 100644 --- a/src/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/src/java/org/apache/hadoop/hbase/master/HMaster.java @@ -1,5 +1,5 @@ /** - * Copyright 2007 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -23,16 +23,15 @@ import java.io.IOException; import java.lang.management.ManagementFactory; import java.lang.management.RuntimeMXBean; import java.lang.reflect.Constructor; -import java.net.InetAddress; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.NavigableMap; import java.util.Random; import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.DelayQueue; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; @@ -51,19 +50,20 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.HServerLoad; -import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.LocalHBaseCluster; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.RegionHistorian; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.TableExistsException; +import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.ServerConnection; import org.apache.hadoop.hbase.client.ServerConnectionManager; -import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.ipc.HBaseRPC; import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion; import org.apache.hadoop.hbase.ipc.HBaseServer; @@ -714,12 +714,14 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, byte [] metaRegionName = m.getRegionName(); HRegionInterface srvr = connection.getHRegionConnection(m.getServer()); byte[] firstRowInTable = Bytes.toBytes(tableName + ",,"); - long scannerid = srvr.openScanner(metaRegionName, COL_REGIONINFO_ARRAY, - firstRowInTable, LATEST_TIMESTAMP, null); + Scan scan = new Scan(firstRowInTable); + scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER); + long scannerid = srvr.openScanner(metaRegionName, scan); try { - RowResult data = srvr.next(scannerid); + Result data = srvr.next(scannerid); if (data != null && data.size() > 0) { - HRegionInfo info = Writables.getHRegionInfo(data.get(COL_REGIONINFO)); + HRegionInfo info = Writables.getHRegionInfo( + data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER)); if (info.getTableDesc().getNameAsString().equals(tableName)) { // A region for this table already exists. Ergo table exists. throw new TableExistsException(tableName); @@ -752,7 +754,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, public void deleteColumn(final byte [] tableName, final byte [] c) throws IOException { - new DeleteColumn(this, tableName, HStoreKey.getFamily(c)).process(); + new DeleteColumn(this, tableName, KeyValue.parseColumn(c)[0]).process(); } public void enableTable(final byte [] tableName) throws IOException { @@ -778,23 +780,23 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, for (MetaRegion m: regions) { byte [] metaRegionName = m.getRegionName(); HRegionInterface srvr = connection.getHRegionConnection(m.getServer()); + Scan scan = new Scan(firstRowInTable); + scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER); + scan.addColumn(CATALOG_FAMILY, SERVER_QUALIFIER); long scannerid = - srvr.openScanner(metaRegionName, - new byte[][] {COL_REGIONINFO, COL_SERVER}, - firstRowInTable, - LATEST_TIMESTAMP, - null); + srvr.openScanner(metaRegionName, scan); try { while (true) { - RowResult data = srvr.next(scannerid); + Result data = srvr.next(scannerid); if (data == null || data.size() <= 0) break; - HRegionInfo info = Writables.getHRegionInfo(data.get(COL_REGIONINFO)); + HRegionInfo info = Writables.getHRegionInfo( + data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER)); if (Bytes.compareTo(info.getTableDesc().getName(), tableName) == 0) { - Cell cell = data.get(COL_SERVER); - if (cell != null) { + byte [] value = data.getValue(CATALOG_FAMILY, SERVER_QUALIFIER); + if (value != null) { HServerAddress server = - new HServerAddress(Bytes.toString(cell.getValue())); + new HServerAddress(Bytes.toString(value)); result.add(new Pair(info, server)); } } else { @@ -816,25 +818,25 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, byte [] firstRowInTable = Bytes.toBytes(Bytes.toString(tableName) + ",,"); byte [] metaRegionName = m.getRegionName(); HRegionInterface srvr = connection.getHRegionConnection(m.getServer()); + Scan scan = new Scan(firstRowInTable); + scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER); + scan.addColumn(CATALOG_FAMILY, SERVER_QUALIFIER); long scannerid = - srvr.openScanner(metaRegionName, - new byte[][] {COL_REGIONINFO, COL_SERVER}, - firstRowInTable, - LATEST_TIMESTAMP, - null); + srvr.openScanner(metaRegionName, scan); try { while (true) { - RowResult data = srvr.next(scannerid); + Result data = srvr.next(scannerid); if (data == null || data.size() <= 0) break; - HRegionInfo info = Writables.getHRegionInfo(data.get(COL_REGIONINFO)); + HRegionInfo info = Writables.getHRegionInfo( + data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER)); if (Bytes.compareTo(info.getTableDesc().getName(), tableName) == 0) { if ((Bytes.compareTo(info.getStartKey(), rowKey) >= 0) && (Bytes.compareTo(info.getEndKey(), rowKey) < 0)) { - Cell cell = data.get(COL_SERVER); - if (cell != null) { + byte [] value = data.getValue(CATALOG_FAMILY, SERVER_QUALIFIER); + if (value != null) { HServerAddress server = - new HServerAddress(Bytes.toString(cell.getValue())); + new HServerAddress(Bytes.toString(value)); return new Pair(info, server); } } @@ -857,15 +859,17 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, for (MetaRegion m: regions) { byte [] metaRegionName = m.getRegionName(); HRegionInterface srvr = connection.getHRegionConnection(m.getServer()); - RowResult data = srvr.getRow(metaRegionName, regionName, - new byte[][] {COL_REGIONINFO, COL_SERVER}, - HConstants.LATEST_TIMESTAMP, 1, -1L); + Get get = new Get(regionName); + get.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER); + get.addColumn(CATALOG_FAMILY, SERVER_QUALIFIER); + Result data = srvr.get(metaRegionName, get); if(data == null || data.size() <= 0) continue; - HRegionInfo info = Writables.getHRegionInfo(data.get(COL_REGIONINFO)); - Cell cell = data.get(COL_SERVER); - if(cell != null) { + HRegionInfo info = Writables.getHRegionInfo( + data.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER)); + byte [] value = data.getValue(CATALOG_FAMILY, SERVER_QUALIFIER); + if(value != null) { HServerAddress server = - new HServerAddress(Bytes.toString(cell.getValue())); + new HServerAddress(Bytes.toString(value)); return new Pair(info, server); } } @@ -876,15 +880,18 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, * Get row from meta table. * @param row * @param columns - * @return RowResult + * @return Result * @throws IOException */ - protected RowResult getFromMETA(final byte [] row, final byte [][] columns) + protected Result getFromMETA(final byte [] row, final byte [] family) throws IOException { MetaRegion meta = this.regionManager.getMetaRegionForRow(row); HRegionInterface srvr = getMETAServer(meta); - return srvr.getRow(meta.getRegionName(), row, columns, - HConstants.LATEST_TIMESTAMP, 1, -1); + + Get get = new Get(row); + get.addFamily(family); + + return srvr.get(meta.getRegionName(), get); } /* @@ -897,10 +904,11 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, return this.connection.getHRegionConnection(meta.getServer()); } - public void modifyTable(final byte[] tableName, int op, Writable[] args) + public void modifyTable(final byte[] tableName, HConstants.Modify op, + Writable[] args) throws IOException { switch (op) { - case MODIFY_TABLE_SET_HTD: + case TABLE_SET_HTD: if (args == null || args.length < 1 || !(args[0] instanceof HTableDescriptor)) throw new IOException("SET_HTD request requires an HTableDescriptor"); @@ -909,10 +917,10 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, new ModifyTableMeta(this, tableName, htd).process(); break; - case MODIFY_TABLE_SPLIT: - case MODIFY_TABLE_COMPACT: - case MODIFY_TABLE_MAJOR_COMPACT: - case MODIFY_TABLE_FLUSH: + case TABLE_SPLIT: + case TABLE_COMPACT: + case TABLE_MAJOR_COMPACT: + case TABLE_FLUSH: if (args != null && args.length > 0) { if (!(args[0] instanceof ImmutableBytesWritable)) throw new IOException( @@ -936,7 +944,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, } break; - case MODIFY_CLOSE_REGION: + case CLOSE_REGION: if (args == null || args.length < 1 || args.length > 2) { throw new IOException("Requires at least a region name; " + "or cannot have more than region name and servername"); @@ -947,12 +955,13 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, if (args.length == 2) { servername = Bytes.toString(((ImmutableBytesWritable)args[1]).get()); } - // Need hri - RowResult rr = getFromMETA(regionname, HConstants.COLUMN_FAMILY_ARRAY); + // Need hri + Result rr = getFromMETA(regionname, HConstants.CATALOG_FAMILY); HRegionInfo hri = getHRegionInfo(rr.getRow(), rr); if (servername == null) { // Get server from the .META. if it wasn't passed as argument - servername = Writables.cellToString(rr.get(COL_SERVER)); + servername = + Bytes.toString(rr.getValue(CATALOG_FAMILY, SERVER_QUALIFIER)); } LOG.info("Marking " + hri.getRegionNameAsString() + " as closed on " + servername + "; cleaning SERVER + STARTCODE; " + @@ -995,7 +1004,8 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, public HBaseConfiguration getConfiguration() { return this.conf; } - + + // TODO ryan rework this function /* * Get HRegionInfo from passed META map of row values. * Returns null if none found (and logs fact that expected COL_REGIONINFO @@ -1005,22 +1015,24 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, * @return Null or found HRegionInfo. * @throws IOException */ - HRegionInfo getHRegionInfo(final byte [] row, final Map map) + HRegionInfo getHRegionInfo(final byte [] row, final Result res) throws IOException { - Cell regioninfo = map.get(COL_REGIONINFO); + byte [] regioninfo = res.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER); if (regioninfo == null) { StringBuilder sb = new StringBuilder(); - for (byte [] e: map.keySet()) { + NavigableMap infoMap = res.getFamilyMap(CATALOG_FAMILY); + for (byte [] e: infoMap.keySet()) { if (sb.length() > 0) { sb.append(", "); } - sb.append(Bytes.toString(e)); + sb.append(Bytes.toString(CATALOG_FAMILY) + ":" + Bytes.toString(e)); } - LOG.warn(Bytes.toString(COL_REGIONINFO) + " is empty for row: " + + LOG.warn(Bytes.toString(CATALOG_FAMILY) + ":" + + Bytes.toString(REGIONINFO_QUALIFIER) + " is empty for row: " + Bytes.toString(row) + "; has keys: " + sb.toString()); return null; } - return Writables.getHRegionInfo(regioninfo.getValue()); + return Writables.getHRegionInfo(regioninfo); } /* @@ -1065,7 +1077,6 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, System.exit(0); } - @SuppressWarnings("null") protected static void doMain(String [] args, Class masterClass) { diff --git a/src/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java b/src/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java index 7caa4bdc965..6e44b5038f3 100644 --- a/src/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java +++ b/src/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java @@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableNotDisabledException; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.util.Bytes; @@ -50,9 +51,9 @@ class ModifyTableMeta extends TableOperation { protected void updateRegionInfo(HRegionInterface server, byte [] regionName, HRegionInfo i) throws IOException { - BatchUpdate b = new BatchUpdate(i.getRegionName()); - b.put(COL_REGIONINFO, Writables.getBytes(i)); - server.batchUpdate(regionName, b, -1L); + Put put = new Put(i.getRegionName()); + put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(i)); + server.put(regionName, put); LOG.debug("updated HTableDescriptor for region " + i.getRegionNameAsString()); } diff --git a/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java b/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java index b1a4b21ea6b..5276a8d371b 100644 --- a/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java +++ b/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java @@ -25,6 +25,9 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.RegionHistorian; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.util.Bytes; @@ -78,11 +81,12 @@ class ProcessRegionOpen extends ProcessRegionStatusChange { " in region " + Bytes.toString(metaRegionName) + " with startcode " + serverInfo.getStartCode() + " and server " + serverInfo.getServerAddress()); - BatchUpdate b = new BatchUpdate(regionInfo.getRegionName()); - b.put(COL_SERVER, + Put p = new Put(regionInfo.getRegionName()); + p.add(CATALOG_FAMILY, SERVER_QUALIFIER, Bytes.toBytes(serverInfo.getServerAddress().toString())); - b.put(COL_STARTCODE, Bytes.toBytes(serverInfo.getStartCode())); - server.batchUpdate(metaRegionName, b, -1L); + p.add(CATALOG_FAMILY, STARTCODE_QUALIFIER, + Bytes.toBytes(serverInfo.getStartCode())); + server.put(metaRegionName, p); if (!historian.isOnline()) { // This is safest place to do the onlining of the historian in // the master. When we get to here, we know there is a .META. diff --git a/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java b/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java index 11da2601cee..bc182445833 100644 --- a/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java +++ b/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java @@ -1,5 +1,5 @@ /** - * Copyright 2008 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.regionserver.HLog; import org.apache.hadoop.hbase.regionserver.HRegion; @@ -113,7 +115,7 @@ class ProcessServerShutdown extends RegionServerOperation { List emptyRows = new ArrayList(); try { while (true) { - RowResult values = null; + Result values = null; try { values = server.next(scannerId); } catch (IOException e) { @@ -129,8 +131,10 @@ class ProcessServerShutdown extends RegionServerOperation { // shutdown server but that would mean that we'd reassign regions that // were already out being assigned, ones that were product of a split // that happened while the shutdown was being processed. - String serverAddress = Writables.cellToString(values.get(COL_SERVER)); - long startCode = Writables.cellToLong(values.get(COL_STARTCODE)); + String serverAddress = + Bytes.toString(values.getValue(CATALOG_FAMILY, SERVER_QUALIFIER)); + long startCode = + Bytes.toLong(values.getValue(CATALOG_FAMILY, STARTCODE_QUALIFIER)); String serverName = null; if (serverAddress != null && serverAddress.length() > 0) { serverName = HServerInfo.getServerName(serverAddress, startCode); @@ -145,6 +149,7 @@ class ProcessServerShutdown extends RegionServerOperation { Bytes.toString(row)); } +// HRegionInfo info = master.getHRegionInfo(row, values.rowResult()); HRegionInfo info = master.getHRegionInfo(row, values); if (info == null) { emptyRows.add(row); @@ -221,9 +226,10 @@ class ProcessServerShutdown extends RegionServerOperation { LOG.debug("process server shutdown scanning root region on " + master.getRootRegionLocation().getBindAddress()); } + Scan scan = new Scan(); + scan.addFamily(CATALOG_FAMILY); long scannerId = server.openScanner( - HRegionInfo.ROOT_REGIONINFO.getRegionName(), COLUMN_FAMILY_ARRAY, - EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null); + HRegionInfo.ROOT_REGIONINFO.getRegionName(), scan); scanMetaRegion(server, scannerId, HRegionInfo.ROOT_REGIONINFO.getRegionName()); return true; @@ -240,9 +246,10 @@ class ProcessServerShutdown extends RegionServerOperation { LOG.debug("process server shutdown scanning " + Bytes.toString(m.getRegionName()) + " on " + m.getServer()); } - long scannerId = - server.openScanner(m.getRegionName(), COLUMN_FAMILY_ARRAY, - EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null); + Scan scan = new Scan(); + scan.addFamily(CATALOG_FAMILY); + long scannerId = server.openScanner( + HRegionInfo.ROOT_REGIONINFO.getRegionName(), scan); scanMetaRegion(server, scannerId, m.getRegionName()); return true; } diff --git a/src/java/org/apache/hadoop/hbase/master/RegionManager.java b/src/java/org/apache/hadoop/hbase/master/RegionManager.java index b50a89c4328..6c65e2a50b5 100644 --- a/src/java/org/apache/hadoop/hbase/master/RegionManager.java +++ b/src/java/org/apache/hadoop/hbase/master/RegionManager.java @@ -49,12 +49,12 @@ import org.apache.hadoop.hbase.HServerLoad; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.RegionHistorian; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.HMsg; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper; @@ -723,9 +723,12 @@ class RegionManager implements HConstants { // 3. Insert into meta HRegionInfo info = region.getRegionInfo(); byte [] regionName = region.getRegionName(); - BatchUpdate b = new BatchUpdate(regionName); - b.put(COL_REGIONINFO, Writables.getBytes(info)); - server.batchUpdate(metaRegionName, b, -1L); + + Put put = new Put(regionName); + byte [] infoBytes = Writables.getBytes(info); + String infoString = new String(infoBytes); + put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(info)); + server.put(metaRegionName, put); // 4. Close the new region to flush it to disk. Close its log file too. region.close(); @@ -1204,18 +1207,21 @@ class RegionManager implements HConstants { * @param op */ public void startAction(byte[] regionName, HRegionInfo info, - HServerAddress server, int op) { + HServerAddress server, HConstants.Modify op) { + if (LOG.isDebugEnabled()) { + LOG.debug("Adding operation " + op + " from tasklist"); + } switch (op) { - case HConstants.MODIFY_TABLE_SPLIT: + case TABLE_SPLIT: startAction(regionName, info, server, this.regionsToSplit); break; - case HConstants.MODIFY_TABLE_COMPACT: + case TABLE_COMPACT: startAction(regionName, info, server, this.regionsToCompact); break; - case HConstants.MODIFY_TABLE_MAJOR_COMPACT: + case TABLE_MAJOR_COMPACT: startAction(regionName, info, server, this.regionsToMajorCompact); break; - case HConstants.MODIFY_TABLE_FLUSH: + case TABLE_FLUSH: startAction(regionName, info, server, this.regionsToFlush); break; default: @@ -1233,18 +1239,21 @@ class RegionManager implements HConstants { * @param regionName * @param op */ - public void endAction(byte[] regionName, int op) { + public void endAction(byte[] regionName, HConstants.Modify op) { + if (LOG.isDebugEnabled()) { + LOG.debug("Removing operation " + op + " from tasklist"); + } switch (op) { - case HConstants.MODIFY_TABLE_SPLIT: + case TABLE_SPLIT: this.regionsToSplit.remove(regionName); break; - case HConstants.MODIFY_TABLE_COMPACT: + case TABLE_COMPACT: this.regionsToCompact.remove(regionName); break; - case HConstants.MODIFY_TABLE_MAJOR_COMPACT: + case TABLE_MAJOR_COMPACT: this.regionsToMajorCompact.remove(regionName); break; - case HConstants.MODIFY_TABLE_FLUSH: + case TABLE_FLUSH: this.regionsToFlush.remove(regionName); break; default: @@ -1267,6 +1276,9 @@ class RegionManager implements HConstants { * @param returnMsgs */ public void applyActions(HServerInfo serverInfo, ArrayList returnMsgs) { + if (LOG.isDebugEnabled()) { + LOG.debug("Applying operation in tasklists to region"); + } applyActions(serverInfo, returnMsgs, this.regionsToCompact, HMsg.Type.MSG_REGION_COMPACT); applyActions(serverInfo, returnMsgs, this.regionsToSplit, diff --git a/src/java/org/apache/hadoop/hbase/master/TableOperation.java b/src/java/org/apache/hadoop/hbase/master/TableOperation.java index 2127bd0a212..bf240655012 100644 --- a/src/java/org/apache/hadoop/hbase/master/TableOperation.java +++ b/src/java/org/apache/hadoop/hbase/master/TableOperation.java @@ -31,10 +31,10 @@ import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Writables; /** * Abstract base class for operations that need to examine all HRegionInfo @@ -80,26 +80,28 @@ abstract class TableOperation implements HConstants { // Open a scanner on the meta region byte [] tableNameMetaStart = Bytes.toBytes(Bytes.toString(tableName) + ",,"); - - long scannerId = server.openScanner(m.getRegionName(), - COLUMN_FAMILY_ARRAY, tableNameMetaStart, HConstants.LATEST_TIMESTAMP, null); + Scan scan = new Scan(tableNameMetaStart).addFamily(CATALOG_FAMILY); + long scannerId = server.openScanner(m.getRegionName(), scan); List emptyRows = new ArrayList(); try { while (true) { - RowResult values = server.next(scannerId); - if(values == null || values.size() == 0) { + Result values = server.next(scannerId); + if(values == null || values.isEmpty()) { break; } HRegionInfo info = this.master.getHRegionInfo(values.getRow(), values); if (info == null) { emptyRows.add(values.getRow()); - LOG.error(Bytes.toString(COL_REGIONINFO) + " not found on " + + LOG.error(Bytes.toString(CATALOG_FAMILY) + ":" + + Bytes.toString(REGIONINFO_QUALIFIER) + " not found on " + Bytes.toString(values.getRow())); continue; } - String serverAddress = Writables.cellToString(values.get(COL_SERVER)); - long startCode = Writables.cellToLong(values.get(COL_STARTCODE)); + String serverAddress = + Bytes.toString(values.getValue(CATALOG_FAMILY, SERVER_QUALIFIER)); + long startCode = + Bytes.toLong(values.getValue(CATALOG_FAMILY, STARTCODE_QUALIFIER)); String serverName = null; if (serverAddress != null && serverAddress.length() > 0) { serverName = HServerInfo.getServerName(serverAddress, startCode); diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java b/src/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java new file mode 100644 index 00000000000..15b7e9eb4e8 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java @@ -0,0 +1,112 @@ +/** + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +/** + * Simple wrapper for a byte buffer and a counter. Does not copy. + *

    + * NOT thread-safe because it is not used in a multi-threaded context, yet. + */ +public class ColumnCount { + private byte [] bytes; + private int offset; + private int length; + private int count; + + /** + * Constructor + * @param column the qualifier to count the versions for + */ + public ColumnCount(byte [] column) { + this(column, 0); + } + + /** + * Constructor + * @param column the qualifier to count the versions for + * @param count initial count + */ + public ColumnCount(byte [] column, int count) { + this(column, 0, column.length, count); + } + + /** + * Constuctor + * @param column the qualifier to count the versions for + * @param offset in the passed buffer where to start the qualifier from + * @param length of the qualifier + * @param count initial count + */ + public ColumnCount(byte [] column, int offset, int length, int count) { + this.bytes = column; + this.offset = offset; + this.length = length; + this.count = count; + } + + /** + * @return the buffer + */ + public byte [] getBuffer(){ + return this.bytes; + } + + /** + * @return the offset + */ + public int getOffset(){ + return this.offset; + } + + /** + * @return the length + */ + public int getLength(){ + return this.length; + } + + /** + * Decrement the current version count + * @return current count + */ + public int decrement() { + return --count; + } + + /** + * Increment the current version count + * @return current count + */ + public int increment() { + return ++count; + } + + /** + * Check to see if needed to fetch more versions + * @param max + * @return true if more versions are needed, false otherwise + */ + public boolean needMore(int max) { + if(this.count < max) { + return true; + } + return false; + } +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java b/src/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java new file mode 100644 index 00000000000..bce59f67d56 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java @@ -0,0 +1,78 @@ +/** + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode; + +/** + * Implementing classes of this interface will be used for the tracking + * and enforcement of columns and numbers of versions during the course of a + * Get or Scan operation. + *

    + * Currently there are two different types of Store/Family-level queries. + *

    • {@link ExplicitColumnTracker} is used when the query specifies + * one or more column qualifiers to return in the family. + *
    • {@link WildcardColumnTracker} is used when the query asks for all + * qualifiers within the family. + *

      + * This class is utilized by {@link QueryMatcher} through two methods: + *

      • {@link checkColumn} is called when a Put satisfies all other + * conditions of the query. This method returns a {@link MatchCode} to define + * what action should be taken. + *
      • {@link update} is called at the end of every StoreFile or Memcache. + *

        + * This class is NOT thread-safe as queries are never multi-threaded + */ +public interface ColumnTracker { + /** + * Keeps track of the number of versions for the columns asked for + * @param bytes + * @param offset + * @param length + * @return + */ + public MatchCode checkColumn(byte [] bytes, int offset, int length); + /** + * Updates internal variables in between files + */ + public void update(); + /** + * Resets the Matcher + */ + public void reset(); + /** + * + * @return + */ + public boolean done(); + + /** + * Used by matcher and scan/get to get a hint of the next column + * to seek to after checkColumn() returns SKIP. Returns the next interesting + * column we want, or NULL there is none (wildcard scanner). + * + * Implementations aren't required to return anything useful unless the most recent + * call was to checkColumn() and the return code was SKIP. This is pretty implementation + * detail-y, but optimizations are like that. + * + * @return null, or a ColumnCount that we should seek to + */ + public ColumnCount getColumnHint(); +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java b/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java index 6e48bb5bb77..6b5596b6221 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java @@ -30,11 +30,11 @@ import org.apache.hadoop.util.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; @@ -202,18 +202,21 @@ class CompactSplitThread extends Thread implements HConstants { // Inform the HRegionServer that the parent HRegion is no-longer online. this.server.removeFromOnlineRegions(oldRegionInfo); - BatchUpdate update = new BatchUpdate(oldRegionInfo.getRegionName()); - update.put(COL_REGIONINFO, Writables.getBytes(oldRegionInfo)); - update.put(COL_SPLITA, Writables.getBytes(newRegions[0].getRegionInfo())); - update.put(COL_SPLITB, Writables.getBytes(newRegions[1].getRegionInfo())); - t.commit(update); + Put put = new Put(oldRegionInfo.getRegionName()); + put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, + Writables.getBytes(oldRegionInfo)); + put.add(CATALOG_FAMILY, SPLITA_QUALIFIER, + Writables.getBytes(newRegions[0].getRegionInfo())); + put.add(CATALOG_FAMILY, SPLITB_QUALIFIER, + Writables.getBytes(newRegions[0].getRegionInfo())); + t.put(put); // Add new regions to META for (int i = 0; i < newRegions.length; i++) { - update = new BatchUpdate(newRegions[i].getRegionName()); - update.put(COL_REGIONINFO, Writables.getBytes( + put = new Put(newRegions[i].getRegionName()); + put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes( newRegions[i].getRegionInfo())); - t.commit(update); + t.put(put); } // Now tell the master about the new regions diff --git a/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java b/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java new file mode 100644 index 00000000000..3e4198388b6 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java @@ -0,0 +1,120 @@ +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + + +/** + * Class that provides static method needed when putting deletes into memcache + */ +public class DeleteCompare { + + /** + * Return codes from deleteCompare. + */ + enum DeleteCode { + /** + * Do nothing. Move to next KV in Memcache + */ + SKIP, + + /** + * Add to the list of deletes. + */ + DELETE, + + /** + * Stop looking at KVs in Memcache. Finalize. + */ + DONE + } + + /** + * Method used when putting deletes into memcache to remove all the previous + * entries that are affected by this Delete + * @param mem + * @param deleteBuffer + * @param deleteRowOffset + * @param deleteRowLength + * @param deleteQualifierOffset + * @param deleteQualifierLength + * @param deleteTimeOffset + * @param deleteType + * @param comparator + * @return SKIP if current KeyValue should not be deleted, DELETE if + * current KeyValue should be deleted and DONE when the current KeyValue is + * out of the Deletes range + */ + public static DeleteCode deleteCompare(KeyValue mem, byte [] deleteBuffer, + int deleteRowOffset, short deleteRowLength, int deleteQualifierOffset, + int deleteQualifierLength, int deleteTimeOffset, byte deleteType, + KeyValue.KeyComparator comparator) { + + //Parsing new KeyValue + byte [] memBuffer = mem.getBuffer(); + int memOffset = mem.getOffset(); + + //Getting key lengths + int memKeyLen = Bytes.toInt(memBuffer, memOffset); + memOffset += Bytes.SIZEOF_INT; + + //Skipping value lengths + memOffset += Bytes.SIZEOF_INT; + + //Getting row lengths + short memRowLen = Bytes.toShort(memBuffer, memOffset); + memOffset += Bytes.SIZEOF_SHORT; + int res = comparator.compareRows(memBuffer, memOffset, memRowLen, + deleteBuffer, deleteRowOffset, deleteRowLength); + if(res > 0) { + return DeleteCode.DONE; + } else if(res < 0){ + System.out.println("SKIPPING ROW"); + return DeleteCode.SKIP; + } + + memOffset += memRowLen; + + //Getting family lengths + byte memFamLen = memBuffer[memOffset]; + memOffset += Bytes.SIZEOF_BYTE + memFamLen; + + //Get column lengths + int memQualifierLen = memKeyLen - memRowLen - memFamLen - + Bytes.SIZEOF_SHORT - Bytes.SIZEOF_BYTE - Bytes.SIZEOF_LONG - + Bytes.SIZEOF_BYTE; + + //Compare timestamp + int tsOffset = memOffset + memQualifierLen; + int timeRes = Bytes.compareTo(memBuffer, tsOffset, Bytes.SIZEOF_LONG, + deleteBuffer, deleteTimeOffset, Bytes.SIZEOF_LONG); + + if(deleteType == KeyValue.Type.DeleteFamily.getCode()) { + if(timeRes <= 0){ + return DeleteCode.DELETE; + } + return DeleteCode.SKIP; + } + + //Compare columns + res = Bytes.compareTo(memBuffer, memOffset, memQualifierLen, + deleteBuffer, deleteQualifierOffset, deleteQualifierLength); + if(res < 0) { + return DeleteCode.SKIP; + } else if(res > 0) { + return DeleteCode.DONE; + } + // same column, compare the time. + if(timeRes == 0) { + return DeleteCode.DELETE; + } else if (timeRes < 0) { + if(deleteType == KeyValue.Type.DeleteColumn.getCode()) { + return DeleteCode.DELETE; + } + return DeleteCode.DONE; + } else { + System.out.println("SKIPPING TS"); + return DeleteCode.SKIP; + } + } +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/DeleteTracker.java b/src/java/org/apache/hadoop/hbase/regionserver/DeleteTracker.java new file mode 100644 index 00000000000..7f65381d805 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/DeleteTracker.java @@ -0,0 +1,97 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +/** + * This interface is used for the tracking and enforcement of Deletes + * during the course of a Get or Scan operation. + *

        + * This class is utilized through three methods: + *

        • {@link add} when encountering a Delete + *
        • {@link isDeleted} when checking if a Put KeyValue has been deleted + *
        • {@link update} when reaching the end of a StoreFile + */ +public interface DeleteTracker { + + /** + * Add the specified KeyValue to the list of deletes to check against for + * this row operation. + *

          + * This is called when a Delete is encountered in a StoreFile. + * @param buffer KeyValue buffer + * @param qualifierOffset column qualifier offset + * @param qualifierLength column qualifier length + * @param timestamp timestamp + * @param type delete type as byte + */ + public void add(byte [] buffer, int qualifierOffset, int qualifierLength, + long timestamp, byte type); + + /** + * Check if the specified KeyValue buffer has been deleted by a previously + * seen delete. + * @param buffer KeyValue buffer + * @param qualifierOffset column qualifier offset + * @param qualifierLength column qualifier length + * @param timestamp timestamp + * @return true is the specified KeyValue is deleted, false if not + */ + public boolean isDeleted(byte [] buffer, int qualifierOffset, + int qualifierLength, long timestamp); + + /** + * @return true if there are no current delete, false otherwise + */ + public boolean isEmpty(); + + /** + * Called at the end of every StoreFile. + *

          + * Many optimized implementations of Trackers will require an update at + * when the end of each StoreFile is reached. + */ + public void update(); + + /** + * Called between rows. + *

          + * This clears everything as if a new DeleteTracker was instantiated. + */ + public void reset(); + + + /** + * Return codes for comparison of two Deletes. + *

          + * The codes tell the merging function what to do. + *

          + * INCLUDE means add the specified Delete to the merged list. + * NEXT means move to the next element in the specified list(s). + */ + enum DeleteCompare { + INCLUDE_OLD_NEXT_OLD, + INCLUDE_OLD_NEXT_BOTH, + INCLUDE_NEW_NEXT_NEW, + INCLUDE_NEW_NEXT_BOTH, + NEXT_OLD, + NEXT_NEW + } + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java b/src/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java new file mode 100644 index 00000000000..09f83029995 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java @@ -0,0 +1,157 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import java.util.ArrayList; +import java.util.List; +import java.util.NavigableSet; +import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * This class is used for the tracking and enforcement of columns and numbers + * of versions during the course of a Get or Scan operation, when explicit + * column qualifiers have been asked for in the query. + * + * With a little magic (see {@link ScanQueryMatcher}), we can use this matcher + * for both scans and gets. The main difference is 'next' and 'done' collapse + * for the scan case (since we see all columns in order), and we only reset + * between rows. + * + *

          + * This class is utilized by {@link QueryMatcher} through two methods: + *

          • {@link checkColumn} is called when a Put satisfies all other + * conditions of the query. This method returns a {@link MatchCode} to define + * what action should be taken. + *
          • {@link update} is called at the end of every StoreFile or Memcache. + *

            + * This class is NOT thread-safe as queries are never multi-threaded + */ +public class ExplicitColumnTracker implements ColumnTracker { + + private int maxVersions; + private List columns; + private int index; + private ColumnCount column; + private NavigableSet origColumns; + + /** + * Default constructor. + * @param columns columns specified user in query + * @param maxVersions maximum versions to return per column + */ + public ExplicitColumnTracker(NavigableSet columns, int maxVersions) { + this.maxVersions = maxVersions; + this.origColumns = columns; + reset(); + } + + /** + * Done when there are no more columns to match against. + */ + public boolean done() { + return this.columns.size() == 0; + } + + public ColumnCount getColumnHint() { + return this.column; + } + + /** + * Checks against the parameters of the query and the columns which have + * already been processed by this query. + * @param bytes KeyValue buffer + * @param offset offset to the start of the qualifier + * @param length length of the qualifier + * @return MatchCode telling QueryMatcher what action to take + */ + public MatchCode checkColumn(byte [] bytes, int offset, int length) { + // No more columns left, we are done with this query + if(this.columns.size() == 0) { + return MatchCode.DONE; // done_row + } + + // No more columns to match against, done with storefile + if(this.column == null) { + return MatchCode.NEXT; // done_row + } + + // Compare specific column to current column + int ret = Bytes.compareTo(column.getBuffer(), column.getOffset(), + column.getLength(), bytes, offset, length); + + // Matches, decrement versions left and include + if(ret == 0) { + if(this.column.decrement() == 0) { + // Done with versions for this column + this.columns.remove(this.index); + if(this.columns.size() == this.index) { + // Will not hit any more columns in this storefile + this.column = null; + } else { + this.column = this.columns.get(this.index); + } + } + return MatchCode.INCLUDE; + } + + // Specified column is bigger than current column + // Move down current column and check again + if(ret <= -1) { + if(++this.index == this.columns.size()) { + // No more to match, do not include, done with storefile + return MatchCode.NEXT; // done_row + } + this.column = this.columns.get(this.index); + return checkColumn(bytes, offset, length); + } + + // Specified column is smaller than current column + // Skip + return MatchCode.SKIP; // skip to next column, with hint? + } + + /** + * Called at the end of every StoreFile or Memcache. + */ + public void update() { + if(this.columns.size() != 0) { + this.index = 0; + this.column = this.columns.get(this.index); + } else { + this.index = -1; + this.column = null; + } + } + + // Called between every row. + public void reset() { + buildColumnList(this.origColumns); + this.index = 0; + this.column = this.columns.get(this.index); + } + + private void buildColumnList(NavigableSet columns) { + this.columns = new ArrayList(columns.size()); + for(byte [] column : columns) { + this.columns.add(new ColumnCount(column,maxVersions)); + } + } +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/FailedLogCloseException.java b/src/java/org/apache/hadoop/hbase/regionserver/FailedLogCloseException.java index cd2dfaf2e75..cc2cc86554f 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/FailedLogCloseException.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/FailedLogCloseException.java @@ -28,10 +28,16 @@ import java.io.IOException; class FailedLogCloseException extends IOException { private static final long serialVersionUID = 1759152841462990925L; + /** + * + */ public FailedLogCloseException() { super(); } + /** + * @param arg0 + */ public FailedLogCloseException(String arg0) { super(arg0); } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/GetDeleteTracker.java b/src/java/org/apache/hadoop/hbase/regionserver/GetDeleteTracker.java new file mode 100644 index 00000000000..d7bcdd0193e --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/GetDeleteTracker.java @@ -0,0 +1,405 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * This class is responsible for the tracking and enforcement of Deletes + * during the course of a Get operation. + *

            + * This class is utilized through three methods: + *

            • {@link add} when encountering a Delete + *
            • {@link isDeleted} when checking if a Put KeyValue has been deleted + *
            • {@link update} when reaching the end of a StoreFile + *

              + * This class is NOT thread-safe as queries are never multi-threaded + */ +public class GetDeleteTracker implements DeleteTracker { + + private long familyStamp = -1L; + protected List deletes = null; + private List newDeletes = new ArrayList(); + private Iterator iterator; + private Delete delete = null; + + private KeyValue.KeyComparator comparator; + + /** + * Constructor + * @param comparator + */ + public GetDeleteTracker(KeyValue.KeyComparator comparator) { + this.comparator = comparator; + } + + /** + * Add the specified KeyValue to the list of deletes to check against for + * this row operation. + *

              + * This is called when a Delete is encountered in a StoreFile. + * @param kv + * @param type + * @param timestamp + */ + @Override + public void add(byte [] buffer, int qualifierOffset, int qualifierLength, + long timestamp, byte type) { + if(type == KeyValue.Type.DeleteFamily.getCode()) { + if(timestamp > familyStamp) { + familyStamp = timestamp; + } + return; + } + if(timestamp > familyStamp) { + this.newDeletes.add(new Delete(buffer, qualifierOffset, qualifierLength, + type, timestamp)); + } + } + + /** + * Check if the specified KeyValue buffer has been deleted by a previously + * seen delete. + * @param buffer KeyValue buffer + * @param qualifierOffset column qualifier offset + * @param qualifierLength column qualifier length + * @param timestamp timestamp + * @return true is the specified KeyValue is deleted, false if not + */ + @Override + public boolean isDeleted(byte [] buffer, int qualifierOffset, + int qualifierLength, long timestamp) { + + // Check against DeleteFamily + if(timestamp <= familyStamp) { + return true; + } + + // Check if there are other deletes + if(this.delete == null) { + return false; + } + + // Check column + int ret = comparator.compareRows(buffer, qualifierOffset, qualifierLength, + this.delete.buffer, this.delete.qualifierOffset, + this.delete.qualifierLength); + if(ret <= -1) { + // Have not reached the next delete yet + return false; + } else if(ret >= 1) { + // Deletes an earlier column, need to move down deletes + if(this.iterator.hasNext()) { + this.delete = this.iterator.next(); + } else { + this.delete = null; + return false; + } + return isDeleted(buffer, qualifierOffset, qualifierLength, timestamp); + } + + // Check Timestamp + if(timestamp > this.delete.timestamp) { + return false; + } + + // Check Type + switch(KeyValue.Type.codeToType(this.delete.type)) { + case Delete: + boolean equal = timestamp == this.delete.timestamp; + + if(this.iterator.hasNext()) { + this.delete = this.iterator.next(); + } else { + this.delete = null; + } + + if(equal){ + return true; + } + // timestamp < this.delete.timestamp + // Delete of an explicit column newer than current + return isDeleted(buffer, qualifierOffset, qualifierLength, timestamp); + case DeleteColumn: + return true; + } + + // should never reach this + return false; + } + + @Override + public boolean isEmpty() { + if(this.familyStamp == 0L && this.delete == null) { + return true; + } + return false; + } + + @Override + public void reset() { + this.deletes = null; + this.delete = null; + this.newDeletes = new ArrayList(); + this.familyStamp = 0L; + this.iterator = null; + } + + /** + * Called at the end of every StoreFile. + *

              + * Many optimized implementations of Trackers will require an update at + * when the end of each StoreFile is reached. + */ + @Override + public void update() { + // If no previous deletes, use new deletes and return + if(this.deletes == null || this.deletes.size() == 0) { + finalize(this.newDeletes); + return; + } + + // If no new delete, retain previous deletes and return + if(this.newDeletes.size() == 0) { + return; + } + + // Merge previous deletes with new deletes + List mergeDeletes = + new ArrayList(this.newDeletes.size()); + int oldIndex = 0; + int newIndex = 0; + + Delete newDelete = newDeletes.get(oldIndex); + Delete oldDelete = deletes.get(oldIndex); + while(true) { + switch(compareDeletes(oldDelete,newDelete)) { + case NEXT_NEW: { + if(++newIndex == newDeletes.size()) { + // Done with new, add the rest of old to merged and return + mergeDown(mergeDeletes, deletes, oldIndex); + finalize(mergeDeletes); + return; + } + newDelete = this.newDeletes.get(newIndex); + break; + } + + case INCLUDE_NEW_NEXT_NEW: { + mergeDeletes.add(newDelete); + if(++newIndex == newDeletes.size()) { + // Done with new, add the rest of old to merged and return + mergeDown(mergeDeletes, deletes, oldIndex); + finalize(mergeDeletes); + return; + } + newDelete = this.newDeletes.get(newIndex); + break; + } + + case INCLUDE_NEW_NEXT_BOTH: { + mergeDeletes.add(newDelete); + ++oldIndex; + ++newIndex; + if(oldIndex == deletes.size()) { + if(newIndex == newDeletes.size()) { + finalize(mergeDeletes); + return; + } + mergeDown(mergeDeletes, newDeletes, newIndex); + finalize(mergeDeletes); + return; + } else if(newIndex == newDeletes.size()) { + mergeDown(mergeDeletes, deletes, oldIndex); + finalize(mergeDeletes); + return; + } + oldDelete = this.deletes.get(oldIndex); + newDelete = this.newDeletes.get(newIndex); + break; + } + + case INCLUDE_OLD_NEXT_BOTH: { + mergeDeletes.add(oldDelete); + ++oldIndex; + ++newIndex; + if(oldIndex == deletes.size()) { + if(newIndex == newDeletes.size()) { + finalize(mergeDeletes); + return; + } + mergeDown(mergeDeletes, newDeletes, newIndex); + finalize(mergeDeletes); + return; + } else if(newIndex == newDeletes.size()) { + mergeDown(mergeDeletes, deletes, oldIndex); + finalize(mergeDeletes); + return; + } + oldDelete = this.deletes.get(oldIndex); + newDelete = this.newDeletes.get(newIndex); + break; + } + + case INCLUDE_OLD_NEXT_OLD: { + mergeDeletes.add(oldDelete); + if(++oldIndex == deletes.size()) { + mergeDown(mergeDeletes, newDeletes, newIndex); + finalize(mergeDeletes); + return; + } + oldDelete = this.deletes.get(oldIndex); + break; + } + + case NEXT_OLD: { + if(++oldIndex == deletes.size()) { + // Done with old, add the rest of new to merged and return + mergeDown(mergeDeletes, newDeletes, newIndex); + finalize(mergeDeletes); + return; + } + oldDelete = this.deletes.get(oldIndex); + } + } + } + } + + private void finalize(List mergeDeletes) { + this.deletes = mergeDeletes; + this.newDeletes = new ArrayList(); + if(this.deletes.size() > 0){ + this.iterator = deletes.iterator(); + this.delete = iterator.next(); + } + } + + private void mergeDown(List mergeDeletes, List srcDeletes, + int srcIndex) { + int index = srcIndex; + while(index < srcDeletes.size()) { + mergeDeletes.add(srcDeletes.get(index++)); + } + } + + + protected DeleteCompare compareDeletes(Delete oldDelete, Delete newDelete) { + + // Compare columns + // Just compairing qualifier portion, can keep on using Bytes.compareTo(). + int ret = Bytes.compareTo(oldDelete.buffer, oldDelete.qualifierOffset, + oldDelete.qualifierLength, newDelete.buffer, newDelete.qualifierOffset, + newDelete.qualifierLength); + + if(ret <= -1) { + return DeleteCompare.INCLUDE_OLD_NEXT_OLD; + } else if(ret >= 1) { + return DeleteCompare.INCLUDE_NEW_NEXT_NEW; + } + + // Same column + + // Branches below can be optimized. Keeping like this until testing + // is complete. + if(oldDelete.type == newDelete.type) { + // the one case where we can merge 2 deletes -> 1 delete. + if(oldDelete.type == KeyValue.Type.Delete.getCode()){ + if(oldDelete.timestamp > newDelete.timestamp) { + return DeleteCompare.INCLUDE_OLD_NEXT_OLD; + } else if(oldDelete.timestamp < newDelete.timestamp) { + return DeleteCompare.INCLUDE_NEW_NEXT_NEW; + } else { + return DeleteCompare.INCLUDE_OLD_NEXT_BOTH; + } + } + if(oldDelete.timestamp < newDelete.timestamp) { + return DeleteCompare.INCLUDE_NEW_NEXT_BOTH; + } + return DeleteCompare.INCLUDE_OLD_NEXT_BOTH; + } + + // old delete is more specific than the new delete. + // if the olddelete is newer then the newdelete, we have to + // keep it + if(oldDelete.type < newDelete.type) { + if(oldDelete.timestamp > newDelete.timestamp) { + return DeleteCompare.INCLUDE_OLD_NEXT_OLD; + } else if(oldDelete.timestamp < newDelete.timestamp) { + return DeleteCompare.NEXT_OLD; + } else { + return DeleteCompare.NEXT_OLD; + } + } + + // new delete is more specific than the old delete. + if(oldDelete.type > newDelete.type) { + if(oldDelete.timestamp > newDelete.timestamp) { + return DeleteCompare.NEXT_NEW; + } else if(oldDelete.timestamp < newDelete.timestamp) { + return DeleteCompare.INCLUDE_NEW_NEXT_NEW; + } else { + return DeleteCompare.NEXT_NEW; + } + } + + // Should never reach, + // throw exception for assertion? + throw new RuntimeException("GetDeleteTracker:compareDelete reached terminal state"); + } + + /** + * Internal class used to store the necessary information for a Delete. + *

              + * Rather than reparsing the KeyValue, or copying fields, this class points + * to the underlying KeyValue buffer with pointers to the qualifier and fields + * for type and timestamp. No parsing work is done in DeleteTracker now. + *

              + * Fields are public because they are accessed often, directly, and only + * within this class. + */ + protected class Delete { + byte [] buffer; + int qualifierOffset; + int qualifierLength; + byte type; + long timestamp; + /** + * Constructor + * @param buffer + * @param qualifierOffset + * @param qualifierLength + * @param type + * @param timestamp + */ + public Delete(byte [] buffer, int qualifierOffset, int qualifierLength, + byte type, long timestamp) { + this.buffer = buffer; + this.qualifierOffset = qualifierOffset; + this.qualifierLength = qualifierLength; + this.type = type; + this.timestamp = timestamp; + } + } +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java deleted file mode 100644 index 84f42a03a03..00000000000 --- a/src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java +++ /dev/null @@ -1,214 +0,0 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.NavigableSet; -import java.util.regex.Pattern; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.ColumnNameParseException; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Abstract base class that implements the InternalScanner. - */ -public abstract class HAbstractScanner implements InternalScanner { - final Log LOG = LogFactory.getLog(this.getClass().getName()); - - // Pattern to determine if a column key is a regex - static final Pattern isRegexPattern = - Pattern.compile("^.*[\\\\+|^&*$\\[\\]\\}{)(]+.*$"); - - /** The kind of match we are doing on a column: */ - private static enum MATCH_TYPE { - /** Just check the column family name */ - FAMILY_ONLY, - /** Column family + matches regex */ - REGEX, - /** Literal matching */ - SIMPLE - } - - private final List matchers = new ArrayList(); - - // True when scanning is done - protected volatile boolean scannerClosed = false; - - // The timestamp to match entries against - protected final long timestamp; - - private boolean wildcardMatch = false; - private boolean multipleMatchers = false; - - /** Constructor for abstract base class */ - protected HAbstractScanner(final long timestamp, - final NavigableSet columns) - throws IOException { - this.timestamp = timestamp; - for (byte [] column: columns) { - ColumnMatcher matcher = new ColumnMatcher(column); - this.wildcardMatch = matcher.isWildCardMatch(); - matchers.add(matcher); - this.multipleMatchers = !matchers.isEmpty(); - } - } - - /** - * For a particular column, find all the matchers defined for the column. - * Compare the column family and column key using the matchers. The first one - * that matches returns true. If no matchers are successful, return false. - * - * @param kv KeyValue to test - * @return true if any of the matchers for the column match the column family - * and the column key. - * - * @throws IOException - */ - protected boolean columnMatch(final KeyValue kv) - throws IOException { - if (matchers == null) { - return false; - } - for(int m = 0; m < this.matchers.size(); m++) { - if (this.matchers.get(m).matches(kv)) { - return true; - } - } - return false; - } - - public boolean isWildcardScanner() { - return this.wildcardMatch; - } - - public boolean isMultipleMatchScanner() { - return this.multipleMatchers; - } - - public abstract boolean next(List results) - throws IOException; - - /** - * This class provides column matching functions that are more sophisticated - * than a simple string compare. There are three types of matching: - *

                - *
              1. Match on the column family name only
              2. - *
              3. Match on the column family + column key regex
              4. - *
              5. Simple match: compare column family + column key literally
              6. - *
            - */ - private static class ColumnMatcher { - private boolean wildCardmatch; - private MATCH_TYPE matchType; - private byte [] family; - private Pattern columnMatcher; - // Column without delimiter so easy compare to KeyValue column - private byte [] col; - private int familylength = 0; - - ColumnMatcher(final byte [] col) throws IOException { - byte [][] parse = parseColumn(col); - // Make up column without delimiter - byte [] columnWithoutDelimiter = - new byte [parse[0].length + parse[1].length]; - System.arraycopy(parse[0], 0, columnWithoutDelimiter, 0, parse[0].length); - System.arraycopy(parse[1], 0, columnWithoutDelimiter, parse[0].length, - parse[1].length); - // First position has family. Second has qualifier. - byte [] qualifier = parse[1]; - try { - if (qualifier == null || qualifier.length == 0) { - this.matchType = MATCH_TYPE.FAMILY_ONLY; - this.family = parse[0]; - this.wildCardmatch = true; - } else if (isRegexPattern.matcher(Bytes.toString(qualifier)).matches()) { - this.matchType = MATCH_TYPE.REGEX; - this.columnMatcher = - Pattern.compile(Bytes.toString(columnWithoutDelimiter)); - this.wildCardmatch = true; - } else { - this.matchType = MATCH_TYPE.SIMPLE; - this.col = columnWithoutDelimiter; - this.familylength = parse[0].length; - this.wildCardmatch = false; - } - } catch(Exception e) { - throw new IOException("Column: " + Bytes.toString(col) + ": " + - e.getMessage()); - } - } - - /** - * @param kv - * @return - * @throws IOException - */ - boolean matches(final KeyValue kv) throws IOException { - if (this.matchType == MATCH_TYPE.SIMPLE) { - return kv.matchingColumnNoDelimiter(this.col, this.familylength); - } else if(this.matchType == MATCH_TYPE.FAMILY_ONLY) { - return kv.matchingFamily(this.family); - } else if (this.matchType == MATCH_TYPE.REGEX) { - // Pass a column without the delimiter since thats whats we're - // expected to match. - int o = kv.getColumnOffset(); - int l = kv.getColumnLength(o); - String columnMinusQualifier = Bytes.toString(kv.getBuffer(), o, l); - return this.columnMatcher.matcher(columnMinusQualifier).matches(); - } else { - throw new IOException("Invalid match type: " + this.matchType); - } - } - - boolean isWildCardMatch() { - return this.wildCardmatch; - } - - /** - * @param c Column name - * @return Return array of size two whose first element has the family - * prefix of passed column c and whose second element is the - * column qualifier. - * @throws ColumnNameParseException - */ - public static byte [][] parseColumn(final byte [] c) - throws ColumnNameParseException { - final byte [][] result = new byte [2][]; - // TODO: Change this so don't do parse but instead use the comparator - // inside in KeyValue which just looks at column family. - final int index = KeyValue.getFamilyDelimiterIndex(c, 0, c.length); - if (index == -1) { - throw new ColumnNameParseException("Impossible column name: " + c); - } - result[0] = new byte [index]; - System.arraycopy(c, 0, result[0], 0, index); - final int len = c.length - (index + 1); - result[1] = new byte[len]; - System.arraycopy(c, index + 1 /*Skip delimiter*/, result[1], 0, - len); - return result; - } - } -} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HLog.java b/src/java/org/apache/hadoop/hbase/regionserver/HLog.java index f514e161b5e..90d8607f738 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HLog.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HLog.java @@ -100,7 +100,7 @@ import org.apache.hadoop.io.compress.DefaultCodec; public class HLog implements HConstants, Syncable { static final Log LOG = LogFactory.getLog(HLog.class); private static final String HLOG_DATFILE = "hlog.dat."; - static final byte [] METACOLUMN = Bytes.toBytes("METACOLUMN:"); + static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY"); static final byte [] METAROW = Bytes.toBytes("METAROW"); private final FileSystem fs; private final Path dir; @@ -701,8 +701,8 @@ public class HLog implements HConstants, Syncable { } private KeyValue completeCacheFlushLogEdit() { - return new KeyValue(METAROW, METACOLUMN, System.currentTimeMillis(), - COMPLETE_CACHE_FLUSH); + return new KeyValue(METAROW, METAFAMILY, null, + System.currentTimeMillis(), COMPLETE_CACHE_FLUSH); } /** @@ -716,11 +716,11 @@ public class HLog implements HConstants, Syncable { } /** - * @param column + * @param family * @return true if the column is a meta column */ - public static boolean isMetaColumn(byte [] column) { - return Bytes.equals(METACOLUMN, column); + public static boolean isMetaFamily(byte [] family) { + return Bytes.equals(METAFAMILY, family); } /** @@ -870,6 +870,7 @@ public class HLog implements HConstants, Syncable { Executors.newFixedThreadPool(DEFAULT_NUMBER_LOG_WRITER_THREAD); for (final byte[] key : logEntries.keySet()) { Thread thread = new Thread(Bytes.toString(key)) { + @Override public void run() { LinkedList entries = logEntries.get(key); LOG.debug("Thread got " + entries.size() + " to process"); diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HLogKey.java b/src/java/org/apache/hadoop/hbase/regionserver/HLogKey.java index 6a246383d4e..99f5bf89fdb 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HLogKey.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HLogKey.java @@ -87,6 +87,9 @@ public class HLogKey implements WritableComparable, HeapSize { return logSeqNum; } + /** + * @return the write time + */ public long getWriteTime() { return this.writeTime; } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java index d3fbb81e95c..d4a82e5ded6 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -1,5 +1,5 @@ /** - * Copyright 2008 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -22,12 +22,9 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.NavigableSet; -import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; @@ -36,7 +33,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.regex.Pattern; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -44,7 +40,6 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DroppedSnapshotException; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -54,19 +49,16 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionHistorian; -import org.apache.hadoop.hbase.ValueOverMaxLengthException; -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.HbaseMapWritable; -import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.Reference.Range; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.StringUtils; @@ -106,14 +98,15 @@ import org.apache.hadoop.util.StringUtils; * regionName is a unique identifier for this HRegion. (startKey, endKey] * defines the keyspace for this HRegion. */ -public class HRegion implements HConstants { +public class HRegion implements HConstants { // , Writable{ static final Log LOG = LogFactory.getLog(HRegion.class); static final String SPLITDIR = "splits"; static final String MERGEDIR = "merges"; final AtomicBoolean closed = new AtomicBoolean(false); - /* Closing can take some time; use the closing flag if there is stuff we don't want - * to do while in closing state; e.g. like offer this region up to the master as a region - * to close if the carrying regionserver is overloaded. Once set, it is never cleared. + /* Closing can take some time; use the closing flag if there is stuff we don't + * want to do while in closing state; e.g. like offer this region up to the + * master as a region to close if the carrying regionserver is overloaded. + * Once set, it is never cleared. */ private final AtomicBoolean closing = new AtomicBoolean(false); private final RegionHistorian historian; @@ -126,6 +119,13 @@ public class HRegion implements HConstants { new ConcurrentHashMap(); protected final Map stores = new ConcurrentSkipListMap(KeyValue.FAMILY_COMPARATOR); + + //These variable are just used for getting data out of the region, to test on + //client side + // private int numStores = 0; + // private int [] storeSize = null; + // private byte [] name = null; + final AtomicLong memcacheSize = new AtomicLong(0); // This is the table subdirectory. @@ -137,7 +137,6 @@ public class HRegion implements HConstants { final Path regiondir; private final Path regionCompactionDir; KeyValue.KVComparator comparator; - private KeyValue.KVComparator comparatorIgnoreTimestamp; /* * Set this when scheduling compaction if want the next compaction to be a @@ -209,6 +208,24 @@ public class HRegion implements HConstants { public final static byte [] REGIONINFO_FILE_BYTES = Bytes.toBytes(REGIONINFO_FILE); + /** + * Should only be used for testing purposes + */ + public HRegion(){ + this.basedir = null; + this.blockingMemcacheSize = 0; + this.conf = null; + this.flushListener = null; + this.fs = null; + this.historian = null; + this.memcacheFlushSize = 0; + this.log = null; + this.regionCompactionDir = null; + this.regiondir = null; + this.regionInfo = null; + this.threadWakeFrequency = 0L; + } + /** * HRegion constructor. * @@ -234,8 +251,6 @@ public class HRegion implements HConstants { HRegionInfo regionInfo, FlushRequester flushListener) { this.basedir = basedir; this.comparator = regionInfo.getComparator(); - this.comparatorIgnoreTimestamp = - this.comparator.getComparatorIgnoringTimestamps(); this.log = log; this.fs = fs; this.conf = conf; @@ -965,197 +980,6 @@ public class HRegion implements HConstants { ////////////////////////////////////////////////////////////////////////////// // get() methods for client use. ////////////////////////////////////////////////////////////////////////////// - - /** - * Fetch multiple versions of a single data item, with timestamp. - * - * @param row - * @param column - * @param ts - * @param nv - * @return Results or null if none. - * @throws IOException - */ - public List get(final byte[] row, final byte[] column, final long ts, - final int nv) - throws IOException { - long timestamp = ts == -1? HConstants.LATEST_TIMESTAMP : ts; - int numVersions = nv == -1? 1 : nv; - splitsAndClosesLock.readLock().lock(); - try { - if (this.closed.get()) { - throw new IOException("Region " + this + " closed"); - } - // Make sure this is a valid row and valid column - checkRow(row); - checkColumn(column); - // Don't need a row lock for a simple get - List result = getStore(column). - get(KeyValue.createFirstOnRow(row, column, timestamp), numVersions); - // Guarantee that we return null instead of a zero-length array, - // if there are no results to return. - return (result == null || result.isEmpty())? null : result; - } finally { - splitsAndClosesLock.readLock().unlock(); - } - } - - /** - * Data structure with a counter that is accessible rather than create a - * new Integer every time we want to up the counter. Initializes at count 1. - */ - static class Counter { - int counter = 1; - } - - /* - * Check to see if we've not gone over threshold for this particular - * column. - * @param kv - * @param versions - * @param versionsCount - * @return True if its ok to add current value. - */ - static boolean okToAddResult(final KeyValue kv, final int versions, - final Map versionsCount) { - if (versionsCount == null) { - return true; - } - if (versionsCount.containsKey(kv)) { - if (versionsCount.get(kv).counter < versions) { - return true; - } - } else { - return true; - } - return false; - } - - /* - * Used adding item found to list of results getting. - * @param kv - * @param versionsCount - * @param results - */ - static void addResult(final KeyValue kv, - final Map versionsCount, - final List results) { - // Don't add if already present; i.e. ignore second entry. - if (results.contains(kv)) return; - results.add(kv); - if (versionsCount == null) { - return; - } - if (!versionsCount.containsKey(kv)) { - versionsCount.put(kv, new HRegion.Counter()); - } else { - versionsCount.get(kv).counter++; - } - } - - /* - * @param versions Number of versions to get. - * @param versionsCount May be null. - * @param columns Columns we want to fetch. - * @return True if has enough versions. - */ - static boolean hasEnoughVersions(final int versions, - final Map versionsCount, - final Set columns) { - if (columns == null || versionsCount == null) { - // Wants all columns so just keep going - return false; - } - if (columns.size() > versionsCount.size()) { - return false; - } - if (versions == 1) { - return true; - } - // Need to look at each to make sure at least versions. - for (Map.Entry e: versionsCount.entrySet()) { - if (e.getValue().counter < versions) { - return false; - } - } - return true; - } - - /** - * Fetch all the columns for the indicated row at a specified timestamp. - * Returns a HbaseMapWritable that maps column names to values. - * - * We should eventually use Bloom filters here, to reduce running time. If - * the database has many column families and is very sparse, then we could be - * checking many files needlessly. A small Bloom for each row would help us - * determine which column groups are useful for that row. That would let us - * avoid a bunch of disk activity. - * - * @param row - * @param columns Array of columns you'd like to retrieve. When null, get all. - * @param ts - * @param numVersions number of versions to retrieve - * @param lockid - * @return HbaseMapWritable values - * @throws IOException - */ - public HbaseMapWritable getFull(final byte [] row, - final NavigableSet columns, final long ts, - final int numVersions, final Integer lockid) - throws IOException { - // Check columns passed - if (columns != null) { - for (byte [] column: columns) { - checkColumn(column); - } - } - List keyvalues = new ArrayList(); - Map versionCounter = - new TreeMap(this.comparatorIgnoreTimestamp); - Integer lid = getLock(lockid,row); - HashSet storeSet = new HashSet(); - try { - // Get the concerned columns or all of them - if (columns != null) { - for (byte[] bs : columns) { - Store store = stores.get(bs); - if (store != null) { - storeSet.add(store); - } - } - } else { - storeSet.addAll(stores.values()); - } - long timestamp = - (ts == HConstants.LATEST_TIMESTAMP)? System.currentTimeMillis(): ts; - KeyValue key = KeyValue.createFirstOnRow(row, timestamp); - // For each column name that is just a column family, open the store - // related to it and fetch everything for that row. HBASE-631 - // Also remove each store from storeSet so that these stores - // won't be opened for no reason. HBASE-783 - if (columns != null) { - for (byte [] bs : columns) { - // TODO: Fix so we use comparator in KeyValue that looks at - // column family portion only. - if (KeyValue.getFamilyDelimiterIndex(bs, 0, bs.length) == (bs.length - 1)) { - Store store = stores.get(bs); - store.getFull(key, null, null, numVersions, versionCounter, - keyvalues, timestamp); - storeSet.remove(store); - } - } - } - for (Store targetStore: storeSet) { - targetStore.getFull(key, columns, null, numVersions, versionCounter, - keyvalues, timestamp); - } - - return Cell.createCells(keyvalues); - } 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 @@ -1165,9 +989,9 @@ public class HRegion implements HConstants { * @return map of values * @throws IOException */ - RowResult getClosestRowBefore(final byte [] row) + Result getClosestRowBefore(final byte [] row) throws IOException{ - return getClosestRowBefore(row, HConstants.COLUMN_FAMILY); + return getClosestRowBefore(row, HConstants.CATALOG_FAMILY); } /** @@ -1176,12 +1000,13 @@ public class HRegion implements HConstants { * ts. * * @param row row key + * @param family * @param columnFamily Must include the column family delimiter character. * @return map of values * @throws IOException */ - public RowResult getClosestRowBefore(final byte [] row, - final byte [] columnFamily) + public Result getClosestRowBefore(final byte [] row, + final byte [] family) throws IOException{ // look across all the HStores for this region and determine what the // closest key is across all column families, since the data may be sparse @@ -1189,7 +1014,7 @@ public class HRegion implements HConstants { checkRow(row); splitsAndClosesLock.readLock().lock(); try { - Store store = getStore(columnFamily); + Store store = getStore(family); KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP); // get the closest key. (HStore.getRowKeyAtOrBefore can return null) key = store.getRowKeyAtOrBefore(kv); @@ -1202,52 +1027,45 @@ public class HRegion implements HConstants { if (!this.comparator.matchingRows(kv, key)) { kv = new KeyValue(key.getRow(), HConstants.LATEST_TIMESTAMP); } - store.getFull(kv, null, null, 1, null, results, System.currentTimeMillis()); - // Convert to RowResult. TODO: Remove need to do this. - return RowResult.createRowResult(results); + Get get = new Get(key.getRow()); + store.get(get, null, results); + + return new Result(results); } finally { splitsAndClosesLock.readLock().unlock(); } } + //TODO /** * Return an iterator that scans over the HRegion, returning the indicated - * columns for only the rows that match the data filter. This Iterator must - * be closed by the caller. + * columns and rows specified by the {@link Scan}. + *

            + * This Iterator must be closed by the caller. * - * @param cols columns to scan. If column name is a column family, all - * columns of the specified column family are returned. Its also possible - * to pass a regex in the column qualifier. A column qualifier is judged to - * be a regex if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param firstRow row which is the starting point of the scan - * @param timestamp only return rows whose timestamp is <= this value - * @param filter row filter + * @param scan configured {@link Scan} * @return InternalScanner * @throws IOException */ - public InternalScanner getScanner(byte[][] cols, byte [] firstRow, - long timestamp, RowFilterInterface filter) + public InternalScanner getScanner(Scan scan) throws IOException { newScannerLock.readLock().lock(); try { if (this.closed.get()) { throw new IOException("Region " + this + " closed"); } - HashSet storeSet = new HashSet(); - NavigableSet columns = - new TreeSet(Bytes.BYTES_COMPARATOR); - // Below we make up set of stores we want scanners on and we fill out the - // list of columns. - for (int i = 0; i < cols.length; i++) { - columns.add(cols[i]); - Store s = stores.get(cols[i]); - if (s != null) { - storeSet.add(s); + // Verify families are all valid + if(scan.hasFamilies()) { + for(byte [] family : scan.getFamilyMap().keySet()) { + checkFamily(family); + } + } else { // Adding all families to scanner + for(byte[] family: regionInfo.getTableDesc().getFamiliesKeys()){ + scan.addFamily(family); } } - return new HScanner(columns, firstRow, timestamp, - storeSet.toArray(new Store [storeSet.size()]), filter); + return new RegionScanner(scan); + } finally { newScannerLock.readLock().unlock(); } @@ -1256,44 +1074,136 @@ public class HRegion implements HConstants { ////////////////////////////////////////////////////////////////////////////// // set() methods for client use. ////////////////////////////////////////////////////////////////////////////// - /** - * @param b - * @throws IOException - */ - public void batchUpdate(BatchUpdate b) throws IOException { - this.batchUpdate(b, null, true); - } - - /** - * @param b + * @param delete + * @param lockid * @param writeToWAL * @throws IOException */ - public void batchUpdate(BatchUpdate b, boolean writeToWAL) throws IOException { - this.batchUpdate(b, null, writeToWAL); - } - - - /** - * @param b - * @param lockid - * @throws IOException - */ - public void batchUpdate(BatchUpdate b, Integer lockid) throws IOException { - this.batchUpdate(b, lockid, true); - } - - /** - * @param b - * @param lockid - * @param writeToWAL if true, then we write this update to the log - * @throws IOException - */ - public void batchUpdate(BatchUpdate b, Integer lockid, boolean writeToWAL) + public void delete(Delete delete, Integer lockid, boolean writeToWAL) throws IOException { checkReadOnly(); - validateValuesLength(b); + checkResources(); + splitsAndClosesLock.readLock().lock(); + Integer lid = null; + try { + byte [] row = delete.getRow(); + // If we did not pass an existing row lock, obtain a new one + lid = getLock(lockid, row); + + //Check to see if this is a deleteRow insert + if(delete.getFamilyMap().isEmpty()){ + for(byte [] family : regionInfo.getTableDesc().getFamiliesKeys()){ + delete.deleteFamily(family); + } + } else { + for(byte [] family : delete.getFamilyMap().keySet()) { + if(family == null) { + throw new NoSuchColumnFamilyException("Empty family is invalid"); + } + checkFamily(family); + } + } + + for(Map.Entry> e: delete.getFamilyMap().entrySet()) { + byte [] family = e.getKey(); + delete(family, e.getValue(), writeToWAL); + } + } finally { + if(lockid == null) releaseRowLock(lid); + splitsAndClosesLock.readLock().unlock(); + } + } + + + /** + * @param family + * @param kvs + * @param writeToWAL + * @throws IOException + */ + public void delete(byte [] family, List kvs, boolean writeToWAL) + throws IOException { + long now = System.currentTimeMillis(); + boolean flush = false; + this.updatesLock.readLock().lock(); + try { + if (writeToWAL) { + this.log.append(regionInfo.getRegionName(), + regionInfo.getTableDesc().getName(), kvs, + (regionInfo.isMetaRegion() || regionInfo.isRootRegion()), now); + } + long size = 0; + Store store = getStore(family); + for (KeyValue kv: kvs) { + // Check if time is LATEST, change to time of most recent addition if so + // This is expensive. + if (kv.isLatestTimestamp() && kv.isDeleteType()) { + List result = new ArrayList(1); + Get g = new Get(kv.getRow()); + NavigableSet qualifiers = + new TreeSet(Bytes.BYTES_COMPARATOR); + qualifiers.add(kv.getQualifier()); + get(store, g, qualifiers, result); + if (result.isEmpty()) { + // Nothing to delete + continue; + } + if (result.size() > 1) { + throw new RuntimeException("Unexpected size: " + result.size()); + } + KeyValue getkv = result.get(0); + Bytes.putBytes(kv.getBuffer(), kv.getTimestampOffset(), + getkv.getBuffer(), getkv.getTimestampOffset(), Bytes.SIZEOF_LONG); + } + size = this.memcacheSize.addAndGet(store.delete(kv)); + } + flush = isFlushSize(size); + } finally { + this.updatesLock.readLock().unlock(); + } + if (flush) { + // Request a cache flush. Do it outside update lock. + requestFlush(); + } + } + + /** + * @param put + * @throws IOException + */ + public void put(Put put) throws IOException { + this.put(put, null, true); + } + + /** + * @param put + * @param writeToWAL + * @throws IOException + */ + public void put(Put put, boolean writeToWAL) throws IOException { + this.put(put, null, writeToWAL); + } + + /** + * @param put + * @param lockid + * @throws IOException + */ + public void put(Put put, Integer lockid) throws IOException { + this.put(put, lockid, true); + } + + /** + * @param put + * @param lockid + * @param writeToWAL + * @throws IOException + */ + public void put(Put put, Integer lockid, boolean writeToWAL) + throws IOException { + checkReadOnly(); +// validateValuesLength(put); // 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 @@ -1307,49 +1217,18 @@ public class HRegion implements HConstants { // #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 = b.getRow(); + byte [] row = put.getRow(); // If we did not pass an existing row lock, obtain a new one Integer lid = getLock(lockid, row); - long now = System.currentTimeMillis(); - long commitTime = b.getTimestamp() == LATEST_TIMESTAMP? - now: b.getTimestamp(); - Set latestTimestampDeletes = null; - List edits = new ArrayList(); + byte [] now = Bytes.toBytes(System.currentTimeMillis()); try { - for (BatchOperation op: b) { - byte [] column = op.getColumn(); - checkColumn(column); - KeyValue kv = null; - if (op.isPut()) { - kv = new KeyValue(row, column, commitTime, op.getValue()); - } else { - // Its a delete. - if (b.getTimestamp() == LATEST_TIMESTAMP) { - // Save off these deletes of the most recent thing added on the - // family. - if (latestTimestampDeletes == null) { - latestTimestampDeletes = - new TreeSet(Bytes.BYTES_RAWCOMPARATOR); - } - latestTimestampDeletes.add(op.getColumn()); - continue; - } - // Its an explicit timestamp delete - kv = new KeyValue(row, column, commitTime, KeyValue.Type.Delete, - HConstants.EMPTY_BYTE_ARRAY); - } - edits.add(kv); - } - if (!edits.isEmpty()) { - update(edits, writeToWAL, now); - } - if (latestTimestampDeletes != null && - !latestTimestampDeletes.isEmpty()) { - // We have some LATEST_TIMESTAMP deletes to run. Can't do them inline - // as edits. Need to do individually after figuring which is latest - // timestamp to delete. - for (byte [] column: latestTimestampDeletes) { - deleteMultiple(row, column, LATEST_TIMESTAMP, 1, now); + for(Map.Entry> entry : + put.getFamilyMap().entrySet()) { + byte [] family = entry.getKey(); + checkFamily(family); + List puts = entry.getValue(); + if(updateKeys(puts, now)) { + put(family, puts, writeToWAL); } } } finally { @@ -1360,127 +1239,119 @@ public class HRegion implements HConstants { } } + + //TODO, Think that gets/puts and deletes should be refactored a bit so that + //the getting of the lock happens before, so that you would just pass it into + //the methods. So in the case of checkAndPut you could just do lockRow, + //get, put, unlockRow or something /** - * Performs an atomic check and save operation. Checks if - * the specified expected values have changed, and if not - * applies the update. * - * @param b the update to apply - * @param expectedValues the expected values to check - * @param lockid - * @param writeToWAL whether or not to write to the write ahead log - * @return true if update was applied + * @param row + * @param family + * @param qualifier + * @param expectedValue + * @param put + * @param lockId + * @param writeToWAL * @throws IOException + * @return true if the new put was execute, false otherwise */ - public boolean checkAndSave(BatchUpdate b, - HbaseMapWritable expectedValues, Integer lockid, - boolean writeToWAL) - throws IOException { - // This is basically a copy of batchUpdate with the atomic check and save - // added in. So you should read this method with batchUpdate. I will - // comment the areas that I have changed where I have not changed, you - // should read the comments from the batchUpdate method - boolean success = true; + public boolean checkAndPut(byte [] row, byte [] family, byte [] qualifier, + byte [] expectedValue, Put put, Integer lockId, boolean writeToWAL) + throws IOException{ checkReadOnly(); - validateValuesLength(b); + //TODO, add check for value length or maybe even better move this to the + //client if this becomes a global setting checkResources(); splitsAndClosesLock.readLock().lock(); try { - byte[] row = b.getRow(); - long now = System.currentTimeMillis(); - Integer lid = getLock(lockid,row); + Get get = new Get(row, put.getRowLock()); + checkFamily(family); + get.addColumn(family, qualifier); + + byte [] now = Bytes.toBytes(System.currentTimeMillis()); + + // Lock row + Integer lid = getLock(lockId, get.getRow()); + List result = new ArrayList(); try { - NavigableSet keySet = - new TreeSet(Bytes.BYTES_COMPARATOR); - keySet.addAll(expectedValues.keySet()); - Map actualValues = getFull(row, keySet, - HConstants.LATEST_TIMESTAMP, 1,lid); - for (byte[] key : keySet) { - // If test fails exit - Cell cell = actualValues.get(key); - byte[] actualValue = new byte[] {}; - if (cell != null) - actualValue = cell.getValue(); - if(!Bytes.equals(actualValue, - expectedValues.get(key))) { - success = false; - break; - } - } - if (success) { - long commitTime = (b.getTimestamp() == LATEST_TIMESTAMP)? - now: b.getTimestamp(); - Set latestTimestampDeletes = null; - List edits = new ArrayList(); - for (BatchOperation op: b) { - byte [] column = op.getColumn(); - KeyValue kv = null; - if (op.isPut()) { - kv = new KeyValue(row, column, commitTime, op.getValue()); - } else { - // Its a delete. - if (b.getTimestamp() == LATEST_TIMESTAMP) { - // Save off these deletes of the most recent thing added on - // the family. - if (latestTimestampDeletes == null) { - latestTimestampDeletes = - new TreeSet(Bytes.BYTES_RAWCOMPARATOR); - } - latestTimestampDeletes.add(op.getColumn()); - } else { - // Its an explicit timestamp delete - kv = new KeyValue(row, column, commitTime, - KeyValue.Type.Delete, HConstants.EMPTY_BYTE_ARRAY); - } - } - edits.add(kv); - } - if (!edits.isEmpty()) { - update(edits, writeToWAL, now); - } - if (latestTimestampDeletes != null && - !latestTimestampDeletes.isEmpty()) { - // We have some LATEST_TIMESTAMP deletes to run. Can't do them inline - // as edits. Need to do individually after figuring which is latest - // timestamp to delete. - for (byte [] column: latestTimestampDeletes) { - deleteMultiple(row, column, LATEST_TIMESTAMP, 1, now); - } - } + //Getting data + for(Map.Entry> entry: + get.getFamilyMap().entrySet()) { + get(this.stores.get(entry.getKey()), get, entry.getValue(), result); } + boolean matches = false; + if (result.size() == 0 && expectedValue.length == 0) { + matches = true; + } else if(result.size() == 1) { + //Compare the expected value with the actual value + byte [] actualValue = result.get(0).getValue(); + matches = Bytes.equals(expectedValue, actualValue); + } + //If matches put the new put + if(matches) { + for(Map.Entry> entry : + put.getFamilyMap().entrySet()) { + byte [] fam = entry.getKey(); + checkFamily(fam); + List puts = entry.getValue(); + if(updateKeys(puts, now)) { + put(fam, puts, writeToWAL); + } + } + return true; + } + return false; } finally { - if(lockid == null) releaseRowLock(lid); + if(lockId == null) releaseRowLock(lid); } } finally { splitsAndClosesLock.readLock().unlock(); - } - return success; + } } - - /* - * Utility method to verify values length - * @param batchUpdate The update to verify - * @throws IOException Thrown if a value is too long + + + /** + * Checks if any stamps are > now. If so, sets them to now. + *

            + * This acts to be prevent users from inserting future stamps as well as + * to replace LATEST_TIMESTAMP with now. + * @param keys + * @param now + * @return */ - private void validateValuesLength(BatchUpdate batchUpdate) - throws IOException { - for (Iterator iter = - batchUpdate.iterator(); iter.hasNext();) { - BatchOperation operation = iter.next(); - if (operation.getValue() != null) { - HColumnDescriptor fam = this.regionInfo.getTableDesc(). - getFamily(operation.getColumn()); - if (fam != null) { - int maxLength = fam.getMaxValueLength(); - if (operation.getValue().length > maxLength) { - throw new ValueOverMaxLengthException("Value in column " - + Bytes.toString(operation.getColumn()) + " is too long. " - + operation.getValue().length + " instead of " + maxLength); - } - } - } + private boolean updateKeys(List keys, byte [] now) { + if(keys == null || keys.isEmpty()) { + return false; } + for(KeyValue key : keys) { + key.updateLatestStamp(now); + } + return true; } + + +// /* +// * Utility method to verify values length. +// * @param batchUpdate The update to verify +// * @throws IOException Thrown if a value is too long +// */ +// private void validateValuesLength(Put put) +// throws IOException { +// Map> families = put.getFamilyMap(); +// for(Map.Entry> entry : families.entrySet()) { +// HColumnDescriptor hcd = +// this.regionInfo.getTableDesc().getFamily(entry.getKey()); +// int maxLen = hcd.getMaxValueLength(); +// for(KeyValue kv : entry.getValue()) { +// if(kv.getValueLength() > maxLen) { +// throw new ValueOverMaxLengthException("Value in column " +// + Bytes.toString(kv.getColumn()) + " is too long. " +// + kv.getValueLength() + " > " + maxLen); +// } +// } +// } +// } /* * Check if resources to support an update. @@ -1517,230 +1388,6 @@ public class HRegion implements HConstants { + Thread.currentThread().getName() + "'"); } } - - /** - * Delete all cells of the same age as the passed timestamp or older. - * @param row - * @param column - * @param ts Delete all entries that have this timestamp or older - * @param lockid Row lock - * @throws IOException - */ - public void deleteAll(final byte [] row, final byte [] column, final long ts, - final Integer lockid) - throws IOException { - checkColumn(column); - checkReadOnly(); - Integer lid = getLock(lockid,row); - try { - // Delete ALL versions rather than column family VERSIONS. If we just did - // VERSIONS, then if 2* VERSION cells, subsequent gets would get old stuff. - deleteMultiple(row, column, ts, ALL_VERSIONS, System.currentTimeMillis()); - } finally { - if(lockid == null) releaseRowLock(lid); - } - } - - /** - * Delete all cells of the same age as the passed timestamp or older. - * @param row - * @param ts Delete all entries that have this timestamp or older - * @param lockid Row lock - * @throws IOException - */ - public void deleteAll(final byte [] row, final long ts, final Integer lockid) - throws IOException { - checkReadOnly(); - Integer lid = getLock(lockid, row); - long now = System.currentTimeMillis(); - long time = ts; - if (ts == HConstants.LATEST_TIMESTAMP) { - time = now; - } - KeyValue kv = KeyValue.createFirstOnRow(row, time); - try { - for (Store store : stores.values()) { - List keyvalues = new ArrayList(); - store.getFull(kv, null, null, ALL_VERSIONS, null, keyvalues, time); - List edits = new ArrayList(); - for (KeyValue key: keyvalues) { - // This is UGLY. COPY OF KEY PART OF KeyValue. - edits.add(key.cloneDelete()); - } - update(edits, now); - } - } finally { - if (lockid == null) releaseRowLock(lid); - } - } - - /** - * Delete all cells for a row with matching columns with timestamps - * less than or equal to timestamp. - * - * @param row The row to operate on - * @param columnRegex The column regex - * @param timestamp Timestamp to match - * @param lockid Row lock - * @throws IOException - */ - public void deleteAllByRegex(final byte [] row, final String columnRegex, - final long timestamp, final Integer lockid) throws IOException { - checkReadOnly(); - Pattern columnPattern = Pattern.compile(columnRegex); - Integer lid = getLock(lockid, row); - long now = System.currentTimeMillis(); - KeyValue kv = new KeyValue(row, timestamp); - try { - for (Store store : stores.values()) { - List keyvalues = new ArrayList(); - store.getFull(kv, null, columnPattern, ALL_VERSIONS, null, keyvalues, - now); - List edits = new ArrayList(); - for (KeyValue key: keyvalues) { - edits.add(key.cloneDelete()); - } - update(edits, now); - } - } finally { - if(lockid == null) releaseRowLock(lid); - } - } - - /** - * Delete all cells for a row with matching column family with timestamps - * less than or equal to timestamp. - * - * @param row The row to operate on - * @param family The column family to match - * @param timestamp Timestamp to match - * @param lockid Row lock - * @throws IOException - */ - public void deleteFamily(byte [] row, byte [] family, long timestamp, - final Integer lockid) - throws IOException{ - checkReadOnly(); - Integer lid = getLock(lockid, row); - long now = System.currentTimeMillis(); - try { - // find the HStore for the column family - Store store = getStore(family); - // find all the keys that match our criteria - List keyvalues = new ArrayList(); - store.getFull(new KeyValue(row, timestamp), null, null, ALL_VERSIONS, - null, keyvalues, now); - // delete all the cells - List edits = new ArrayList(); - for (KeyValue kv: keyvalues) { - edits.add(kv.cloneDelete()); - } - update(edits, now); - } finally { - if(lockid == null) releaseRowLock(lid); - } - } - - /** - * Delete all cells for a row with all the matching column families by - * familyRegex with timestamps less than or equal to timestamp. - * - * @param row The row to operate on - * @param familyRegex The column family regex for matching. This regex - * expression just match the family name, it didn't include : - * @param timestamp Timestamp to match - * @param lockid Row lock - * @throws IOException - */ - public void deleteFamilyByRegex(byte [] row, String familyRegex, - final long timestamp, final Integer lockid) - throws IOException { - checkReadOnly(); - // construct the family regex pattern - Pattern familyPattern = Pattern.compile(familyRegex); - Integer lid = getLock(lockid, row); - long now = System.currentTimeMillis(); - KeyValue kv = new KeyValue(row, timestamp); - try { - for(Store store: stores.values()) { - String familyName = Bytes.toString(store.getFamily().getName()); - // check the family name match the family pattern. - if(!(familyPattern.matcher(familyName).matches())) - continue; - - List keyvalues = new ArrayList(); - store.getFull(kv, null, null, ALL_VERSIONS, null, keyvalues, now); - List edits = new ArrayList(); - for (KeyValue k: keyvalues) { - edits.add(k.cloneDelete()); - } - update(edits, now); - } - } finally { - if(lockid == null) releaseRowLock(lid); - } - } - - /* - * Delete one or many cells. - * Used to support {@link #deleteAll(byte [], byte [], long)} and deletion of - * latest cell. - * @param row - * @param column - * @param ts Timestamp to start search on. - * @param versions How many versions to delete. Pass - * {@link HConstants#ALL_VERSIONS} to delete all. - * @param now - * @throws IOException - */ - private void deleteMultiple(final byte [] row, final byte [] column, - final long ts, final int versions, final long now) - throws IOException { - checkReadOnly(); - // We used to have a getKeys method that purportedly only got the keys and - // not the keys and values. We now just do getFull. For memcache values, - // shouldn't matter if we get key and value since it'll be the entry that - // is in memcache. For the keyvalues from storefile, could be saving if - // we only returned key component. TODO. - List keys = get(row, column, ts, versions); - if (keys != null && keys.size() > 0) { - // I think the below edits don't have to be storted. Its deletes. - // hey don't have to go in in exact sorted order (we don't have to worry - // about the meta or root sort comparator here). - List edits = new ArrayList(); - for (KeyValue key: keys) { - edits.add(key.cloneDelete()); - } - update(edits, now); - } - } - - /** - * Tests for the existence of any cells for a given coordinate. - * - * @param row the row - * @param column the column, or null - * @param timestamp the timestamp, or HConstants.LATEST_VERSION for any - * @param lockid the existing lock, or null - * @return true if cells exist for the row, false otherwise - * @throws IOException - */ - public boolean exists(final byte[] row, final byte[] column, - final long timestamp, final Integer lockid) - throws IOException { - checkRow(row); - Integer lid = getLock(lockid, row); - try { - NavigableSet columns = null; - if (column != null) { - columns = new TreeSet(Bytes.BYTES_COMPARATOR); - columns.add(column); - } - return !getFull(row, columns, timestamp, 1, lid).isEmpty(); - } finally { - if (lockid == null) releaseRowLock(lid); - } - } /** * @throws IOException Throws exception if region is in read-only mode. @@ -1758,9 +1405,9 @@ public class HRegion implements HConstants { * @praram now * @throws IOException */ - private void update(final List edits, final long now) + private void put(final byte [] family, final List edits) throws IOException { - this.update(edits, true, now); + this.put(family, edits, true); } /** @@ -1771,9 +1418,8 @@ public class HRegion implements HConstants { * @param now * @throws IOException */ - private void update(final List edits, boolean writeToWAL, - final long now) - throws IOException { + private void put(final byte [] family, final List edits, + boolean writeToWAL) throws IOException { if (edits == null || edits.isEmpty()) { return; } @@ -1781,14 +1427,15 @@ public class HRegion implements HConstants { this.updatesLock.readLock().lock(); try { if (writeToWAL) { + long now = System.currentTimeMillis(); this.log.append(regionInfo.getRegionName(), regionInfo.getTableDesc().getName(), edits, (regionInfo.isMetaRegion() || regionInfo.isRootRegion()), now); } long size = 0; + Store store = getStore(family); for (KeyValue kv: edits) { - // TODO: Fix -- do I have to do a getColumn here? - size = this.memcacheSize.addAndGet(getStore(kv.getColumn()).add(kv)); + size = this.memcacheSize.addAndGet(store.add(kv)); } flush = isFlushSize(size); } finally { @@ -1826,7 +1473,6 @@ public class HRegion implements HConstants { } // Do any reconstruction needed from the log - @SuppressWarnings("unused") protected void doReconstructionLog(Path oldLogFile, long minSeqId, long maxSeqId, Progressable reporter) throws UnsupportedEncodingException, IOException { @@ -1865,23 +1511,6 @@ public class HRegion implements HConstants { Bytes.toString(row) + "'"); } } - - /* - * Make sure this is a valid column for the current table - * @param columnName - * @throws NoSuchColumnFamilyException - */ - private void checkColumn(final byte [] column) - throws NoSuchColumnFamilyException { - if (column == null) { - return; - } - if (!regionInfo.getTableDesc().hasFamily(column)) { - throw new NoSuchColumnFamilyException("Column family on " + - Bytes.toString(column) + " does not exist in region " + this - + " in table " + regionInfo.getTableDesc()); - } - } /** * Obtain a lock on the given row. Blocks until success. @@ -1906,7 +1535,7 @@ public class HRegion implements HConstants { * @throws IOException * @return The id of the held lock. */ - Integer obtainRowLock(final byte [] row) throws IOException { + public Integer obtainRowLock(final byte [] row) throws IOException { checkRow(row); splitsAndClosesLock.readLock().lock(); try { @@ -2018,161 +1647,86 @@ public class HRegion implements HConstants { return this.basedir; } + + //TODO /** - * HScanner is an iterator through a bunch of rows in an HRegion. + * RegionScanner is an iterator through a bunch of rows in an HRegion. + *

            + * It is used to combine scanners from multiple Stores (aka column families). */ - private class HScanner implements InternalScanner { - private InternalScanner[] scanners; - private List [] resultSets; - private RowFilterInterface filter; - - /** Create an HScanner with a handle on many HStores. */ - @SuppressWarnings("unchecked") - HScanner(final NavigableSet columns, byte [] firstRow, - long timestamp, final Store [] stores, final RowFilterInterface filter) - throws IOException { - this.filter = filter; - this.scanners = new InternalScanner[stores.length]; + class RegionScanner implements InternalScanner { + + private KeyValueHeap storeHeap; + private byte [] stopRow; + + RegionScanner(Scan scan) throws IOException { + if(Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) { + this.stopRow = null; + } else { + this.stopRow = scan.getStopRow(); + } + + List scanners = new ArrayList(); try { - for (int i = 0; i < stores.length; i++) { - // Only pass relevant columns to each store - NavigableSet columnSubset = - new TreeSet(Bytes.BYTES_COMPARATOR); - for (byte [] c: columns) { - if (KeyValue.FAMILY_COMPARATOR.compare(stores[i].storeName, c) == 0) { - columnSubset.add(c); - } - } - RowFilterInterface f = filter; - if (f != null) { - // Need to replicate filters. - // At least WhileMatchRowFilter will mess up the scan if only - // one shared across many rows. See HADOOP-2467. - f = WritableUtils.clone(filter, conf); - } - scanners[i] = stores[i].getScanner(timestamp, columnSubset, firstRow, f); + for(Map.Entry> entry : + scan.getFamilyMap().entrySet()) { + Store store = stores.get(entry.getKey()); + scanners.add(store.getScanner(scan, entry.getValue())); } } catch (IOException e) { - for (int i = 0; i < this.scanners.length; i++) { - if (scanners[i] != null) { - closeScanner(i); + for(KeyValueScanner scanner : scanners) { + if(scanner != null) { + close(scanner); } } throw e; } - - // Advance to the first key in each store. - // All results will match the required column-set and scanTime. - this.resultSets = new List[scanners.length]; - for (int i = 0; i < scanners.length; i++) { - resultSets[i] = new ArrayList(); - if(scanners[i] != null && !scanners[i].next(resultSets[i])) { - closeScanner(i); - } - } - + + this.storeHeap = + new KeyValueHeap(scanners.toArray(new KeyValueScanner[0]), comparator); + // As we have now successfully completed initialization, increment the // activeScanner count. activeScannerCount.incrementAndGet(); } + /** + * Get the next row of results from this region. + * @param results list to append results to + * @return true if there are more rows, false if scanner is done + */ public boolean next(List results) throws IOException { - boolean moreToFollow = false; - boolean filtered = false; - do { - // Find the lowest key across all stores. - KeyValue chosen = null; - long chosenTimestamp = -1; - for (int i = 0; i < this.scanners.length; i++) { - if (this.resultSets[i] == null || this.resultSets[i].isEmpty()) { - continue; - } - KeyValue kv = this.resultSets[i].get(0); - if (chosen == null || - (comparator.compareRows(kv, chosen) < 0) || - ((comparator.compareRows(kv, chosen) == 0) && - (kv.getTimestamp() > chosenTimestamp))) { - chosen = kv; - chosenTimestamp = chosen.getTimestamp(); - } - } - - // Store results from each sub-scanner. - if (chosenTimestamp >= 0) { - for (int i = 0; i < scanners.length; i++) { - if (this.resultSets[i] == null || this.resultSets[i].isEmpty()) { - continue; - } - KeyValue kv = this.resultSets[i].get(0); - if (comparator.compareRows(kv, chosen) == 0) { - results.addAll(this.resultSets[i]); - resultSets[i].clear(); - if (!scanners[i].next(resultSets[i])) { - closeScanner(i); - } - } - } - } - - moreToFollow = chosenTimestamp >= 0; - if (results == null || results.size() <= 0) { - // If we got no results, then there is no more to follow. - moreToFollow = false; - } - - filtered = filter == null ? false : filter.filterRow(results); - if (filter != null && filter.filterAllRemaining()) { - moreToFollow = false; - } - - if (moreToFollow) { - if (filter != null) { - filter.rowProcessed(filtered, chosen.getBuffer(), chosen.getRowOffset(), - chosen.getRowLength()); - } - if (filtered) { - results.clear(); - } - } - } while(filtered && moreToFollow); - - // Make sure scanners closed if no more results - if (!moreToFollow) { - for (int i = 0; i < scanners.length; i++) { - if (null != scanners[i]) { - closeScanner(i); - } - } + // This method should probably be reorganized a bit... has gotten messy + KeyValue kv = this.storeHeap.peek(); + if(kv == null) { + return false; } - - return moreToFollow; - } - - /** Shut down a single scanner */ - void closeScanner(int i) { - try { - try { - scanners[i].close(); - } catch (IOException e) { - LOG.warn("Failed closing scanner " + i, e); + byte [] currentRow = kv.getRow(); + // See if we passed stopRow + if(stopRow != null && + comparator.compareRows(stopRow, 0, stopRow.length, + currentRow, 0, currentRow.length) + <= 0){ + return false; + } + this.storeHeap.next(results); + while(true) { + kv = this.storeHeap.peek(); + if(kv == null) { + return false; } - } finally { - scanners[i] = null; - // These data members can be null if exception in constructor - if (resultSets != null) { - resultSets[i] = null; + byte [] row = kv.getRow(); + if(!Bytes.equals(currentRow, row)) { + return true; } + this.storeHeap.next(results); } } public void close() { try { - for(int i = 0; i < scanners.length; i++) { - if(scanners[i] != null) { - closeScanner(i); - } - } + storeHeap.close(); } finally { synchronized (activeScannerCount) { int count = activeScannerCount.decrementAndGet(); @@ -2188,14 +1742,22 @@ public class HRegion implements HConstants { } } } - - public boolean isWildcardScanner() { - throw new UnsupportedOperationException("Unimplemented on HScanner"); + /** + * + * @param scanner to be closed + */ + public void close(KeyValueScanner scanner) { + try { + scanner.close(); + } catch(NullPointerException npe) {} + } + + /** + * @return the current storeHeap + */ + public KeyValueHeap getStoreHeap() { + return this.storeHeap; } - - public boolean isMultipleMatchScanner() { - throw new UnsupportedOperationException("Unimplemented on HScanner"); - } } // Utility methods @@ -2282,9 +1844,9 @@ public class HRegion implements HConstants { Integer lid = meta.obtainRowLock(row); try { List edits = new ArrayList(); - edits.add(new KeyValue(row, COL_REGIONINFO, System.currentTimeMillis(), - Writables.getBytes(r.getRegionInfo()))); - meta.update(edits, System.currentTimeMillis()); + edits.add(new KeyValue(row, CATALOG_FAMILY, REGIONINFO_QUALIFIER, + System.currentTimeMillis(), Writables.getBytes(r.getRegionInfo()))); + meta.put(HConstants.CATALOG_FAMILY, edits); } finally { meta.releaseRowLock(lid); } @@ -2304,8 +1866,9 @@ public class HRegion implements HConstants { public static void removeRegionFromMETA(final HRegionInterface srvr, final byte [] metaRegionName, final byte [] regionName) throws IOException { - srvr.deleteFamily(metaRegionName, regionName, HConstants.COLUMN_FAMILY, - HConstants.LATEST_TIMESTAMP, -1L); + Delete delete = new Delete(regionName); + delete.deleteFamily(HConstants.CATALOG_FAMILY); + srvr.delete(metaRegionName, delete); } /** @@ -2319,14 +1882,18 @@ public class HRegion implements HConstants { public static void offlineRegionInMETA(final HRegionInterface srvr, final byte [] metaRegionName, final HRegionInfo info) throws IOException { - BatchUpdate b = new BatchUpdate(info.getRegionName()); + // Puts and Deletes used to be "atomic" here. We can use row locks if + // we need to keep that property, or we can expand Puts and Deletes to + // allow them to be committed at once. + byte [] row = info.getRegionName(); + Put put = new Put(row); info.setOffline(true); - b.put(COL_REGIONINFO, Writables.getBytes(info)); - b.delete(COL_SERVER); - b.delete(COL_STARTCODE); - // If carrying splits, they'll be in place when we show up on new - // server. - srvr.batchUpdate(metaRegionName, b, -1L); + put.add(CATALOG_FAMILY, REGIONINFO_QUALIFIER, Writables.getBytes(info)); + srvr.put(metaRegionName, put); + Delete del = new Delete(row); + del.deleteColumns(CATALOG_FAMILY, SERVER_QUALIFIER); + del.deleteColumns(CATALOG_FAMILY, STARTCODE_QUALIFIER); + srvr.delete(metaRegionName, del); } /** @@ -2340,12 +1907,10 @@ public class HRegion implements HConstants { public static void cleanRegionInMETA(final HRegionInterface srvr, final byte [] metaRegionName, final HRegionInfo info) throws IOException { - BatchUpdate b = new BatchUpdate(info.getRegionName()); - b.delete(COL_SERVER); - b.delete(COL_STARTCODE); - // If carrying splits, they'll be in place when we show up on new - // server. - srvr.batchUpdate(metaRegionName, b, LATEST_TIMESTAMP); + Delete del = new Delete(info.getRegionName()); + del.deleteColumns(CATALOG_FAMILY, SERVER_QUALIFIER); + del.deleteColumns(CATALOG_FAMILY, STARTCODE_QUALIFIER); + srvr.delete(metaRegionName, del); } /** @@ -2638,67 +2203,127 @@ public class HRegion implements HConstants { } } - public long incrementColumnValue(byte[] row, byte[] column, long amount) + + // + // HBASE-880 + // + /** + * @param get + * @param lockid + * @return result + * @throws IOException + */ + public Result get(final Get get, final Integer lockid) throws IOException { + // Verify families are all valid + if(get.hasFamilies()) { + for(byte [] family : get.familySet()) { + checkFamily(family); + } + } else { // Adding all families to scanner + for(byte[] family: regionInfo.getTableDesc().getFamiliesKeys()){ + get.addFamily(family); + } + } + // Lock row + Integer lid = getLock(lockid, get.getRow()); + List result = new ArrayList(); + try { + for(Map.Entry> entry: + get.getFamilyMap().entrySet()) { + get(this.stores.get(entry.getKey()), get, entry.getValue(), result); + } + } finally { + if(lockid == null) releaseRowLock(lid); + } + return new Result(result); + } + + private void get(final Store store, final Get get, + final NavigableSet qualifiers, List result) + throws IOException { + store.get(get, qualifiers, result); + } + + /** + * + * @param row + * @param family + * @param qualifier + * @param amount + * @return + * @throws IOException + */ + public long incrementColumnValue(byte [] row, byte [] family, + byte [] qualifier, long amount) throws IOException { checkRow(row); - checkColumn(column); + // Lock row Integer lid = obtainRowLock(row); - splitsAndClosesLock.readLock().lock(); + long result = 0L; try { - KeyValue kv = new KeyValue(row, column); - long ts = System.currentTimeMillis(); - byte [] value = null; - - Store store = getStore(column); - - List c; - // Try the memcache first. - store.lock.readLock().lock(); - try { - c = store.memcache.get(kv, 1); - } finally { - store.lock.readLock().unlock(); - } - // Pick the latest value out of List c: - if (c.size() >= 1) { - // Use the memcache timestamp value. - LOG.debug("Overwriting the memcache value for " + Bytes.toString(row) + - "/" + Bytes.toString(column)); - ts = c.get(0).getTimestamp(); - value = c.get(0).getValue(); - } - - if (value == null) { - // Check the store (including disk) for the previous value. - c = store.get(kv, 1); - if (c != null && c.size() == 1) { - LOG.debug("Using HFile previous value for " + Bytes.toString(row) + - "/" + Bytes.toString(column)); - value = c.get(0).getValue(); - } else if (c != null && c.size() > 1) { - throw new DoNotRetryIOException("more than 1 value returned in " + - "incrementColumnValue from Store"); - } - } - - if (value == null) { - // Doesn't exist - LOG.debug("Creating new counter value for " + Bytes.toString(row) + - "/"+ Bytes.toString(column)); - value = Bytes.toBytes(amount); - } else { - if (amount == 0) return Bytes.toLong(value); - value = Bytes.incrementBytes(value, amount); - } - - BatchUpdate b = new BatchUpdate(row, ts); - b.put(column, value); - batchUpdate(b, lid, true); - return Bytes.toLong(value); + Store store = stores.get(family); + result = store.incrementColumnValue(row, family, qualifier, amount); } finally { - splitsAndClosesLock.readLock().unlock(); - releaseRowLock(lid); + if(lid == null) { + releaseRowLock(lid); + } + } + return result; + } + + + // + // New HBASE-880 Helpers + // + + private void checkFamily(final byte [] family) + throws NoSuchColumnFamilyException { + if(!regionInfo.getTableDesc().hasFamily(family)) { + throw new NoSuchColumnFamilyException("Column family " + + Bytes.toString(family) + " does not exist in region " + this + + " in table " + regionInfo.getTableDesc()); } } -} + + +// //HBaseAdmin Debugging +// /** +// * @return number of stores in the region +// */ +// public int getNumStores() { +// return this.numStores; +// } +// /** +// * @return the name of the region +// */ +// public byte [] getRegionsName() { +// return this.name; +// } +// /** +// * @return the number of files in every store +// */ +// public int [] getStoresSize() { +// return this.storeSize; +// } +// +// //Writable, used for debugging purposes only +// public void readFields(final DataInput in) +// throws IOException { +// this.name = Bytes.readByteArray(in); +// this.numStores = in.readInt(); +// this.storeSize = new int [numStores]; +// for(int i=0; i ee: r.stores.entrySet()) { Store store = ee.getValue(); storefiles += store.getStorefilesCount(); - try { - storefileIndexSize += store.getStorefilesIndexSize(); - } catch (IOException ex) { - LOG.warn("error getting store file index size for " + store + - ": " + StringUtils.stringifyException(ex)); - } + storefileIndexSize += store.getStorefilesIndexSize(); } } } @@ -1630,7 +1621,7 @@ public class HRegionServer implements HConstants, HRegionInterface, super(Thread.currentThread().getName() + ".regionCloser." + r.toString()); this.r = r; } - + @Override public void run() { try { @@ -1701,46 +1692,9 @@ public class HRegionServer implements HConstants, HRegionInterface, return getRegion(regionName).getRegionInfo(); } - public Cell [] get(final byte [] regionName, final byte [] row, - final byte [] column, final long timestamp, final int numVersions) - throws IOException { - checkOpen(); - requestCount.incrementAndGet(); - try { - List results = - getRegion(regionName).get(row, column, timestamp, numVersions); - return Cell.createSingleCellArray(results); - } catch (Throwable t) { - throw convertThrowableToIOE(cleanup(t)); - } - } - public RowResult getRow(final byte [] regionName, final byte [] row, - final byte [][] columns, final long ts, - final int numVersions, final long lockId) - throws IOException { - checkOpen(); - requestCount.incrementAndGet(); - try { - // convert the columns array into a set so it's easy to check later. - NavigableSet columnSet = null; - if (columns != null) { - columnSet = new TreeSet(Bytes.BYTES_COMPARATOR); - columnSet.addAll(Arrays.asList(columns)); - } - HRegion region = getRegion(regionName); - HbaseMapWritable result = - region.getFull(row, columnSet, ts, numVersions, getLockFromId(lockId)); - if (result == null || result.isEmpty()) - return null; - return new RowResult(row, result); - } catch (Throwable t) { - throw convertThrowableToIOE(cleanup(t)); - } - } - - public RowResult getClosestRowBefore(final byte [] regionName, - final byte [] row, final byte [] columnFamily) + public Result getClosestRowBefore(final byte [] regionName, + final byte [] row, final byte [] family) throws IOException { checkOpen(); requestCount.incrementAndGet(); @@ -1748,49 +1702,41 @@ public class HRegionServer implements HConstants, HRegionInterface, // locate the region we're operating on HRegion region = getRegion(regionName); // ask the region for all the data - RowResult rr = region.getClosestRowBefore(row, columnFamily); - return rr; + + Result r = region.getClosestRowBefore(row, family); + return r; } catch (Throwable t) { throw convertThrowableToIOE(cleanup(t)); } } - - public RowResult next(final long scannerId) throws IOException { - RowResult[] rrs = next(scannerId, 1); - return rrs.length == 0 ? null : rrs[0]; - } - public RowResult [] next(final long scannerId, int nbRows) throws IOException { + /** {@inheritDoc} */ + public Result get(byte [] regionName, Get get) throws IOException { checkOpen(); - List> results = new ArrayList>(); + requestCount.incrementAndGet(); try { - String scannerName = String.valueOf(scannerId); - InternalScanner s = scanners.get(scannerName); - if (s == null) { - throw new UnknownScannerException("Name: " + scannerName); - } - this.leases.renewLease(scannerName); - for (int i = 0; i < nbRows; i++) { - requestCount.incrementAndGet(); - // Collect values to be returned here - List values = new ArrayList(); - while (s.next(values)) { - if (!values.isEmpty()) { - // Row has something in it. Return the value. - results.add(values); - break; - } - } - } - return RowResult.createRowResultArray(results); - } catch (Throwable t) { + HRegion region = getRegion(regionName); + return region.get(get, getLockFromId(get.getLockId())); + } catch(Throwable t) { throw convertThrowableToIOE(cleanup(t)); } } - public void batchUpdate(final byte [] regionName, BatchUpdate b, long lockId) + public boolean exists(byte [] regionName, Get get) throws IOException { + checkOpen(); + requestCount.incrementAndGet(); + try { + HRegion region = getRegion(regionName); + Result r = region.get(get, getLockFromId(get.getLockId())); + return r != null && !r.isEmpty(); + } catch(Throwable t) { + throw convertThrowableToIOE(cleanup(t)); + } + } + + public void put(final byte [] regionName, final Put put) throws IOException { - if (b.getRow() == null) + if (put.getRow() == null) throw new IllegalArgumentException("update has null row"); checkOpen(); @@ -1798,24 +1744,24 @@ public class HRegionServer implements HConstants, HRegionInterface, HRegion region = getRegion(regionName); try { cacheFlusher.reclaimMemcacheMemory(); - region.batchUpdate(b, getLockFromId(b.getRowLock())); + region.put(put, getLockFromId(put.getLockId())); } catch (Throwable t) { throw convertThrowableToIOE(cleanup(t)); } } - public int batchUpdates(final byte[] regionName, final BatchUpdate [] b) + public int put(final byte[] regionName, final Put [] puts) throws IOException { int i = 0; checkOpen(); try { HRegion region = getRegion(regionName); this.cacheFlusher.reclaimMemcacheMemory(); - Integer[] locks = new Integer[b.length]; - for (i = 0; i < b.length; i++) { + Integer[] locks = new Integer[puts.length]; + for (i = 0; i < puts.length; i++) { this.requestCount.incrementAndGet(); - locks[i] = getLockFromId(b[i].getRowLock()); - region.batchUpdate(b[i], locks[i]); + locks[i] = getLockFromId(puts[i].getLockId()); + region.put(puts[i], locks[i]); } } catch(WrongRegionException ex) { return i; @@ -1827,38 +1773,49 @@ public class HRegionServer implements HConstants, HRegionInterface, return -1; } - public boolean checkAndSave(final byte [] regionName, final BatchUpdate b, - final HbaseMapWritable expectedValues) - throws IOException { - if (b.getRow() == null) - throw new IllegalArgumentException("update has null row"); + + /** + * + * @param regionName + * @param row + * @param family + * @param qualifier + * @param value the expected value + * @param put + * @throws IOException + * @return true if the new put was execute, false otherwise + */ + public boolean checkAndPut(final byte[] regionName, final byte [] row, + final byte [] family, final byte [] qualifier, final byte [] value, + final Put put) throws IOException{ + //Getting actual value + Get get = new Get(row); + get.addColumn(family, qualifier); + checkOpen(); this.requestCount.incrementAndGet(); HRegion region = getRegion(regionName); try { cacheFlusher.reclaimMemcacheMemory(); - return region.checkAndSave(b, - expectedValues,getLockFromId(b.getRowLock()), true); + return region.checkAndPut(row, family, qualifier, value, put, + getLockFromId(put.getLockId()), true); } catch (Throwable t) { throw convertThrowableToIOE(cleanup(t)); } } - + // // remote scanner interface // - public long openScanner(byte [] regionName, byte [][] cols, byte [] firstRow, - final long timestamp, final RowFilterInterface filter) + public long openScanner(byte [] regionName, Scan scan) throws IOException { checkOpen(); NullPointerException npe = null; if (regionName == null) { npe = new NullPointerException("regionName is null"); - } else if (cols == null) { - npe = new NullPointerException("columns to scan is null"); - } else if (firstRow == null) { - npe = new NullPointerException("firstRow for scanner is null"); + } else if (scan == null) { + npe = new NullPointerException("scan is null"); } if (npe != null) { throw new IOException("Invalid arguments to openScanner", npe); @@ -1866,8 +1823,7 @@ public class HRegionServer implements HConstants, HRegionInterface, requestCount.incrementAndGet(); try { HRegion r = getRegion(regionName); - InternalScanner s = - r.getScanner(cols, firstRow, timestamp, filter); + InternalScanner s = r.getScanner(scan); long scannerId = addScanner(s); return scannerId; } catch (Throwable t) { @@ -1886,6 +1842,47 @@ public class HRegionServer implements HConstants, HRegionInterface, createLease(scannerName, new ScannerListener(scannerName)); return scannerId; } + + public Result next(final long scannerId) throws IOException { + Result [] res = next(scannerId, 1); + if(res == null || res.length == 0) { + return null; + } + return res[0]; + } + + public Result [] next(final long scannerId, int nbRows) throws IOException { + checkOpen(); + List results = new ArrayList(); + try { + long start = System.currentTimeMillis(); + String scannerName = String.valueOf(scannerId); + InternalScanner s = scanners.get(scannerName); + if (s == null) { + throw new UnknownScannerException("Name: " + scannerName); + } + this.leases.renewLease(scannerName); + for (int i = 0; i < nbRows; i++) { + requestCount.incrementAndGet(); + // Collect values to be returned here + List values = new ArrayList(); + boolean moreRows = s.next(values); + if(!values.isEmpty()) { + results.add(new Result(values)); + } + if(!moreRows) { + break; + } + } + LOG.debug("Result[]next time: " + (System.currentTimeMillis()-start) + " (ms)"); + return results.toArray(new Result[0]); + } catch (Throwable t) { + throw convertThrowableToIOE(cleanup(t)); + } + } + + + public void close(final long scannerId) throws IOException { try { @@ -1937,45 +1934,23 @@ public class HRegionServer implements HConstants, HRegionInterface, // Methods that do the actual work for the remote API // - public void deleteAll(final byte [] regionName, final byte [] row, - final byte [] column, final long timestamp, final long lockId) + public void delete(final byte [] regionName, final Delete delete) throws IOException { - HRegion region = getRegion(regionName); - region.deleteAll(row, column, timestamp, getLockFromId(lockId)); - } - - public void deleteAll(final byte [] regionName, final byte [] row, - final long timestamp, final long lockId) - throws IOException { - HRegion region = getRegion(regionName); - region.deleteAll(row, timestamp, getLockFromId(lockId)); - } - - public void deleteAllByRegex(byte[] regionName, byte[] row, String colRegex, - long timestamp, long lockId) throws IOException { - getRegion(regionName).deleteAllByRegex(row, colRegex, timestamp, - getLockFromId(lockId)); - } - - public void deleteFamily(byte [] regionName, byte [] row, byte [] family, - long timestamp, final long lockId) - throws IOException{ - getRegion(regionName).deleteFamily(row, family, timestamp, - getLockFromId(lockId)); - } - - public void deleteFamilyByRegex(byte[] regionName, byte[] row, String familyRegex, - long timestamp, long lockId) throws IOException { - getRegion(regionName).deleteFamilyByRegex(row, familyRegex, timestamp, - getLockFromId(lockId)); - } - - public boolean exists(byte[] regionName, byte[] row, byte[] column, - long timestamp, long lockId) - throws IOException { - return getRegion(regionName).exists(row, column, timestamp, - getLockFromId(lockId)); + checkOpen(); + try { + boolean writeToWAL = true; + this.cacheFlusher.reclaimMemcacheMemory(); + this.requestCount.incrementAndGet(); + Integer lock = getLockFromId(delete.getLockId()); + HRegion region = getRegion(regionName); + region.delete(delete, lock, writeToWAL); + } catch(WrongRegionException ex) { + } catch (NotServingRegionException ex) { + } catch (Throwable t) { + throw convertThrowableToIOE(cleanup(t)); + } } + public long lockRow(byte [] regionName, byte [] row) throws IOException { @@ -2023,7 +1998,7 @@ public class HRegionServer implements HConstants, HRegionInterface, * @return intId Integer row lock used internally in HRegion * @throws IOException Thrown if this is not a valid client lock id. */ - private Integer getLockFromId(long lockId) + Integer getLockFromId(long lockId) throws IOException { if(lockId == -1L) { return null; @@ -2147,6 +2122,10 @@ public class HRegionServer implements HConstants, HRegionInterface, return Collections.unmodifiableCollection(onlineRegions.values()); } + public HRegion [] getOnlineRegionsAsArray() { + return getOnlineRegions().toArray(new HRegion[0]); + } + /** * @return The HRegionInfos from online regions sorted */ @@ -2410,7 +2389,6 @@ public class HRegionServer implements HConstants, HRegionInterface, } catch (Throwable t) { LOG.error( "Can not start region server because "+ StringUtils.stringifyException(t) ); - System.exit(-1); } break; } @@ -2426,39 +2404,20 @@ public class HRegionServer implements HConstants, HRegionInterface, } } - /** - * @param args - */ - public static void main(String [] args) { - Configuration conf = new HBaseConfiguration(); - @SuppressWarnings("unchecked") - Class regionServerClass = (Class) conf - .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class); - doMain(args, regionServerClass); - } - /** {@inheritDoc} */ - public long incrementColumnValue(byte[] regionName, byte[] row, - byte[] column, long amount) throws IOException { + public long incrementColumnValue(byte [] regionName, byte [] row, + byte [] family, byte [] qualifier, long amount) + throws IOException { checkOpen(); - - NullPointerException npe = null; + if (regionName == null) { - npe = new NullPointerException("regionName is null"); - } else if (row == null) { - npe = new NullPointerException("row is null"); - } else if (column == null) { - npe = new NullPointerException("column is null"); - } - if (npe != null) { - IOException io = new IOException( - "Invalid arguments to incrementColumnValue", npe); - throw io; + throw new IOException("Invalid arguments to incrementColumnValue " + + "regionName is null"); } requestCount.incrementAndGet(); try { HRegion region = getRegion(regionName); - return region.incrementColumnValue(row, column, amount); + return region.incrementColumnValue(row, family, qualifier, amount); } catch (IOException e) { checkFileSystem(); throw e; @@ -2479,4 +2438,17 @@ public class HRegionServer implements HConstants, HRegionInterface, public HServerInfo getHServerInfo() throws IOException { return serverInfo; } + + + /** + * @param args + */ + public static void main(String [] args) { + Configuration conf = new HBaseConfiguration(); + @SuppressWarnings("unchecked") + Class regionServerClass = (Class) conf + .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class); + doMain(args, regionServerClass); + } + } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java index eff23bda5fb..83b6cbc593d 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java @@ -1,5 +1,5 @@ /** - * Copyright 2007 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -41,11 +41,9 @@ import org.apache.hadoop.hbase.KeyValue; */ public interface InternalScanner extends Closeable { /** - * Grab the next row's worth of values. The scanner will return the most - * recent data value for each row that is not newer than the target time - * passed when the scanner was created. + * Grab the next row's worth of values. * @param results - * @return true if data was returned + * @return true if more rows exist after this one, false if scanner is done * @throws IOException */ public boolean next(List results) @@ -55,11 +53,5 @@ public interface InternalScanner extends Closeable { * Closes the scanner and releases any resources it has allocated * @throws IOException */ - public void close() throws IOException; - - /** @return true if the scanner is matching a column family or regex */ - public boolean isWildcardScanner(); - - /** @return true if the scanner is matching multiple column family members */ - public boolean isMultipleMatchScanner(); + public void close() throws IOException; } \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java b/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java new file mode 100644 index 00000000000..b1a329f8def --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java @@ -0,0 +1,197 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.Comparator; +import java.util.List; +import java.util.PriorityQueue; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValue.KVComparator; + +/** + * Implements a heap merge across any number of KeyValueScanners. + *

            + * Implements KeyValueScanner itself. + *

            + * This class is used at the Region level to merge across Stores + * and at the Store level to merge across the Memcache and StoreFiles. + *

            + * In the Region case, we also need InternalScanner.next(List), so this class + * also implements InternalScanner. WARNING: As is, if you try to use this + * as an InternalScanner at the Store level, you will get runtime exceptions. + */ +public class KeyValueHeap implements KeyValueScanner, InternalScanner { + + private PriorityQueue heap; + + private KeyValueScanner current = null; + + private KVScannerComparator comparator; + + /** + * Constructor + * @param scanners + * @param comparator + */ + public KeyValueHeap(KeyValueScanner [] scanners, KVComparator comparator) { + this.comparator = new KVScannerComparator(comparator); + this.heap = new PriorityQueue(scanners.length, + this.comparator); + for(KeyValueScanner scanner : scanners) { + if(scanner.peek() != null) { + this.heap.add(scanner); + } + } + this.current = heap.poll(); + } + + public KeyValue peek() { + if(this.current == null) { + return null; + } + return this.current.peek(); + } + + public KeyValue next() { + if(this.current == null) { + return null; + } + KeyValue kvReturn = this.current.next(); + KeyValue kvNext = this.current.peek(); + if(kvNext == null) { + this.current.close(); + this.current = this.heap.poll(); + } else { + KeyValueScanner topScanner = this.heap.peek(); + if(topScanner == null || + this.comparator.compare(kvNext, topScanner.peek()) > 0) { + this.heap.add(this.current); + this.current = this.heap.poll(); + } + } + return kvReturn; + } + + /** + * Gets the next row of keys from the top-most scanner. + *

            + * This method takes care of updating the heap. + *

            + * This can ONLY be called when you are using Scanners that implement + * InternalScanner as well as KeyValueScanner (a {@link StoreScanner}). + * @return true if there are more keys, false if all scanners are done + */ + public boolean next(List result) throws IOException { + InternalScanner currentAsInternal = (InternalScanner)this.current; + currentAsInternal.next(result); + KeyValue pee = this.current.peek(); + if(pee == null) { + this.current.close(); + } else { + this.heap.add(this.current); + } + this.current = this.heap.poll(); + return (this.current != null); + } + + private class KVScannerComparator implements Comparator { + private KVComparator kvComparator; + /** + * Constructor + * @param kvComparator + */ + public KVScannerComparator(KVComparator kvComparator) { + this.kvComparator = kvComparator; + } + public int compare(KeyValueScanner left, KeyValueScanner right) { + return compare(left.peek(), right.peek()); + } + /** + * Compares two KeyValue + * @param left + * @param right + * @return less than 0 if left is smaller, 0 if equal etc.. + */ + public int compare(KeyValue left, KeyValue right) { + return this.kvComparator.compare(left, right); + } + /** + * @return KVComparator + */ + public KVComparator getComparator() { + return this.kvComparator; + } + } + + public void close() { + if(this.current != null) { + this.current.close(); + } + KeyValueScanner scanner; + while((scanner = this.heap.poll()) != null) { + scanner.close(); + } + } + + /** + * Seeks all scanners at or below the specified seek key. If we earlied-out + * of a row, we may end up skipping values that were never reached yet. + * Rather than iterating down, we want to give the opportunity to re-seek. + *

            + * As individual scanners may run past their ends, those scanners are + * automatically closed and removed from the heap. + * @param seekKey KeyValue to seek at or after + * @return true if KeyValues exist at or after specified key, false if not + */ + public boolean seek(KeyValue seekKey) { + if(this.current == null) { + return false; + } + this.heap.add(this.current); + this.current = null; + + KeyValueScanner scanner; + while((scanner = this.heap.poll()) != null) { + KeyValue topKey = scanner.peek(); + if(comparator.getComparator().compare(seekKey, topKey) <= 0) { // Correct? + // Top KeyValue is at-or-after Seek KeyValue + this.current = scanner; + return true; + } + if(!scanner.seek(seekKey)) { + scanner.close(); + } else { + this.heap.add(scanner); + } + } + // Heap is returning empty, scanner is done + return false; + } + + /** + * @return the current Heap + */ + public PriorityQueue getHeap() { + return this.heap; + } +} diff --git a/src/java/org/apache/hadoop/hbase/client/transactional/CommitUnsuccessfulException.java b/src/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java similarity index 53% rename from src/java/org/apache/hadoop/hbase/client/transactional/CommitUnsuccessfulException.java rename to src/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java index 76573630071..4c5b8443017 100644 --- a/src/java/org/apache/hadoop/hbase/client/transactional/CommitUnsuccessfulException.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java @@ -1,5 +1,5 @@ -/** - * Copyright 2008 The Apache Software Foundation +/* + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -17,40 +17,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.client.transactional; +package org.apache.hadoop.hbase.regionserver; -/** Thrown when a transaction cannot be committed. - * +import org.apache.hadoop.hbase.KeyValue; + +/** + * Scanner that returns the next KeyValue. */ -public class CommitUnsuccessfulException extends Exception { - - private static final long serialVersionUID = 7062921444531109202L; - - /** Default Constructor */ - public CommitUnsuccessfulException() { - super(); - } - +public interface KeyValueScanner { /** - * @param arg0 message - * @param arg1 cause + * Look at the next KeyValue in this scanner, but do not iterate scanner. + * @return the next KeyValue */ - public CommitUnsuccessfulException(String arg0, Throwable arg1) { - super(arg0, arg1); - } - + public KeyValue peek(); + /** - * @param arg0 message + * Return the next KeyValue in this scanner, iterating the scanner + * @return the next KeyValue */ - public CommitUnsuccessfulException(String arg0) { - super(arg0); - } - + public KeyValue next(); + /** - * @param arg0 cause + * Seek the scanner at or after the specified KeyValue. + * @param key + * @return true if scanner has values left, false if end of scanner */ - public CommitUnsuccessfulException(Throwable arg0) { - super(arg0); - } - -} + public boolean seek(KeyValue key); + + /** + * Close the KeyValue scanner. + */ + public void close(); +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java b/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java index f2dc207e198..8441d641a68 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java @@ -1,5 +1,5 @@ /** - * Copyright 2008 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -27,22 +27,19 @@ import java.rmi.UnexpectedException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.NavigableSet; import java.util.Set; import java.util.SortedMap; import java.util.SortedSet; -import java.util.TreeMap; import java.util.TreeSet; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.regex.Pattern; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.regionserver.HRegion.Counter; +import org.apache.hadoop.hbase.regionserver.DeleteCompare.DeleteCode; import org.apache.hadoop.hbase.util.Bytes; /** @@ -82,7 +79,7 @@ class Memcache { // TODO: Fix this guess by studying jprofiler private final static int ESTIMATED_KV_HEAP_TAX = 60; - + /** * Default constructor. Used for tests. */ @@ -202,7 +199,86 @@ class Memcache { } return size; } + + /** + * Write a delete + * @param delete + * @return approximate size of the passed key and value. + */ + long delete(final KeyValue delete) { + long size = 0; + this.lock.readLock().lock(); + //Have to find out what we 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 delete and remove all the affected entries already in memcache + //2. In the case of a Delete and the matching put is found then don't insert + // the delete + //TODO Would be nice with if we had an iterator for this, so we could remove + //things that needs to be removed while iterating and don't have to go + //back and do it afterwards + + try { + boolean notpresent = false; + List deletes = new ArrayList(); + SortedSet tailSet = this.memcache.tailSet(delete); + //Parse the delete, so that it is only done once + byte [] deleteBuffer = delete.getBuffer(); + int deleteOffset = delete.getOffset(); + + int deleteKeyLen = Bytes.toInt(deleteBuffer, deleteOffset); + deleteOffset += Bytes.SIZEOF_INT + Bytes.SIZEOF_INT; + + short deleteRowLen = Bytes.toShort(deleteBuffer, deleteOffset); + deleteOffset += Bytes.SIZEOF_SHORT; + int deleteRowOffset = deleteOffset; + + deleteOffset += deleteRowLen; + + byte deleteFamLen = deleteBuffer[deleteOffset]; + deleteOffset += Bytes.SIZEOF_BYTE + deleteFamLen; + + int deleteQualifierOffset = deleteOffset; + int deleteQualifierLen = deleteKeyLen - deleteRowLen - deleteFamLen - + Bytes.SIZEOF_SHORT - Bytes.SIZEOF_BYTE - Bytes.SIZEOF_LONG - + Bytes.SIZEOF_BYTE; + + deleteOffset += deleteQualifierLen; + + int deleteTimestampOffset = deleteOffset; + deleteOffset += Bytes.SIZEOF_LONG; + byte deleteType = deleteBuffer[deleteOffset]; + + //Comparing with tail from memcache + for(KeyValue mem : tailSet) { + + DeleteCode res = DeleteCompare.deleteCompare(mem, deleteBuffer, + deleteRowOffset, deleteRowLen, deleteQualifierOffset, + deleteQualifierLen, deleteTimestampOffset, deleteType, + comparator.getRawComparator()); + if(res == DeleteCode.DONE) { + break; + } else if (res == DeleteCode.DELETE) { + deletes.add(mem); + } // SKIP + } + + //Delete all the entries effected by the last added delete + for(KeyValue del : deletes) { + notpresent = this.memcache.remove(del); + size -= heapSize(del, notpresent); + } + + //Adding the delete to memcache + notpresent = this.memcache.add(delete); + size += heapSize(delete, notpresent); + } finally { + this.lock.readLock().unlock(); + } + return size; + } + /* * Calculate how the memcache size has changed, approximately. Be careful. * If class changes, be sure to change the size calculation. @@ -218,43 +294,6 @@ class Memcache { ESTIMATED_KV_HEAP_TAX + 57 + kv.getLength(): 0; // Guess no change in size. } - /** - * Look back through all the backlog TreeMaps to find the target. - * @param kv - * @param numVersions - * @return Set of KeyValues. Empty size not null if no results. - */ - List get(final KeyValue kv, final int numVersions) { - List results = new ArrayList(); - get(kv, numVersions, results, - new TreeSet(this.comparatorIgnoreType), - System.currentTimeMillis()); - return results; - } - - /** - * Look back through all the backlog TreeMaps to find the target. - * @param key - * @param versions - * @param results - * @param deletes Pass a Set that has a Comparator that ignores key type. - * @param now - * @return True if enough versions. - */ - boolean get(final KeyValue key, final int versions, - List results, final NavigableSet deletes, - final long now) { - this.lock.readLock().lock(); - try { - if (get(this.memcache, key, versions, results, deletes, now)) { - return true; - } - return get(this.snapshot, key, versions , results, deletes, now); - } finally { - this.lock.readLock().unlock(); - } - } - /** * @param kv Find the row that comes after this one. If null, we return the * first. @@ -307,86 +346,6 @@ class Memcache { return result; } - /** - * Return all the available columns for the given key. The key indicates a - * row and timestamp, but not a column name. - * @param origin Where to start searching. Specifies a row and timestamp. - * Columns are specified in following arguments. - * @param columns Pass null for all columns else the wanted subset. - * @param columnPattern Column pattern to match. - * @param numVersions number of versions to retrieve - * @param versionsCount Map of KV to Count. Uses a Comparator that doesn't - * look at timestamps so only Row/Column are compared. - * @param deletes Pass a Set that has a Comparator that ignores key type. - * @param results Where to stick row results found. - * @return True if we found enough results for passed columns - * and numVersions. - */ - boolean getFull(final KeyValue key, NavigableSet columns, - final Pattern columnPattern, - int numVersions, final Map versionsCount, - final NavigableSet deletes, - final List results, final long now) { - this.lock.readLock().lock(); - try { - // Used to be synchronized but now with weak iteration, no longer needed. - if (getFull(this.memcache, key, columns, columnPattern, numVersions, - versionsCount, deletes, results, now)) { - // Has enough results. - return true; - } - return getFull(this.snapshot, key, columns, columnPattern, numVersions, - versionsCount, deletes, results, now); - } finally { - this.lock.readLock().unlock(); - } - } - - /* - * @param set - * @param target Where to start searching. - * @param columns - * @param versions - * @param versionCounter - * @param deletes Pass a Set that has a Comparator that ignores key type. - * @param keyvalues - * @return True if enough results found. - */ - private boolean getFull(final ConcurrentSkipListSet set, - final KeyValue target, final Set columns, - final Pattern columnPattern, - final int versions, final Map versionCounter, - final NavigableSet deletes, List keyvalues, - final long now) { - boolean hasEnough = false; - if (target == null) { - return hasEnough; - } - NavigableSet tailset = set.tailSet(target); - if (tailset == null || tailset.isEmpty()) { - return hasEnough; - } - // TODO: This loop same as in HStore.getFullFromStoreFile. Make sure they - // are the same. - for (KeyValue kv: tailset) { - // Make sure we have not passed out the row. If target key has a - // column on it, then we are looking explicit key+column combination. If - // we've passed it out, also break. - if (target.isEmptyColumn()? !this.comparator.matchingRows(target, kv): - !this.comparator.matchingRowColumn(target, kv)) { - break; - } - if (!Store.getFullCheck(this.comparator, target, kv, columns, columnPattern)) { - continue; - } - if (Store.doKeyValue(kv, versions, versionCounter, columns, deletes, now, - this.ttl, keyvalues, tailset)) { - hasEnough = true; - break; - } - } - return hasEnough; - } /** * @param row Row to look for. @@ -554,45 +513,6 @@ class Memcache { } } - /* - * Examine a single map for the desired key. - * - * TODO - This is kinda slow. We need a data structure that allows for - * proximity-searches, not just precise-matches. - * - * @param set - * @param key - * @param results - * @param versions - * @param keyvalues - * @param deletes Pass a Set that has a Comparator that ignores key type. - * @param now - * @return True if enough versions. - */ - private boolean get(final ConcurrentSkipListSet set, - final KeyValue key, final int versions, - final List keyvalues, - final NavigableSet deletes, - final long now) { - NavigableSet tailset = set.tailSet(key); - if (tailset.isEmpty()) { - return false; - } - boolean enoughVersions = false; - for (KeyValue kv : tailset) { - if (this.comparator.matchingRowColumn(kv, key)) { - if (Store.doKeyValue(kv, versions, deletes, now, this.ttl, keyvalues, - tailset)) { - enoughVersions = true; - break; - } - } else { - // By L.N. HBASE-684, map is sorted, so we can't find match any more. - break; - } - } - return enoughVersions; - } /* * @param set @@ -621,93 +541,160 @@ class Memcache { /** * @return a scanner over the keys in the Memcache */ - InternalScanner getScanner(long timestamp, - final NavigableSet targetCols, final byte [] firstRow) - throws IOException { + KeyValueScanner getScanner() { this.lock.readLock().lock(); try { - return new MemcacheScanner(timestamp, targetCols, firstRow); + return new MemcacheScanner(); } finally { this.lock.readLock().unlock(); } } - ////////////////////////////////////////////////////////////////////////////// - // MemcacheScanner implements the InternalScanner. - // It lets the caller scan the contents of the Memcache. - ////////////////////////////////////////////////////////////////////////////// - - private class MemcacheScanner extends HAbstractScanner { - private KeyValue current; - private final NavigableSet columns; - private final NavigableSet deletes; - private final Map versionCounter; - private final long now = System.currentTimeMillis(); - - MemcacheScanner(final long timestamp, final NavigableSet columns, - final byte [] firstRow) - throws IOException { - // Call to super will create ColumnMatchers and whether this is a regex - // scanner or not. Will also save away timestamp. Also sorts rows. - super(timestamp, columns); - this.deletes = new TreeSet(comparatorIgnoreType); - this.versionCounter = - new TreeMap(comparatorIgnoreTimestamp); - this.current = KeyValue.createFirstOnRow(firstRow, timestamp); - // If we're being asked to scan explicit columns rather than all in - // a family or columns that match regexes, cache the sorted array of - // columns. - this.columns = isWildcardScanner()? null: columns; + // + // HBASE-880/1249/1304 + // + + /** + * Perform a single-row Get on the memcache and snapshot, placing results + * into the specified KV list. + *

            + * This will return true if it is determined that the query is complete + * and it is not necessary to check any storefiles after this. + *

            + * Otherwise, it will return false and you should continue on. + * @param startKey Starting KeyValue + * @param matcher Column matcher + * @param result List to add results to + * @return true if done with store (early-out), false if not + * @throws IOException + */ + public boolean get(QueryMatcher matcher, List result) + throws IOException { + this.lock.readLock().lock(); + try { + if(internalGet(this.memcache, matcher, result) || matcher.isDone()) { + return true; + } + matcher.update(); + if(internalGet(this.snapshot, matcher, result) || matcher.isDone()) { + return true; + } + return false; + } finally { + this.lock.readLock().unlock(); } + } + + /** + * + * @param set memcache or snapshot + * @param matcher query matcher + * @param result list to add results to + * @return true if done with store (early-out), false if not + * @throws IOException + */ + private boolean internalGet(SortedSet set, QueryMatcher matcher, + List result) throws IOException { + if(set.isEmpty()) return false; + // Seek to startKey + SortedSet tailSet = set.tailSet(matcher.getStartKey()); + + for (KeyValue kv : tailSet) { + QueryMatcher.MatchCode res = matcher.match(kv); + switch(res) { + case INCLUDE: + result.add(kv); + break; + case SKIP: + break; + case NEXT: + return false; + case DONE: + return true; + default: + throw new RuntimeException("Unexpected " + res); + } + } + return false; + } + + ////////////////////////////////////////////////////////////////////////////// + // MemcacheScanner implements the KeyValueScanner. + // It lets the caller scan the contents of the Memcache. + // This behaves as if it were a real scanner but does not maintain position + // in the Memcache tree. + ////////////////////////////////////////////////////////////////////////////// - @Override - public boolean next(final List keyvalues) - throws IOException { - if (this.scannerClosed) { + protected class MemcacheScanner implements KeyValueScanner { + private KeyValue current = null; + private List result = new ArrayList(); + private int idx = 0; + + MemcacheScanner() {} + + public boolean seek(KeyValue key) { + try { + if(key == null) { + close(); + return false; + } + current = key; + return cacheNextRow(); + } catch(Exception e) { + close(); return false; } - while (keyvalues.isEmpty() && this.current != null) { - // Deletes are per row. - if (!deletes.isEmpty()) { - deletes.clear(); + } + + public KeyValue peek() { + if(idx >= result.size()) { + if(!cacheNextRow()) { + return null; } - if (!versionCounter.isEmpty()) { - versionCounter.clear(); - } - // The getFull will take care of expired and deletes inside memcache. - // The first getFull when row is the special empty bytes will return - // nothing so we go around again. Alternative is calling a getNextRow - // if row is null but that looks like it would take same amount of work - // so leave it for now. - getFull(this.current, isWildcardScanner()? null: this.columns, null, 1, - versionCounter, deletes, keyvalues, this.now); - for (KeyValue bb: keyvalues) { - if (isWildcardScanner()) { - // Check the results match. We only check columns, not timestamps. - // We presume that timestamps have been handled properly when we - // called getFull. - if (!columnMatch(bb)) { - keyvalues.remove(bb); - } - } - } - // Add any deletes found so they are available to the StoreScanner#next. - if (!this.deletes.isEmpty()) { - keyvalues.addAll(deletes); - } - this.current = getNextRow(this.current); - // Change current to be column-less and to have the scanners' now. We - // do this because first item on 'next row' may not have the scanners' - // now time which will cause trouble down in getFull; same reason no - // column. - if (this.current != null) this.current = this.current.cloneRow(this.now); + return peek(); } - return !keyvalues.isEmpty(); + return result.get(idx); + } + + public KeyValue next() { + if(idx >= result.size()) { + if(!cacheNextRow()) { + return null; + } + return next(); + } + return result.get(idx++); + } + + boolean cacheNextRow() { + NavigableSet keys; + try { + keys = memcache.tailSet(current); + } catch(Exception e) { + close(); + return false; + } + if(keys == null || keys.isEmpty()) { + close(); + return false; + } + current = null; + byte [] row = keys.first().getRow(); + for(KeyValue key : keys) { + if(comparator.compareRows(key, row) != 0) { + current = key; + break; + } + result.add(key); + } + return true; } public void close() { - if (!scannerClosed) { - scannerClosed = true; + current = null; + idx = 0; + if(!result.isEmpty()) { + result.clear(); } } } @@ -721,8 +708,7 @@ class Memcache { * @throws InterruptedException * @throws IOException */ - public static void main(String [] args) - throws InterruptedException, IOException { + public static void main(String [] args) { RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean(); LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" + runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion()); diff --git a/src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java b/src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java index 0f0a7249243..f5a24dcd813 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java @@ -62,11 +62,11 @@ class MemcacheFlusher extends Thread implements FlushRequester { protected final long globalMemcacheLimit; protected final long globalMemcacheLimitLowMark; - public static final float DEFAULT_UPPER = 0.4f; - public static final float DEFAULT_LOWER = 0.25f; - public static final String UPPER_KEY = + private static final float DEFAULT_UPPER = 0.4f; + private static final float DEFAULT_LOWER = 0.25f; + private static final String UPPER_KEY = "hbase.regionserver.globalMemcache.upperLimit"; - public static final String LOWER_KEY = + private static final String LOWER_KEY = "hbase.regionserver.globalMemcache.lowerLimit"; private long blockingStoreFilesNumber; private long blockingWaitTime; diff --git a/src/java/org/apache/hadoop/hbase/regionserver/MinorCompactingStoreScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/MinorCompactingStoreScanner.java new file mode 100644 index 00000000000..fa9775ad104 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/MinorCompactingStoreScanner.java @@ -0,0 +1,119 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + +import java.util.List; +import java.io.IOException; + +/** + * A scanner that does a minor compaction at the same time. Doesn't need to + * implement ChangedReadersObserver, since it doesn't scan memcache, only store files + * and optionally the memcache-snapshot. + */ +public class MinorCompactingStoreScanner implements KeyValueScanner, InternalScanner { + private QueryMatcher matcher; + + private KeyValueHeap heap; + + + MinorCompactingStoreScanner(Store store, + KeyValueScanner [] scanners) { + Scan scan = new Scan(); + + // No max version, no ttl matching, start at first row, all columns. + matcher = new ScanQueryMatcher(scan, store.getFamily().getName(), + null, Long.MAX_VALUE, store.comparator.getRawComparator(), + store.versionsToReturn(Integer.MAX_VALUE)); + + for (KeyValueScanner scanner : scanners ) { + scanner.seek(matcher.getStartKey()); + } + + heap = new KeyValueHeap(scanners, store.comparator); + } + + MinorCompactingStoreScanner(String cfName, KeyValue.KVComparator comparator, + KeyValueScanner [] scanners) { + Scan scan = new Scan(); + matcher = new ScanQueryMatcher(scan, Bytes.toBytes(cfName), + null, Long.MAX_VALUE, comparator.getRawComparator(), + Integer.MAX_VALUE); + + for (KeyValueScanner scanner : scanners ) { + scanner.seek(matcher.getStartKey()); + } + + heap = new KeyValueHeap(scanners, comparator); + } + + public KeyValue peek() { + return heap.peek(); + } + + public KeyValue next() { + return heap.next(); + } + + @Override + public boolean seek(KeyValue key) { + // cant seek. + throw new UnsupportedOperationException("Can't seek a MinorCompactingStoreScanner"); + } + + @Override + public boolean next(List results) throws IOException { + KeyValue peeked = heap.peek(); + if (peeked == null) { + close(); + return false; + } + matcher.setRow(peeked.getRow()); + KeyValue kv; + while ((kv = heap.peek()) != null) { + // if delete type, output no matter what: + if (kv.getType() != KeyValue.Type.Put.getCode()) + results.add(kv); + + switch (matcher.match(kv)) { + case INCLUDE: + results.add(heap.next()); + continue; + case DONE: + if (results.isEmpty()) { + matcher.setRow(heap.peek().getRow()); + continue; + } + return true; + } + heap.next(); + } + close(); + return false; + } + + public void close() { + heap.close(); + } +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/QueryMatcher.java b/src/java/org/apache/hadoop/hbase/regionserver/QueryMatcher.java new file mode 100644 index 00000000000..7b5ca17bd9f --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/QueryMatcher.java @@ -0,0 +1,373 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import java.util.NavigableSet; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValue.KeyComparator; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.io.TimeRange; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * This is the primary class used to process KeyValues during a Get or Scan + * operation. + *

            + * It encapsulates the handling of the column and version input parameters to + * the query through a {@link ColumnTracker}. + *

            + * Deletes are handled using the {@link DeleteTracker}. + *

            + * All other query parameters are accessed from the client-specified Get. + *

            + * The primary method used is {@link match} with the current KeyValue. It will + * return a {@link MatchCode} + * + * , deletes, + * versions, + */ +public class QueryMatcher { + + /** + * {@link match} return codes. These instruct the scanner moving through + * Memcaches and StoreFiles what to do with the current KeyValue. + *

            + * Additionally, this contains "early-out" language to tell the scanner to + * move on to the next File (Memcache or Storefile), or to return immediately. + */ + static enum MatchCode { + /** + * Include KeyValue in the returned result + */ + INCLUDE, + + /** + * Do not include KeyValue in the returned result + */ + SKIP, + + /** + * Do not include, jump to next StoreFile or Memcache (in time order) + */ + NEXT, + + /** + * Do not include, return current result + */ + DONE, + + /** + * These codes are used by the ScanQueryMatcher + */ + + /** + * Done with the row, seek there. + */ + SEEK_NEXT_ROW, + /** + * Done with column, seek to next. + */ + SEEK_NEXT_COL, + + /** + * Done with scan, thanks to the row filter. + */ + DONE_SCAN, + } + + /** Keeps track of deletes */ + protected DeleteTracker deletes; + + /** Keeps track of columns and versions */ + protected ColumnTracker columns; + + /** Key to seek to in Memcache and StoreFiles */ + protected KeyValue startKey; + + /** Row comparator for the region this query is for */ + KeyComparator rowComparator; + + /** Row the query is on */ + protected byte [] row; + + /** TimeRange the query is for */ + protected TimeRange tr; + + /** Oldest allowed version stamp for TTL enforcement */ + protected long oldestStamp; + + /** + * Constructs a QueryMatcher for a Get. + * @param get + * @param row + * @param family + * @param columns + * @param ttl + * @param rowComparator + */ + public QueryMatcher(Get get, byte [] row, byte [] family, + NavigableSet columns, long ttl, KeyComparator rowComparator, + int maxVersions) { + this.row = row; + this.tr = get.getTimeRange(); + this.oldestStamp = System.currentTimeMillis() - ttl; + this.rowComparator = rowComparator; + this.deletes = new GetDeleteTracker(rowComparator); + this.startKey = KeyValue.createFirstOnRow(row); + // Single branch to deal with two types of Gets (columns vs all in family) + if(columns == null || columns.size() == 0) { + this.columns = new WildcardColumnTracker(maxVersions); + } else { + this.columns = new ExplicitColumnTracker(columns, maxVersions); + } + } + + // For the subclasses. + protected QueryMatcher() { + } + + /** + * Constructs a copy of an existing QueryMatcher with a new row. + * @param matcher + * @param row + */ + public QueryMatcher(QueryMatcher matcher, byte [] row) { + this.row = row; + this.tr = matcher.getTimeRange(); + this.oldestStamp = matcher.getOldestStamp(); + this.rowComparator = matcher.getRowComparator(); + this.columns = matcher.getColumnTracker(); + this.deletes = matcher.getDeleteTracker(); + this.startKey = matcher.getStartKey(); + reset(); + } + + /** + * Main method for ColumnMatcher. + *

            + * Determines whether the specified KeyValue should be included in the + * result or not. + *

            + * Contains additional language to early-out of the current file or to + * return immediately. + *

            + * Things to be checked:

              + *
            • Row + *
            • TTL + *
            • Type + *
            • TimeRange + *
            • Deletes + *
            • Column + *
            • Versions + * @param kv KeyValue to check + * @return MatchCode: include, skip, next, done + */ + public MatchCode match(KeyValue kv) { + if(this.columns.done()) { + return MatchCode.DONE; // done_row + } + + // Directly act on KV buffer + byte [] bytes = kv.getBuffer(); + int offset = kv.getOffset(); + + int keyLength = Bytes.toInt(bytes, offset); + offset += KeyValue.ROW_OFFSET; + + short rowLength = Bytes.toShort(bytes, offset); + offset += Bytes.SIZEOF_SHORT; + + // scanners are relying on us to check the row first, and return + // "NEXT" when we are there. + /* Check ROW + * If past query's row, go to next StoreFile + * If not reached query's row, go to next KeyValue + */ + int ret = this.rowComparator.compareRows(row, 0, row.length, + bytes, offset, rowLength); + if(ret <= -1) { + // Have reached the next row + return MatchCode.NEXT; // got_to_next_row (end) + } else if(ret >= 1) { + // At a previous row + return MatchCode.SKIP; // skip_to_cur_row + } + offset += rowLength; + + byte familyLength = bytes[offset]; + offset += Bytes.SIZEOF_BYTE + familyLength; + + int columnLength = keyLength + KeyValue.ROW_OFFSET - + (offset - kv.getOffset()) - KeyValue.TIMESTAMP_TYPE_SIZE; + int columnOffset = offset; + offset += columnLength; + + /* Check TTL + * If expired, go to next KeyValue + */ + long timestamp = Bytes.toLong(bytes, offset); + if(isExpired(timestamp)) { + // reached the expired part, for scans, this indicates we're done. + return MatchCode.NEXT; // done_row + } + offset += Bytes.SIZEOF_LONG; + + /* Check TYPE + * If a delete within (or after) time range, add to deletes + * Move to next KeyValue + */ + byte type = bytes[offset]; + // if delete type == delete family, return done_row + + if(isDelete(type)) { + if(tr.withinOrAfterTimeRange(timestamp)) { + this.deletes.add(bytes, columnOffset, columnLength, timestamp, type); + } + return MatchCode.SKIP; // skip the delete cell. + } + + /* Check TimeRange + * If outside of range, move to next KeyValue + */ + if(!tr.withinTimeRange(timestamp)) { + return MatchCode.SKIP; // optimization chances here. + } + + /* Check Deletes + * If deleted, move to next KeyValue + */ + if(!deletes.isEmpty() && deletes.isDeleted(bytes, columnOffset, + columnLength, timestamp)) { + // 2 types of deletes: + // affects 1 cell or 1 column, so just skip the keyvalues. + // - delete family, so just skip to the next row. + return MatchCode.SKIP; + } + + /* Check Column and Versions + * Returns a MatchCode directly, identical language + * If matched column without enough versions, include + * If enough versions of this column or does not match, skip + * If have moved past + * If enough versions of everything, + */ + return columns.checkColumn(bytes, columnOffset, columnLength); + } + + // should be in KeyValue. + protected boolean isDelete(byte type) { + return (type != KeyValue.Type.Put.getCode()); + } + + protected boolean isExpired(long timestamp) { + return (timestamp < oldestStamp); + } + + /** + * If matcher returns SEEK_NEXT_COL you may be able + * to get a hint of the next column to seek to - call this. + * If it returns null, there is no hint. + * + * @return immediately after match returns SEEK_NEXT_COL - null if no hint, + * else the next column we want + */ + public ColumnCount getSeekColumn() { + return this.columns.getColumnHint(); + } + + /** + * Called after reading each section (memcache, snapshot, storefiles). + *

              + * This method will update the internal structures to be accurate for + * the next section. + */ + public void update() { + this.deletes.update(); + this.columns.update(); + } + + /** + * Resets the current columns and deletes + */ + public void reset() { + this.deletes.reset(); + this.columns.reset(); + } + + /** + * Set current row + * @param row + */ + public void setRow(byte [] row) { + this.row = row; + } + + + /** + * + * @return the start key + */ + public KeyValue getStartKey() { + return this.startKey; + } + + /** + * @return the TimeRange + */ + public TimeRange getTimeRange() { + return this.tr; + } + + /** + * @return the oldest stamp + */ + public long getOldestStamp() { + return this.oldestStamp; + } + + /** + * @return current KeyComparator + */ + public KeyComparator getRowComparator() { + return this.rowComparator; + } + + /** + * @return ColumnTracker + */ + public ColumnTracker getColumnTracker() { + return this.columns; + } + + /** + * @return DeleteTracker + */ + public DeleteTracker getDeleteTracker() { + return this.deletes; + } + /** + * + * @return + */ + public boolean isDone() { + return this.columns.done(); + } +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ScanDeleteTracker.java b/src/java/org/apache/hadoop/hbase/regionserver/ScanDeleteTracker.java new file mode 100644 index 00000000000..118913d57ae --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/ScanDeleteTracker.java @@ -0,0 +1,161 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.hbase.KeyValue; + +/** + * This class is responsible for the tracking and enforcement of Deletes + * during the course of a Scan operation. + * + * It only has to enforce Delete and DeleteColumn, since the + * DeleteFamily is handled at a higher level. + * + *

              + * This class is utilized through three methods: + *

              • {@link add} when encountering a Delete or DeleteColumn + *
              • {@link isDeleted} when checking if a Put KeyValue has been deleted + *
              • {@link update} when reaching the end of a StoreFile or row for scans + *

                + * This class is NOT thread-safe as queries are never multi-threaded + */ +public class ScanDeleteTracker implements DeleteTracker { + + private long familyStamp = -1L; + private byte [] deleteBuffer = null; + private int deleteOffset = 0; + private int deleteLength = 0; + private byte deleteType = 0; + private long deleteTimestamp = 0L; + + private KeyValue.KeyComparator comparator; + + /** + * Constructor for ScanDeleteTracker + * @param comparator + */ + public ScanDeleteTracker(KeyValue.KeyComparator comparator) { + this.comparator = comparator; + } + + /** + * Add the specified KeyValue to the list of deletes to check against for + * this row operation. + *

                + * This is called when a Delete is encountered in a StoreFile. + * @param buffer KeyValue buffer + * @param qualifierOffset column qualifier offset + * @param qualifierLength column qualifier length + * @param timestamp timestamp + * @param type delete type as byte + */ + @Override + public void add(byte[] buffer, int qualifierOffset, int qualifierLength, + long timestamp, byte type) { + if(timestamp > familyStamp) { + if(type == KeyValue.Type.DeleteFamily.getCode()) { + familyStamp = timestamp; + return; + } + + if(deleteBuffer != null && type < deleteType) { + // same column, so ignore less specific delete + if(comparator.compareRows(deleteBuffer, deleteOffset, deleteLength, + buffer, qualifierOffset, qualifierLength) == 0){ + return; + } + } + // new column, or more general delete type + deleteBuffer = buffer; + deleteOffset = qualifierOffset; + deleteLength = qualifierLength; + deleteType = type; + deleteTimestamp = timestamp; + } + // missing else is never called. + } + + /** + * Check if the specified KeyValue buffer has been deleted by a previously + * seen delete. + * + * @param buffer KeyValue buffer + * @param qualifierOffset column qualifier offset + * @param qualifierLength column qualifier length + * @param timestamp timestamp + * @return true is the specified KeyValue is deleted, false if not + */ + @Override + public boolean isDeleted(byte [] buffer, int qualifierOffset, + int qualifierLength, long timestamp) { + if(timestamp < familyStamp) { + return true; + } + + if(deleteBuffer != null) { + // TODO ryan use a specific comparator + int ret = comparator.compareRows(deleteBuffer, deleteOffset, deleteLength, + buffer, qualifierOffset, qualifierLength); + + if(ret == 0) { + if(deleteType == KeyValue.Type.DeleteColumn.getCode()) { + return true; + } + // Delete (aka DeleteVersion) + // If the timestamp is the same, keep this one + if (timestamp == deleteTimestamp) { + return true; + } + // use assert or not? + assert timestamp < deleteTimestamp; + + // different timestamp, let's clear the buffer. + deleteBuffer = null; + } else if(ret < 0){ + // Next column case. + deleteBuffer = null; + } else { + //Should never happen, throw Exception + } + } + + return false; + } + + @Override + public boolean isEmpty() { + return deleteBuffer == null && familyStamp == 0; + } + + @Override + // called between every row. + public void reset() { + familyStamp = 0L; + deleteBuffer = null; + } + + @Override + // should not be called at all even (!) + public void update() { + this.reset(); + } + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java b/src/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java new file mode 100644 index 00000000000..2e398ed7720 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java @@ -0,0 +1,242 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.filter.Filter.ReturnCode; + +import java.io.IOException; +import java.util.NavigableSet; + +/** + * A query matcher that is specifically designed for the scan case. + */ +public class ScanQueryMatcher extends QueryMatcher { + + private Filter filter; + // have to support old style filter for now. + private RowFilterInterface oldFilter; + // Optimization so we can skip lots of compares when we decide to skip + // to the next row. + private boolean stickyNextRow; + private KeyValue stopKey = null; + + /** + * Constructs a QueryMatcher for a Scan. + * @param scan + * @param family + * @param columns + * @param ttl + * @param rowComparator + */ + public ScanQueryMatcher(Scan scan, byte [] family, + NavigableSet columns, long ttl, + KeyValue.KeyComparator rowComparator, int maxVersions) { + this.row = row; + this.tr = scan.getTimeRange(); + this.oldestStamp = System.currentTimeMillis() - ttl; + this.rowComparator = rowComparator; + // shouldn't this be ScanDeleteTracker? + this.deletes = new ScanDeleteTracker(rowComparator); + this.startKey = KeyValue.createFirstOnRow(scan.getStartRow()); + this.stopKey = KeyValue.createFirstOnRow(scan.getStopRow()); + this.filter = scan.getFilter(); + this.oldFilter = scan.getOldFilter(); + + // Single branch to deal with two types of reads (columns vs all in family) + if(columns == null || columns.size() == 0) { + // use a specialized scan for wildcard column tracker. + this.columns = new ScanWildcardColumnTracker(maxVersions); + } else { + // We can share the ExplicitColumnTracker, diff is we reset + // between rows, not between storefiles. + this.columns = new ExplicitColumnTracker(columns,maxVersions); + } + } + + /** + * Determines if the caller should do one of several things: + * - seek/skip to the next row (MatchCode.SEEK_NEXT_ROW) + * - seek/skip to the next column (MatchCode.SEEK_NEXT_COL) + * - include the current KeyValue (MatchCode.INCLUDE) + * - ignore the current KeyValue (MatchCode.SKIP) + * - got to the next row (MatchCode.DONE) + * + * @param kv KeyValue to check + * @return + * @throws IOException + */ + public MatchCode match(KeyValue kv) { + if (filter != null && filter.filterAllRemaining()) { + return MatchCode.DONE_SCAN; + } else if (oldFilter != null && oldFilter.filterAllRemaining()) { + // the old filter runs only if the other filter didnt work. + return MatchCode.DONE_SCAN; + } + + String kvStr = kv.toString(); + byte [] bytes = kv.getBuffer(); + int offset = kv.getOffset(); + int initialOffset = offset; + int kvLength = kv.getLength(); + + int keyLength = Bytes.toInt(bytes, offset, Bytes.SIZEOF_INT); + offset += KeyValue.ROW_OFFSET; + + short rowLength = Bytes.toShort(bytes, offset, Bytes.SIZEOF_SHORT); + offset += Bytes.SIZEOF_SHORT; + + int ret = this.rowComparator.compareRows(row, 0, row.length, + bytes, offset, rowLength); + if (ret <= -1) { + return MatchCode.DONE; + } else if (ret >= 1) { + // could optimize this, if necessary? + // Could also be called SEEK_TO_CURRENT_ROW, but this + // should be rare/never happens. + return MatchCode.SKIP; + } + + // optimize case. + if (this.stickyNextRow) + return MatchCode.SEEK_NEXT_ROW; + + // Give the row filter a chance to do it's job. + if (filter != null && filter.filterRowKey(bytes, offset, rowLength)) { + stickyNextRow = true; // optimize to keep from calling the filter too much. + return MatchCode.SEEK_NEXT_ROW; + } else if (oldFilter != null && oldFilter.filterRowKey(bytes, offset, rowLength)) { + stickyNextRow = true; + return MatchCode.SEEK_NEXT_ROW; + } + + + if (this.columns.done()) { + stickyNextRow = true; + return MatchCode.SEEK_NEXT_ROW; + } + + //Passing rowLength + offset += rowLength; + + //Skipping family + byte familyLength = bytes [offset]; + offset += familyLength + 1; + + int qualLength = keyLength + KeyValue.ROW_OFFSET - + (offset - initialOffset) - KeyValue.TIMESTAMP_TYPE_SIZE; + + long timestamp = kv.getTimestamp(); + if (isExpired(timestamp)) { + // done, the rest wil also be expired as well. + stickyNextRow = true; + return MatchCode.SEEK_NEXT_ROW; + } + + byte type = kv.getType(); + if (isDelete(type)) { + if (tr.withinOrAfterTimeRange(timestamp)) { + this.deletes.add(bytes, offset, qualLength, timestamp, type); + // Can't early out now, because DelFam come before any other keys + } + // May be able to optimize the SKIP here, if we matched + // due to a DelFam, we can skip to next row + // due to a DelCol, we can skip to next col + // But it requires more info out of isDelete(). + // needful -> million column challenge. + return MatchCode.SKIP; + } + + if (!tr.withinTimeRange(timestamp)) { + return MatchCode.SKIP; + } + + if (deletes.isDeleted(bytes, offset, + qualLength, timestamp)) { + return MatchCode.SKIP; + } + + MatchCode colChecker = + columns.checkColumn(bytes, offset, qualLength); + + // if SKIP -> SEEK_NEXT_COL + // if (NEXT,DONE) -> SEEK_NEXT_ROW + // if (INCLUDE) -> INCLUDE + if (colChecker == MatchCode.SKIP) { + return MatchCode.SEEK_NEXT_COL; + } else if (colChecker == MatchCode.NEXT || colChecker == MatchCode.DONE) { + stickyNextRow = true; + return MatchCode.SEEK_NEXT_ROW; + } + + // else INCLUDE + // if (colChecker == MatchCode.INCLUDE) + // give the filter a chance to run. + if (filter == null) + return MatchCode.INCLUDE; + + ReturnCode filterResponse = filter.filterKeyValue(kv); + if (filterResponse == ReturnCode.INCLUDE) + return MatchCode.INCLUDE; + + if (filterResponse == ReturnCode.SKIP) + return MatchCode.SKIP; + + // else + //if (filterResponse == ReturnCode.NEXT_ROW) + stickyNextRow = true; + return MatchCode.SEEK_NEXT_ROW; + } + + /** + * If the row was otherwise going to be included, call this to last-minute + * check. + * @return + */ + public boolean filterEntireRow() { + if (filter == null) + return false; + return filter.filterRow(); + } + + /** + * Set current row + * @param row + */ + @Override + public void setRow(byte [] row) { + this.row = row; + reset(); + } + + @Override + public void reset() { + super.reset(); + + stickyNextRow = false; + if (filter != null) + filter.reset(); + } +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java b/src/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java new file mode 100644 index 00000000000..5e0b6162fcd --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java @@ -0,0 +1,124 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Keeps track of the columns for a scan if they are not explicitly specified + */ +public class ScanWildcardColumnTracker implements ColumnTracker { + private byte [] columnBuffer = null; + private int columnOffset = 0; + private int columnLength = 0; + private int currentCount = 0; + private int maxVersions; + + /** + * Return maxVersions of every row. + * @param maxVersion + */ + public ScanWildcardColumnTracker(int maxVersion) { + this.maxVersions = maxVersion; + } + + /** + * Can only return INCLUDE or SKIP, since returning "NEXT" or + * "DONE" would imply we have finished with this row, when + * this class can't figure that out. + * + * @param bytes + * @param offset + * @param length + * @return + */ + @Override + public MatchCode checkColumn(byte[] bytes, int offset, int length) { + if (columnBuffer == null) { + // first iteration. + columnBuffer = bytes; + columnOffset = offset; + columnLength = length; + currentCount = 0; + + if (++currentCount > maxVersions) + return MatchCode.SKIP; + return MatchCode.INCLUDE; + } + int cmp = Bytes.compareTo(bytes, offset, length, + columnBuffer, columnOffset, columnLength); + if (cmp == 0) { + if (++currentCount > maxVersions) + return MatchCode.SKIP; // skip to next col + return MatchCode.INCLUDE; + } + + // new col > old col + if (cmp > 0) { + // switched columns, lets do something.x + columnBuffer = bytes; + columnOffset = offset; + columnLength = length; + currentCount = 0; + + if (++currentCount > maxVersions) + return MatchCode.SKIP; + return MatchCode.INCLUDE; + } + // new col < oldcol + // if (cmp < 0) { + throw new RuntimeException("ScanWildcardColumnTracker.checkColumn ran " + + "into a column actually smaller than the previous column!"); + } + + @Override + public void update() { + // no-op, shouldn't even be called + throw new UnsupportedOperationException( + "ScanWildcardColumnTracker.update should never be called!"); + } + + @Override + public void reset() { + columnBuffer = null; + } + + /** + * Used by matcher and scan/get to get a hint of the next column + * to seek to after checkColumn() returns SKIP. Returns the next interesting + * column we want, or NULL there is none (wildcard scanner). + * @return + */ + public ColumnCount getColumnHint() { + return null; + } + + + /** + * We can never know a-priori if we are done, so always return false. + * @return false + */ + @Override + public boolean done() { + return false; + } +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/Store.java b/src/java/org/apache/hadoop/hbase/regionserver/Store.java index a9166861d0c..ab932302745 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -24,21 +24,17 @@ import java.io.IOException; import java.io.UnsupportedEncodingException; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.NavigableMap; import java.util.NavigableSet; import java.util.Set; -import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.regex.Pattern; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -51,12 +47,13 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.KeyValue.KeyComparator; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.SequenceFile; import org.apache.hadoop.hbase.io.hfile.Compression; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileScanner; -import org.apache.hadoop.hbase.regionserver.HRegion.Counter; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.util.Progressable; @@ -171,7 +168,13 @@ public class Store implements HConstants { this.comparatorIgnoringType = this.comparator.getComparatorIgnoringType(); // getTimeToLive returns ttl in seconds. Convert to milliseconds. this.ttl = family.getTimeToLive(); - if (ttl != HConstants.FOREVER) { + if (ttl == HConstants.FOREVER) { + // default is unlimited ttl. + ttl = Long.MAX_VALUE; + } else if (ttl == -1) { + ttl = Long.MAX_VALUE; + } else { + // second -> ms adjust for user data this.ttl *= 1000; } this.memcache = new Memcache(this.ttl, this.comparator); @@ -304,9 +307,8 @@ public class Store implements HConstants { } // Check this edit is for me. Also, guard against writing the special // METACOLUMN info such as HBASE::CACHEFLUSH entries - if (/* Commented out for now -- St.Ack val.isTransactionEntry() ||*/ - val.matchingColumnNoDelimiter(HLog.METACOLUMN, - HLog.METACOLUMN.length - 1) || + if (/* commented out for now - stack via jgray key.isTransactionEntry() || */ + val.matchingFamily(HLog.METAFAMILY) || !Bytes.equals(key.getRegionName(), regioninfo.getRegionName()) || !val.matchingFamily(family.getName())) { continue; @@ -396,6 +398,21 @@ public class Store implements HConstants { lock.readLock().unlock(); } } + + /** + * Adds a value to the memcache + * + * @param kv + * @return memcache size delta + */ + protected long delete(final KeyValue kv) { + lock.readLock().lock(); + try { + return this.memcache.delete(kv); + } finally { + lock.readLock().unlock(); + } + } /** * @return All store files. @@ -476,7 +493,7 @@ public class Store implements HConstants { if (cache.size() == 0) { return null; } - long now = System.currentTimeMillis(); + long oldestTimestamp = System.currentTimeMillis() - ttl; // TODO: We can fail in the below block before we complete adding this // flush to list of store files. Add cleanup of anything put on filesystem // if we fail. @@ -486,7 +503,7 @@ public class Store implements HConstants { int entries = 0; try { for (KeyValue kv: cache) { - if (!isExpired(kv, ttl, now)) { + if (!isExpired(kv, oldestTimestamp)) { writer.append(kv); entries++; flushed += this.memcache.heapSize(kv, true); @@ -575,8 +592,10 @@ public class Store implements HConstants { * @param o Observer no longer interested in changes in set of Readers. */ void deleteChangedReaderObserver(ChangedReadersObserver o) { - if (!this.changedReaderObservers.remove(o)) { - LOG.warn("Not in set" + o); + if(this.changedReaderObservers.size() > 0) { + if (!this.changedReaderObservers.remove(o)) { + LOG.warn("Not in set" + o); + } } } @@ -793,140 +812,49 @@ public class Store implements HConstants { return result; } - /* - * @param r StoreFile list to reverse - * @return A new array of content of readers, reversed. - */ - private StoreFile [] reverse(final List r) { - List copy = new ArrayList(r); - Collections.reverse(copy); - // MapFile.Reader is instance of StoreFileReader so this should be ok. - return copy.toArray(new StoreFile[0]); - } - - /* - * @param rdrs List of StoreFiles - * @param keys Current keys - * @param done Which readers are done - * @return The lowest current key in passed rdrs - */ - private int getLowestKey(final HFileScanner [] rdrs, final KeyValue [] keys, - final boolean [] done) { - int lowestKey = -1; - for (int i = 0; i < rdrs.length; i++) { - if (done[i]) { - continue; - } - if (lowestKey < 0) { - lowestKey = i; - } else { - if (this.comparator.compare(keys[i], keys[lowestKey]) < 0) { - lowestKey = i; - } - } - } - return lowestKey; - } - - /* - * Compact a list of StoreFiles. + /** + * Do a minor/major compaction. Uses the scan infrastructure to make it easy. * - * We work by iterating through the readers in parallel looking at newest - * store file first. We always increment the lowest-ranked one. Updates to a - * single row/column will appear ranked by timestamp. - * @param compactedOut Where to write compaction. - * @param pReaders List of readers sorted oldest to newest. - * @param majorCompaction True to force a major compaction regardless of - * thresholds + * @param writer output writer + * @param filesToCompact which files to compact + * @param majorCompaction true to major compact (prune all deletes, max versions, etc) * @throws IOException */ - private void compact(final HFile.Writer compactedOut, - final List pReaders, final boolean majorCompaction) - throws IOException { - // Reverse order so newest store file is first. - StoreFile[] files = reverse(pReaders); - HFileScanner [] rdrs = new HFileScanner[files.length]; - KeyValue [] kvs = new KeyValue[rdrs.length]; - boolean[] done = new boolean[rdrs.length]; - // Now, advance through the readers in order. This will have the - // effect of a run-time sort of the entire dataset. - int numDone = 0; - for (int i = 0; i < rdrs.length; i++) { - rdrs[i] = files[i].getReader().getScanner(); - done[i] = !rdrs[i].seekTo(); - if (done[i]) { - numDone++; - } else { - kvs[i] = rdrs[i].getKeyValue(); - } + private void compact(HFile.Writer writer, + List filesToCompact, + boolean majorCompaction) throws IOException { + // for each file, obtain a scanner: + KeyValueScanner [] scanners = new KeyValueScanner[filesToCompact.size()]; + // init: + for(int i = 0; i < filesToCompact.size(); ++i) { + // TODO open a new HFile.Reader w/o block cache. + scanners[i] = new StoreFileScanner(filesToCompact.get(i).getReader().getScanner()); } - long now = System.currentTimeMillis(); - int timesSeen = 0; - KeyValue lastSeen = KeyValue.LOWESTKEY; - KeyValue lastDelete = null; - int maxVersions = family.getMaxVersions(); - while (numDone < done.length) { - // Get lowest key in all store files. - int lowestKey = getLowestKey(rdrs, kvs, done); - KeyValue kv = kvs[lowestKey]; - // If its same row and column as last key, increment times seen. - if (this.comparator.matchingRowColumn(lastSeen, kv)) { - timesSeen++; - // Reset last delete if not exact timestamp -- lastDelete only stops - // exactly the same key making it out to the compacted store file. - if (lastDelete != null && - lastDelete.getTimestamp() != kv.getTimestamp()) { - lastDelete = null; - } - } else { - timesSeen = 1; - lastDelete = null; - } - - // Don't write empty rows or columns. Only remove cells on major - // compaction. Remove if expired or > VERSIONS - if (kv.nonNullRowAndColumn()) { - if (!majorCompaction) { - // Write out all values if not a major compaction. - compactedOut.append(kv); - } else { - boolean expired = false; - boolean deleted = false; - if (timesSeen <= maxVersions && !(expired = isExpired(kv, ttl, now))) { - // If this value key is same as a deleted key, skip - if (lastDelete != null && - this.comparatorIgnoringType.compare(kv, lastDelete) == 0) { - deleted = true; - } else if (kv.isDeleteType()) { - // If a deleted value, skip - deleted = true; - lastDelete = kv; - } else { - compactedOut.append(kv); - } - } - if (expired || deleted) { - // HBASE-855 remove one from timesSeen because it did not make it - // past expired check -- don't count against max versions. - timesSeen--; - } - } - } - - // Update last-seen items - lastSeen = kv; - - // Advance the smallest key. If that reader's all finished, then - // mark it as done. - if (!rdrs[lowestKey].next()) { - done[lowestKey] = true; - rdrs[lowestKey] = null; - numDone++; - } else { - kvs[lowestKey] = rdrs[lowestKey].getKeyValue(); - } + InternalScanner scanner; + if (majorCompaction) { + Scan scan = new Scan(); + scan.setMaxVersions(family.getMaxVersions()); + // TODO pass in the scanners/store files. + scanner = new StoreScanner(this, scan, null); + } else { + scanner = new MinorCompactingStoreScanner(this, scanners); } + + // since scanner.next() can return 'false' but still be delivering data, + // we have to use a do/while loop. + ArrayList row = new ArrayList(); + boolean more = true; + while ( more ) { + more = scanner.next(row); + // output to writer: + for (KeyValue kv : row) { + writer.append(kv); + } + row.clear(); + } + + scanner.close(); } /* @@ -1007,321 +935,25 @@ public class Store implements HConstants { // Accessors. // (This is the only section that is directly useful!) ////////////////////////////////////////////////////////////////////////////// + /** + * @return the number of files in this store + */ + public int getNumberOfstorefiles() { + return this.storefiles.size(); + } - /** - * 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. - * @param key - Where to start searching. Specifies a row. - * Columns are specified in following arguments. - * @param columns Can be null which means get all - * @param columnPattern Can be null. - * @param numVersions - * @param versionsCounter Can be null. - * @param keyvalues - * @param now - Where to start searching. Specifies a timestamp. - * @throws IOException - */ - public void getFull(KeyValue key, final NavigableSet columns, - final Pattern columnPattern, - final int numVersions, Map versionsCounter, - List keyvalues, final long now) - throws IOException { - // if the key is null, we're not even looking for anything. return. - if (key == null) { - return; - } - int versions = versionsToReturn(numVersions); - NavigableSet deletes = - new TreeSet(this.comparatorIgnoringType); - // Create a Map that has results by column so we can keep count of versions. - // It duplicates columns but doing check of columns, we don't want to make - // column set each time. - this.lock.readLock().lock(); - try { - // get from the memcache first. - if (this.memcache.getFull(key, columns, columnPattern, versions, - versionsCounter, deletes, keyvalues, now)) { - // May have gotten enough results, enough to return. - return; - } - Map m = this.storefiles.descendingMap(); - for (Iterator> i = m.entrySet().iterator(); - i.hasNext();) { - if (getFullFromStoreFile(i.next().getValue(), key, columns, - columnPattern, versions, versionsCounter, deletes, keyvalues)) { - return; - } - } - } finally { - this.lock.readLock().unlock(); - } - } - - /* - * @param f - * @param key Where to start searching. Specifies a row and timestamp. - * Columns are specified in following arguments. - * @param columns - * @param versions - * @param versionCounter - * @param deletes - * @param keyvalues - * @return True if we found enough results to satisfy the versions - * and columns passed. - * @throws IOException - */ - private boolean getFullFromStoreFile(StoreFile f, KeyValue target, - Set columns, final Pattern columnPattern, int versions, - Map versionCounter, - NavigableSet deletes, - List keyvalues) - throws IOException { - long now = System.currentTimeMillis(); - HFileScanner scanner = f.getReader().getScanner(); - if (!getClosest(scanner, target)) { - return false; - } - boolean hasEnough = false; - do { - KeyValue kv = scanner.getKeyValue(); - // Make sure we have not passed out the row. If target key has a - // column on it, then we are looking explicit key+column combination. If - // we've passed it out, also break. - if (target.isEmptyColumn()? !this.comparator.matchingRows(target, kv): - !this.comparator.matchingRowColumn(target, kv)) { - break; - } - if (!Store.getFullCheck(this.comparator, target, kv, columns, columnPattern)) { - continue; - } - if (Store.doKeyValue(kv, versions, versionCounter, columns, deletes, now, - this.ttl, keyvalues, null)) { - hasEnough = true; - break; - } - } while (scanner.next()); - return hasEnough; - } - - /** - * Code shared by {@link Memcache#getFull(KeyValue, NavigableSet, Pattern, int, Map, NavigableSet, List, long)} - * and {@link #getFullFromStoreFile(StoreFile, KeyValue, Set, Pattern, int, Map, NavigableSet, List)} - * @param c - * @param target - * @param candidate - * @param columns - * @param columnPattern - * @return True if candidate matches column and timestamp. - */ - static boolean getFullCheck(final KeyValue.KVComparator c, - final KeyValue target, final KeyValue candidate, - final Set columns, final Pattern columnPattern) { - // Does column match? - if (!Store.matchingColumns(candidate, columns)) { - return false; - } - // if the column pattern is not null, we use it for column matching. - // we will skip the keys whose column doesn't match the pattern. - if (columnPattern != null) { - if (!(columnPattern.matcher(candidate.getColumnString()).matches())) { - return false; - } - } - if (c.compareTimestamps(target, candidate) > 0) { - return false; - } - return true; - } /* * @param wantedVersions How many versions were asked for. * @return wantedVersions or this families' VERSIONS. */ - private int versionsToReturn(final int wantedVersions) { + int versionsToReturn(final int wantedVersions) { if (wantedVersions <= 0) { throw new IllegalArgumentException("Number of versions must be > 0"); } // Make sure we do not return more than maximum versions for this store. int maxVersions = this.family.getMaxVersions(); - return wantedVersions > maxVersions && - wantedVersions != HConstants.ALL_VERSIONS? maxVersions: wantedVersions; - } - - /** - * Get the value for the indicated HStoreKey. Grab the target value and the - * previous numVersions - 1 values, as well. - * - * Use {@link HConstants.ALL_VERSIONS} to retrieve all versions. - * @param key - * @param numVersions Number of versions to fetch. Must be > 0. - * @return values for the specified versions - * @throws IOException - */ - List get(final KeyValue key, final int numVersions) - throws IOException { - // This code below is very close to the body of the getKeys method. Any - // changes in the flow below should also probably be done in getKeys. - // TODO: Refactor so same code used. - long now = System.currentTimeMillis(); - int versions = versionsToReturn(numVersions); - // Keep a list of deleted cell keys. We need this because as we go through - // the memcache and store files, the cell with the delete marker may be - // in one store and the old non-delete cell value in a later store. - // If we don't keep around the fact that the cell was deleted in a newer - // record, we end up returning the old value if user is asking for more - // than one version. This List of deletes should not be large since we - // are only keeping rows and columns that match those set on the get and - // which have delete values. If memory usage becomes an issue, could - // redo as bloom filter. Use sorted set because test for membership should - // be faster than calculating a hash. Use a comparator that ignores ts. - NavigableSet deletes = - new TreeSet(this.comparatorIgnoringType); - List keyvalues = new ArrayList(); - this.lock.readLock().lock(); - try { - // Check the memcache - if (this.memcache.get(key, versions, keyvalues, deletes, now)) { - return keyvalues; - } - Map m = this.storefiles.descendingMap(); - boolean hasEnough = false; - for (Map.Entry e: m.entrySet()) { - StoreFile f = e.getValue(); - HFileScanner scanner = f.getReader().getScanner(); - if (!getClosest(scanner, key)) { - // Move to next file. - continue; - } - do { - KeyValue kv = scanner.getKeyValue(); - // Make sure below matches what happens up in Memcache#get. - if (this.comparator.matchingRowColumn(kv, key)) { - if (doKeyValue(kv, versions, deletes, now, this.ttl, keyvalues, null)) { - hasEnough = true; - break; - } - } else { - // Row and column don't match. Must have gone past. Move to next file. - break; - } - } while (scanner.next()); - if (hasEnough) { - break; // Break out of files loop. - } - } - return keyvalues.isEmpty()? null: keyvalues; - } finally { - this.lock.readLock().unlock(); - } - } - - /* - * Small method to check if we are over the max number of versions - * or we acheived this family max versions. - * The later happens when we have the situation described in HBASE-621. - * @param versions - * @param c - * @return - */ - static boolean hasEnoughVersions(final int versions, final List c) { - return versions > 0 && !c.isEmpty() && c.size() >= versions; - } - - /* - * Used when doing getFulls. - * @param kv - * @param versions - * @param versionCounter - * @param columns - * @param deletes - * @param now - * @param ttl - * @param keyvalues - * @param set - * @return True if enough versions. - */ - static boolean doKeyValue(final KeyValue kv, - final int versions, - final Map versionCounter, - final Set columns, - final NavigableSet deletes, - final long now, - final long ttl, - final List keyvalues, - final SortedSet set) { - boolean hasEnough = false; - if (kv.isDeleteType()) { - if (!deletes.contains(kv)) { - deletes.add(kv); - } - } else if (!deletes.contains(kv)) { - // Skip expired cells - if (!isExpired(kv, ttl, now)) { - if (HRegion.okToAddResult(kv, versions, versionCounter)) { - HRegion.addResult(kv, versionCounter, keyvalues); - if (HRegion.hasEnoughVersions(versions, versionCounter, columns)) { - hasEnough = true; - } - } - } else { - // Remove the expired. - Store.expiredOrDeleted(set, kv); - } - } - return hasEnough; - } - - /* - * Used when doing get. - * @param kv - * @param versions - * @param deletes - * @param now - * @param ttl - * @param keyvalues - * @param set - * @return True if enough versions. - */ - static boolean doKeyValue(final KeyValue kv, final int versions, - final NavigableSet deletes, - final long now, final long ttl, - final List keyvalues, final SortedSet set) { - boolean hasEnough = false; - if (!kv.isDeleteType()) { - // Filter out expired results - if (notExpiredAndNotInDeletes(ttl, kv, now, deletes)) { - if (!keyvalues.contains(kv)) { - keyvalues.add(kv); - if (hasEnoughVersions(versions, keyvalues)) { - hasEnough = true; - } - } - } else { - if (set != null) { - expiredOrDeleted(set, kv); - } - } - } else { - // Cell holds a delete value. - deletes.add(kv); - } - return hasEnough; - } - - /* - * Test that the target matches the origin. If the origin - * has an empty column, then it just tests row equivalence. Otherwise, it uses - * HStoreKey.matchesRowCol(). - * @param c Comparator to use. - * @param origin Key we're testing against - * @param target Key we're testing - */ - static boolean matchingRowColumn(final KeyValue.KVComparator c, - final KeyValue origin, final KeyValue target) { - return origin.isEmptyColumn()? c.matchingRows(target, origin): - c.matchingRowColumn(target, origin); + return wantedVersions > maxVersions ? maxVersions: wantedVersions; } static void expiredOrDeleted(final Set set, final KeyValue kv) { @@ -1411,13 +1043,12 @@ public class Store implements HConstants { */ static boolean notExpiredAndNotInDeletes(final long ttl, final KeyValue key, final long now, final Set deletes) { - return !isExpired(key, ttl, now) && (deletes == null || deletes.isEmpty() || + return !isExpired(key, now-ttl) && (deletes == null || deletes.isEmpty() || !deletes.contains(key)); } - static boolean isExpired(final KeyValue key, final long ttl, - final long now) { - return ttl != HConstants.FOREVER && now > key.getTimestamp() + ttl; + static boolean isExpired(final KeyValue key, final long oldestTimestamp) { + return key.getTimestamp() < oldestTimestamp; } /* Find a candidate for row that is at or before passed key, searchkey, in hfile. @@ -1693,13 +1324,12 @@ public class Store implements HConstants { /** * Return a scanner for both the memcache and the HStore files */ - protected InternalScanner getScanner(long timestamp, - final NavigableSet targetCols, - byte [] firstRow, RowFilterInterface filter) + protected KeyValueScanner getScanner(Scan scan, + final NavigableSet targetCols) throws IOException { lock.readLock().lock(); try { - return new StoreScanner(this, targetCols, firstRow, timestamp, filter); + return new StoreScanner(this, scan, targetCols); } finally { lock.readLock().unlock(); } @@ -1722,7 +1352,7 @@ public class Store implements HConstants { * @throws IOException if there was a problem getting file sizes from the * filesystem */ - long getStorefilesIndexSize() throws IOException { + long getStorefilesIndexSize() { long size = 0; for (StoreFile s: storefiles.values()) size += s.getReader().indexSize(); @@ -1805,4 +1435,114 @@ public class Store implements HConstants { } return false; } + + // + // HBASE-880/1249/1304 + // + + /** + * Retrieve results from this store given the specified Get parameters. + * @param get Get operation + * @param columns List of columns to match, can be empty (not null) + * @param result List to add results to + * @throws IOException + */ + public void get(Get get, NavigableSet columns, List result) + throws IOException { + KeyComparator keyComparator = this.comparator.getRawComparator(); + + // Column matching and version enforcement + QueryMatcher matcher = new QueryMatcher(get, get.getRow(), + this.family.getName(), columns, this.ttl, keyComparator, + versionsToReturn(get.getMaxVersions())); + + // Read from Memcache + if(this.memcache.get(matcher, result)) { + // Received early-out from memcache + return; + } + + // Check if we even have storefiles + if(this.storefiles.isEmpty()) { + return; + } + + // Get storefiles for this store + List storefileScanners = new ArrayList(); + for(StoreFile sf : this.storefiles.descendingMap().values()) { + storefileScanners.add(sf.getReader().getScanner()); + } + + // StoreFileGetScan will handle reading this store's storefiles + StoreFileGetScan scanner = new StoreFileGetScan(storefileScanners, matcher); + + // Run a GET scan and put results into the specified list + scanner.get(result); + } + + /** + * Increments the value for the given row/family/qualifier + * @param row + * @param family + * @param qualifier + * @param amount + * @return + * @throws IOException + */ + public long incrementColumnValue(byte [] row, byte [] family, + byte [] qualifier, long amount) throws IOException{ + long value = 0; + List result = new ArrayList(); + KeyComparator keyComparator = this.comparator.getRawComparator(); + + // Setting up the QueryMatcher + Get get = new Get(row); + NavigableSet qualifiers = + new TreeSet(Bytes.BYTES_COMPARATOR); + qualifiers.add(qualifier); + QueryMatcher matcher = new QueryMatcher(get, row, family, qualifiers, + this.ttl, keyComparator, 1); + + // Read from Memcache + if(this.memcache.get(matcher, result)) { + // Received early-out from memcache + KeyValue kv = result.get(0); + byte [] buffer = kv.getBuffer(); + int valueOffset = kv.getValueOffset(); + value = Bytes.toLong(buffer, valueOffset, Bytes.SIZEOF_LONG) + amount; + Bytes.putBytes(buffer, valueOffset, Bytes.toBytes(value), 0, + Bytes.SIZEOF_LONG); + return value; + } + + // Check if we even have storefiles + if(this.storefiles.isEmpty()) { + return addNewKeyValue(row, family, qualifier, value, amount); + } + + // Get storefiles for this store + List storefileScanners = new ArrayList(); + for(StoreFile sf : this.storefiles.descendingMap().values()) { + storefileScanners.add(sf.getReader().getScanner()); + } + + // StoreFileGetScan will handle reading this store's storefiles + StoreFileGetScan scanner = new StoreFileGetScan(storefileScanners, matcher); + + // Run a GET scan and put results into the specified list + scanner.get(result); + if(result.size() > 0) { + value = Bytes.toLong(result.get(0).getValue()); + } + return addNewKeyValue(row, family, qualifier, value, amount); + } + + private long addNewKeyValue(byte [] row, byte [] family, byte [] qualifier, + long value, long amount) { + long newValue = value + amount; + KeyValue newKv = new KeyValue(row, family, qualifier, Bytes.toBytes(newValue)); + add(newKv); + return newValue; + } + } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/src/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 7940c0cc1a7..453ac9c5a27 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -40,10 +40,7 @@ import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.Compression; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.LruBlockCache; -import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Hash; -import org.apache.hadoop.io.RawComparator; /** * A Store data file. Stores usually have one or more of these files. They @@ -58,7 +55,7 @@ import org.apache.hadoop.io.RawComparator; public class StoreFile implements HConstants { static final Log LOG = LogFactory.getLog(StoreFile.class.getName()); - public static final String HFILE_CACHE_SIZE_KEY = "hfile.block.cache.size"; + private static final String HFILE_CACHE_SIZE_KEY = "hfile.block.cache.size"; private static BlockCache hfileBlockCache = null; @@ -100,14 +97,15 @@ public class StoreFile implements HConstants { private final HBaseConfiguration conf; /** - * Constructor, loads a reader and it's indices, etc. May allocate a substantial - * amount of ram depending on the underlying files (10-20MB?). + * Constructor, loads a reader and it's indices, etc. May allocate a + * substantial amount of ram depending on the underlying files (10-20MB?). * @param fs * @param p * @param conf * @throws IOException */ - StoreFile(final FileSystem fs, final Path p, final HBaseConfiguration conf) throws IOException { + StoreFile(final FileSystem fs, final Path p, final HBaseConfiguration conf) + throws IOException { this.conf = conf; this.fs = fs; this.path = p; @@ -208,6 +206,11 @@ public class StoreFile implements HConstants { return this.sequenceid; } + /** + * + * @param conf + * @return + */ public static synchronized BlockCache getBlockCache(HBaseConfiguration conf) { if (hfileBlockCache != null) return hfileBlockCache; @@ -221,6 +224,9 @@ public class StoreFile implements HConstants { return hfileBlockCache; } + /** + * @return the blockcache + */ public BlockCache getBlockCache() { return getBlockCache(conf); } @@ -237,8 +243,8 @@ public class StoreFile implements HConstants { throw new IllegalAccessError("Already open"); } if (isReference()) { - this.reader = new HalfHFileReader(this.fs, this.referencePath, getBlockCache(), - this.reference); + this.reader = new HalfHFileReader(this.fs, this.referencePath, + getBlockCache(), this.reference); } else { this.reader = new StoreFileReader(this.fs, this.path, getBlockCache()); } @@ -276,6 +282,13 @@ public class StoreFile implements HConstants { * Override to add some customization on HFile.Reader */ static class StoreFileReader extends HFile.Reader { + /** + * + * @param fs + * @param path + * @param cache + * @throws IOException + */ public StoreFileReader(FileSystem fs, Path path, BlockCache cache) throws IOException { super(fs, path, cache); @@ -296,6 +309,14 @@ public class StoreFile implements HConstants { * Override to add some customization on HalfHFileReader. */ static class HalfStoreFileReader extends HalfHFileReader { + /** + * + * @param fs + * @param p + * @param c + * @param r + * @throws IOException + */ public HalfStoreFileReader(FileSystem fs, Path p, BlockCache c, Reference r) throws IOException { super(fs, p, c, r); @@ -445,7 +466,6 @@ public class StoreFile implements HConstants { * @param dir * @param suffix * @return Path to a file that doesn't exist at time of this invocation. - * @return * @throws IOException */ static Path getRandomFilename(final FileSystem fs, final Path dir, @@ -465,8 +485,8 @@ public class StoreFile implements HConstants { * Write file metadata. * Call before you call close on the passed w since its written * as metadata to that file. - * - * @param w + * + * @param w hfile writer * @param maxSequenceId Maximum sequence id. * @throws IOException */ diff --git a/src/java/org/apache/hadoop/hbase/regionserver/StoreFileGetScan.java b/src/java/org/apache/hadoop/hbase/regionserver/StoreFileGetScan.java new file mode 100644 index 00000000000..aa73ee5434a --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/StoreFileGetScan.java @@ -0,0 +1,110 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.io.hfile.HFileScanner; + +/** + * Use to execute a get by scanning all the store files in order. + */ +public class StoreFileGetScan { + + private List scanners; + private QueryMatcher matcher; + + private KeyValue startKey; + + /** + * Constructor + * @param scanners + * @param matcher + */ + public StoreFileGetScan(List scanners, QueryMatcher matcher) { + this.scanners = scanners; + this.matcher = matcher; + this.startKey = matcher.getStartKey(); + } + + /** + * Performs a GET operation across multiple StoreFiles. + *

                + * This style of StoreFile scanning goes through each + * StoreFile in its entirety, most recent first, before + * proceeding to the next StoreFile. + *

                + * This strategy allows for optimal, stateless (no persisted Scanners) + * early-out scenarios. + * @param result List to add results to + * @throws IOException + */ + public void get(List result) throws IOException { + for(HFileScanner scanner : this.scanners) { + this.matcher.update(); + if(getStoreFile(scanner, result) || matcher.isDone()) { + return; + } + } + } + + /** + * Performs a GET operation on a single StoreFile. + * @param scanner + * @param result + * @return true if done with this store, false if must continue to next + * @throws IOException + */ + public boolean getStoreFile(HFileScanner scanner, List result) + throws IOException { + if(scanner.seekTo(startKey.getBuffer(), startKey.getKeyOffset(), + startKey.getKeyLength()) == -1) { + // No keys in StoreFile at or after specified startKey + // First row may be = our row, so we have to check anyways. + byte [] firstKey = scanner.getReader().getFirstKey(); + short rowLen = Bytes.toShort(firstKey, 0, Bytes.SIZEOF_SHORT); + int rowOffset = Bytes.SIZEOF_SHORT; + if (this.matcher.rowComparator.compareRows(firstKey, rowOffset, rowLen, + startKey.getBuffer(), startKey.getRowOffset(), startKey.getRowLength()) + != 0) + return false; + scanner.seekTo(); + } + do { + KeyValue kv = scanner.getKeyValue(); + switch(matcher.match(kv)) { + case INCLUDE: + result.add(kv); + break; + case SKIP: + break; + case NEXT: + return false; + case DONE: + return true; + } + } while(scanner.next()); + return false; + } + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java index 7f44f73327a..1f5ea694284 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java @@ -1,5 +1,5 @@ /** - * Copyright 2008 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -21,306 +21,81 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.NavigableSet; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.hfile.HFileScanner; /** - * A scanner that iterates through HStore files + * A KeyValue scanner that iterates over a single HFile */ -class StoreFileScanner extends HAbstractScanner -implements ChangedReadersObserver { - // Keys retrieved from the sources - private volatile KeyValue keys[]; +class StoreFileScanner implements KeyValueScanner { - // Readers we go against. - private volatile HFileScanner [] scanners; + private HFileScanner hfs; + private KeyValue cur = null; - // Store this scanner came out of. - private final Store store; - - // Used around replacement of Readers if they change while we're scanning. - private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - - private final long now = System.currentTimeMillis(); - /** - * @param store - * @param timestamp - * @param columns - * @param firstRow - * @param deletes Set of running deletes - * @throws IOException + * Implements a {@link KeyValueScanner} on top of the specified {@link HFileScanner} + * @param hfs HFile scanner */ - public StoreFileScanner(final Store store, final long timestamp, - final NavigableSet columns, final byte [] firstRow) - throws IOException { - super(timestamp, columns); - this.store = store; - this.store.addChangedReaderObserver(this); + public StoreFileScanner(HFileScanner hfs) { + this.hfs = hfs; + } + + public KeyValue peek() { + return cur; + } + + public KeyValue next() { + KeyValue retKey = cur; + cur = hfs.getKeyValue(); try { - openScanner(firstRow); - } catch (Exception ex) { - close(); - IOException e = new IOException("HStoreScanner failed construction"); - e.initCause(ex); - throw e; + hfs.next(); + } catch(IOException e) { + // Only occurs if the scanner is not seeked, this is never the case + // as we seek immediately after construction in StoreScanner } + return retKey; } - - /* - * Go open new scanners and cue them at firstRow. - * Closes existing Readers if any. - * @param firstRow - * @throws IOException - */ - private void openScanner(final byte [] firstRow) throws IOException { - List s = - new ArrayList(this.store.getStorefiles().size()); - Map map = this.store.getStorefiles().descendingMap(); - for (StoreFile f: map.values()) { - s.add(f.getReader().getScanner()); - } - this.scanners = s.toArray(new HFileScanner [] {}); - this.keys = new KeyValue[this.scanners.length]; - // Advance the readers to the first pos. - KeyValue firstKey = (firstRow != null && firstRow.length > 0)? - new KeyValue(firstRow, HConstants.LATEST_TIMESTAMP): null; - for (int i = 0; i < this.scanners.length; i++) { - if (firstKey != null) { - if (seekTo(i, firstKey)) { - continue; - } - } - while (getNext(i)) { - if (columnMatch(i)) { - break; - } - } - } - } - - /** - * For a particular column i, find all the matchers defined for the column. - * Compare the column family and column key using the matchers. The first one - * that matches returns true. If no matchers are successful, return false. - * - * @param i index into the keys array - * @return true if any of the matchers for the column match the column family - * and the column key. - * @throws IOException - */ - boolean columnMatch(int i) throws IOException { - return columnMatch(keys[i]); - } - - /** - * Get the next set of values for this scanner. - * - * @param key The key that matched - * @param results All the results for key - * @return true if a match was found - * @throws IOException - * - * @see org.apache.hadoop.hbase.regionserver.InternalScanner#next(org.apache.hadoop.hbase.HStoreKey, java.util.SortedMap) - */ - @Override - public boolean next(List results) - throws IOException { - if (this.scannerClosed) { - return false; - } - this.lock.readLock().lock(); + + public boolean seek(KeyValue key) { try { - // Find the next viable row label (and timestamp). - KeyValue viable = getNextViableRow(); - if (viable == null) { + if(!seekAtOrAfter(hfs, key)) { + close(); return false; } - - // Grab all the values that match this row/timestamp - boolean addedItem = false; - for (int i = 0; i < keys.length; i++) { - // Fetch the data - while ((keys[i] != null) && - (this.store.comparator.compareRows(this.keys[i], viable) == 0)) { - // If we are doing a wild card match or there are multiple matchers - // per column, we need to scan all the older versions of this row - // to pick up the rest of the family members - if(!isWildcardScanner() - && !isMultipleMatchScanner() - && (keys[i].getTimestamp() != viable.getTimestamp())) { - break; - } - if (columnMatch(i)) { - // We only want the first result for any specific family member - // TODO: Do we have to keep a running list of column entries in - // the results across all of the StoreScanner? Like we do - // doing getFull? - if (!results.contains(keys[i])) { - results.add(keys[i]); - addedItem = true; - } - } - - if (!getNext(i)) { - closeSubScanner(i); - } - } - // Advance the current scanner beyond the chosen row, to - // a valid timestamp, so we're ready next time. - while ((keys[i] != null) && - ((this.store.comparator.compareRows(this.keys[i], viable) <= 0) || - (keys[i].getTimestamp() > this.timestamp) || - !columnMatch(i))) { - getNext(i); - } - } - return addedItem; - } finally { - this.lock.readLock().unlock(); + cur = hfs.getKeyValue(); + hfs.next(); + return true; + } catch(IOException ioe) { + close(); + return false; } } - - /* - * @return An instance of ViableRow + + public void close() { + // Nothing to close on HFileScanner? + cur = null; + } + + /** + * + * @param s + * @param k + * @return * @throws IOException */ - private KeyValue getNextViableRow() throws IOException { - // Find the next viable row label (and timestamp). - KeyValue viable = null; - long viableTimestamp = -1; - long ttl = store.ttl; - for (int i = 0; i < keys.length; i++) { - // The first key that we find that matches may have a timestamp greater - // than the one we're looking for. We have to advance to see if there - // is an older version present, since timestamps are sorted descending - while (keys[i] != null && - keys[i].getTimestamp() > this.timestamp && - columnMatch(i) && - getNext(i)) { - if (columnMatch(i)) { - break; - } - } - if((keys[i] != null) - // If we get here and keys[i] is not null, we already know that the - // column matches and the timestamp of the row is less than or equal - // to this.timestamp, so we do not need to test that here - && ((viable == null) || - (this.store.comparator.compareRows(this.keys[i], viable) < 0) || - ((this.store.comparator.compareRows(this.keys[i], viable) == 0) && - (keys[i].getTimestamp() > viableTimestamp)))) { - if (ttl == HConstants.FOREVER || now < keys[i].getTimestamp() + ttl) { - viable = keys[i]; - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("getNextViableRow :" + keys[i] + ": expired, skipped"); - } - } - } - } - return viable; - } - - /* - * The user didn't want to start scanning at the first row. This method - * seeks to the requested row. - * - * @param i which iterator to advance - * @param firstRow seek to this row - * @return true if we found the first row and so the scanner is properly - * primed or true if the row was not found and this scanner is exhausted. - */ - private boolean seekTo(int i, final KeyValue firstKey) + public static boolean seekAtOrAfter(HFileScanner s, KeyValue k) throws IOException { - if (firstKey == null) { - if (!this.scanners[i].seekTo()) { - closeSubScanner(i); - return true; - } - } else { - // TODO: sort columns and pass in column as part of key so we get closer. - if (!Store.getClosest(this.scanners[i], firstKey)) { - closeSubScanner(i); - return true; - } - } - this.keys[i] = this.scanners[i].getKeyValue(); - return isGoodKey(this.keys[i]); - } - - /** - * Get the next value from the specified reader. - * - * @param i which reader to fetch next value from - * @return true if there is more data available - */ - private boolean getNext(int i) throws IOException { - boolean result = false; - while (true) { - if ((this.scanners[i].isSeeked() && !this.scanners[i].next()) || - (!this.scanners[i].isSeeked() && !this.scanners[i].seekTo())) { - closeSubScanner(i); - break; - } - this.keys[i] = this.scanners[i].getKeyValue(); - if (isGoodKey(this.keys[i])) { - result = true; - break; - } - } - return result; - } - - /* - * @param kv - * @return True if good key candidate. - */ - private boolean isGoodKey(final KeyValue kv) { - return !Store.isExpired(kv, this.store.ttl, this.now); - } - - /** Close down the indicated reader. */ - private void closeSubScanner(int i) { - this.scanners[i] = null; - this.keys[i] = null; - } - - /** Shut it down! */ - public void close() { - if (!this.scannerClosed) { - this.store.deleteChangedReaderObserver(this); - try { - for(int i = 0; i < this.scanners.length; i++) { - closeSubScanner(i); - } - } finally { - this.scannerClosed = true; - } - } - } - - // Implementation of ChangedReadersObserver - - public void updateReaders() throws IOException { - this.lock.writeLock().lock(); - try { - // The keys are currently lined up at the next row to fetch. Pass in - // the current row as 'first' row and readers will be opened and cue'd - // up so future call to next will start here. - KeyValue viable = getNextViableRow(); - openScanner(viable.getRow()); - LOG.debug("Replaced Scanner Readers at row " + - viable.getRow().toString()); - } finally { - this.lock.writeLock().unlock(); + int result = s.seekTo(k.getBuffer(), k.getKeyOffset(), k.getKeyLength()); + if(result < 0) { + // Passed KV is smaller than first KV in file, work from start of file + return s.seekTo(); + } else if(result > 0) { + // Passed KV is larger than current KV in file, if there is a next + // it is the "after", if not then this scanner is done. + return s.next(); } + // Seeked to the exact key + return true; } } \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index 5d0bdc4c654..a36c6795bb6 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -1,5 +1,5 @@ /** - * Copyright 2008 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -25,288 +25,238 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.NavigableSet; -import java.util.TreeSet; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.filter.RowFilterInterface; -import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.io.hfile.HFileScanner; /** - * Scanner scans both the memcache and the HStore + * Scanner scans both the memcache and the HStore. Coaleace KeyValue stream + * into List for a single row. */ -class StoreScanner implements InternalScanner, ChangedReadersObserver { +class StoreScanner implements KeyValueScanner, InternalScanner, +ChangedReadersObserver { static final Log LOG = LogFactory.getLog(StoreScanner.class); - private InternalScanner [] scanners; - private List [] resultSets; - private boolean wildcardMatch = false; - private boolean multipleMatchers = false; - private RowFilterInterface dataFilter; private Store store; - private final long timestamp; - private final NavigableSet columns; - - // Indices for memcache scanner and hstorefile scanner. - private static final int MEMS_INDEX = 0; - private static final int HSFS_INDEX = MEMS_INDEX + 1; - + + private ScanQueryMatcher matcher; + + private KeyValueHeap heap; + // Used around transition from no storefile to the first. private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); // Used to indicate that the scanner has closed (see HBASE-1107) private final AtomicBoolean closing = new AtomicBoolean(false); - /** Create an Scanner with a handle on the memcache and HStore files. */ - @SuppressWarnings("unchecked") - StoreScanner(Store store, final NavigableSet targetCols, - byte [] firstRow, long timestamp, RowFilterInterface filter) - throws IOException { + /** + * Opens a scanner across memcache, snapshot, and all StoreFiles. + */ + StoreScanner(Store store, Scan scan, final NavigableSet columns) { this.store = store; - this.dataFilter = filter; - if (null != dataFilter) { - dataFilter.reset(); - } - this.scanners = new InternalScanner[2]; - this.resultSets = new List[scanners.length]; - // Save these args in case we need them later handling change in readers - // See updateReaders below. - this.timestamp = timestamp; - this.columns = targetCols; - try { - scanners[MEMS_INDEX] = - store.memcache.getScanner(timestamp, targetCols, firstRow); - scanners[HSFS_INDEX] = - new StoreFileScanner(store, timestamp, targetCols, firstRow); - for (int i = MEMS_INDEX; i < scanners.length; i++) { - checkScannerFlags(i); - } - } catch (IOException e) { - doClose(); - throw e; - } - - // Advance to the first key in each scanner. - // All results will match the required column-set and scanTime. - for (int i = MEMS_INDEX; i < scanners.length; i++) { - setupScanner(i); + matcher = new ScanQueryMatcher(scan, store.getFamily().getName(), + columns, store.ttl, store.comparator.getRawComparator(), + store.versionsToReturn(scan.getMaxVersions())); + + List scanners = getStoreFileScanners(); + scanners.add(store.memcache.getScanner()); + + // Seek all scanners to the initial key + for(KeyValueScanner scanner : scanners) { + scanner.seek(matcher.getStartKey()); } + + // Combine all seeked scanners with a heap + heap = new KeyValueHeap( + scanners.toArray(new KeyValueScanner[scanners.size()]), store.comparator); + this.store.addChangedReaderObserver(this); } - - /* - * @param i Index. - */ - private void checkScannerFlags(final int i) { - if (this.scanners[i].isWildcardScanner()) { - this.wildcardMatch = true; - } - if (this.scanners[i].isMultipleMatchScanner()) { - this.multipleMatchers = true; - } - } - - /* - * Do scanner setup. - * @param i - * @throws IOException - */ - private void setupScanner(final int i) throws IOException { - this.resultSets[i] = new ArrayList(); - if (this.scanners[i] != null && !this.scanners[i].next(this.resultSets[i])) { - closeScanner(i); + + // Constructor for testing. + StoreScanner(Scan scan, byte [] colFamily, + long ttl, KeyValue.KVComparator comparator, + final NavigableSet columns, + KeyValueScanner [] scanners) { + this.store = null; + this.matcher = new ScanQueryMatcher(scan, colFamily, columns, ttl, + comparator.getRawComparator(), scan.getMaxVersions()); + + // Seek all scanners to the initial key + for(KeyValueScanner scanner : scanners) { + scanner.seek(matcher.getStartKey()); } + + heap = new KeyValueHeap( + scanners, comparator); } - /** @return true if the scanner is a wild card scanner */ - public boolean isWildcardScanner() { - return this.wildcardMatch; + public KeyValue peek() { + return this.heap.peek(); } - /** @return true if the scanner is a multiple match scanner */ - public boolean isMultipleMatchScanner() { - return this.multipleMatchers; - } - - public boolean next(List results) - throws IOException { - this.lock.readLock().lock(); - try { - // Filtered flag is set by filters. If a cell has been 'filtered out' - // -- i.e. it is not to be returned to the caller -- the flag is 'true'. - boolean filtered = true; - boolean moreToFollow = true; - while (filtered && moreToFollow) { - // Find the lowest-possible key. - KeyValue chosen = null; - long chosenTimestamp = -1; - for (int i = 0; i < this.scanners.length; i++) { - KeyValue kv = this.resultSets[i] == null || this.resultSets[i].isEmpty()? - null: this.resultSets[i].get(0); - if (kv == null) { - continue; - } - if (scanners[i] != null && - (chosen == null || - (this.store.comparator.compareRows(kv, chosen) < 0) || - ((this.store.comparator.compareRows(kv, chosen) == 0) && - (kv.getTimestamp() > chosenTimestamp)))) { - chosen = kv; - chosenTimestamp = chosen.getTimestamp(); - } - } - - // Filter whole row by row key? - filtered = dataFilter == null || chosen == null? false: - dataFilter.filterRowKey(chosen.getBuffer(), chosen.getRowOffset(), - chosen.getRowLength()); - - // Store results for each sub-scanner. - if (chosenTimestamp >= 0 && !filtered) { - NavigableSet deletes = - new TreeSet(this.store.comparatorIgnoringType); - for (int i = 0; i < scanners.length && !filtered; i++) { - if ((scanners[i] != null && !filtered && moreToFollow && - this.resultSets[i] != null && !this.resultSets[i].isEmpty())) { - // Test this resultset is for the 'chosen' row. - KeyValue firstkv = resultSets[i].get(0); - if (!this.store.comparator.matchingRows(firstkv, chosen)) { - continue; - } - // Its for the 'chosen' row, work it. - for (KeyValue kv: resultSets[i]) { - if (kv.isDeleteType()) { - deletes.add(kv); - } else if ((deletes.isEmpty() || !deletes.contains(kv)) && - !filtered && moreToFollow && !results.contains(kv)) { - if (this.dataFilter != null) { - // Filter whole row by column data? - int rowlength = kv.getRowLength(); - int columnoffset = kv.getColumnOffset(rowlength); - filtered = dataFilter.filterColumn(kv.getBuffer(), - kv.getRowOffset(), rowlength, - kv.getBuffer(), columnoffset, kv.getColumnLength(columnoffset), - kv.getBuffer(), kv.getValueOffset(), kv.getValueLength()); - if (filtered) { - results.clear(); - break; - } - } - results.add(kv); - /* REMOVING BECAUSE COULD BE BUNCH OF DELETES IN RESULTS - AND WE WANT TO INCLUDE THEM -- below short-circuit is - probably not wanted. - // If we are doing a wild card match or there are multiple - // matchers per column, we need to scan all the older versions of - // this row to pick up the rest of the family members - if (!wildcardMatch && !multipleMatchers && - (kv.getTimestamp() != chosenTimestamp)) { - break; - } - */ - } - } - // Move on to next row. - resultSets[i].clear(); - if (!scanners[i].next(resultSets[i])) { - closeScanner(i); - } - } - } - } - - moreToFollow = chosenTimestamp >= 0; - if (dataFilter != null) { - if (dataFilter.filterAllRemaining()) { - moreToFollow = false; - } - } - - if (results.isEmpty() && !filtered) { - // There were no results found for this row. Marked it as - // 'filtered'-out otherwise we will not move on to the next row. - filtered = true; - } - } - - // If we got no results, then there is no more to follow. - if (results == null || results.isEmpty()) { - moreToFollow = false; - } - - // Make sure scanners closed if no more results - if (!moreToFollow) { - for (int i = 0; i < scanners.length; i++) { - if (null != scanners[i]) { - closeScanner(i); - } - } - } - - return moreToFollow; - } finally { - this.lock.readLock().unlock(); - } - } - - /** Shut down a single scanner */ - void closeScanner(int i) { - try { - try { - scanners[i].close(); - } catch (IOException e) { - LOG.warn(Bytes.toString(store.storeName) + " failed closing scanner " + - i, e); - } - } finally { - scanners[i] = null; - resultSets[i] = null; - } + public KeyValue next() { + // throw runtime exception perhaps? + throw new RuntimeException("Never call StoreScanner.next()"); } public void close() { this.closing.set(true); - this.store.deleteChangedReaderObserver(this); - doClose(); + // under test, we dont have a this.store + if (this.store != null) + this.store.deleteChangedReaderObserver(this); + this.heap.close(); } - - private void doClose() { - for (int i = MEMS_INDEX; i < scanners.length; i++) { - if (scanners[i] != null) { - closeScanner(i); + + public boolean seek(KeyValue key) { + + return this.heap.seek(key); + } + + /** + * Get the next row of values from this Store. + * @param result + * @return true if there are more rows, false if scanner is done + */ + public boolean next(List result) throws IOException { + // this wont get us the next row if the previous round hasn't iterated + // past all the cols from the previous row. Potential bug! + KeyValue peeked = this.heap.peek(); + if (peeked == null) { + close(); + return false; + } + matcher.setRow(peeked.getRow()); + KeyValue kv; + while((kv = this.heap.peek()) != null) { + QueryMatcher.MatchCode mc = matcher.match(kv); + switch(mc) { + case INCLUDE: + KeyValue next = this.heap.next(); + result.add(next); + continue; + case DONE: + // what happens if we have 0 results? + if (result.isEmpty()) { + // try the next one. + matcher.setRow(this.heap.peek().getRow()); + continue; + } + if (matcher.filterEntireRow()) { + // wow, well, um, reset the result and continue. + result.clear(); + matcher.setRow(heap.peek().getRow()); + continue; + } + + return true; + + case DONE_SCAN: + close(); + return false; + + case SEEK_NEXT_ROW: + // TODO see comments in SEEK_NEXT_COL + /* + KeyValue rowToSeek = + new KeyValue(kv.getRow(), + 0, + KeyValue.Type.Minimum); + heap.seek(rowToSeek); + */ + heap.next(); + break; + + case SEEK_NEXT_COL: + // TODO hfile needs 'hinted' seeking to prevent it from + // reseeking from the start of the block on every dang seek. + // We need that API and expose it the scanner chain. + /* + ColumnCount hint = matcher.getSeekColumn(); + KeyValue colToSeek; + if (hint == null) { + // seek to the 'last' key on this column, this is defined + // as the key with the same row, fam, qualifier, + // smallest timestamp, largest type. + colToSeek = + new KeyValue(kv.getRow(), + kv.getFamily(), + kv.getColumn(), + Long.MIN_VALUE, + KeyValue.Type.Minimum); + } else { + // This is ugmo. Move into KeyValue convience method. + // First key on a column is: + // same row, cf, qualifier, max_timestamp, max_type, no value. + colToSeek = + new KeyValue(kv.getRow(), + 0, + kv.getRow().length, + + kv.getFamily(), + 0, + kv.getFamily().length, + + hint.getBuffer(), + hint.getOffset(), + hint.getLength(), + + Long.MAX_VALUE, + KeyValue.Type.Maximum, + null, + 0, + 0); + } + heap.seek(colToSeek); + */ + + heap.next(); + break; + + case SKIP: + this.heap.next(); + break; } } + if(result.size() > 0) { + return true; + } + // No more keys + close(); + return false; } - + + private List getStoreFileScanners() { + List s = + new ArrayList(this.store.getStorefilesCount()); + Map map = this.store.getStorefiles().descendingMap(); + for(StoreFile sf : map.values()) { + s.add(sf.getReader().getScanner()); + } + List scanners = + new ArrayList(s.size()+1); + for(HFileScanner hfs : s) { + scanners.add(new StoreFileScanner(hfs)); + } + return scanners; + } + // Implementation of ChangedReadersObserver - public void updateReaders() throws IOException { if (this.closing.get()) { return; } this.lock.writeLock().lock(); try { - Map map = this.store.getStorefiles(); - if (this.scanners[HSFS_INDEX] == null && map != null && map.size() > 0) { - // Presume that we went from no readers to at least one -- need to put - // a HStoreScanner in place. - try { - // I think its safe getting key from mem at this stage -- it shouldn't have - // been flushed yet - // TODO: MAKE SURE WE UPDATE FROM TRUNNK. - this.scanners[HSFS_INDEX] = new StoreFileScanner(this.store, - this.timestamp, this. columns, this.resultSets[MEMS_INDEX].get(0).getRow()); - checkScannerFlags(HSFS_INDEX); - setupScanner(HSFS_INDEX); - LOG.debug("Added a StoreFileScanner to outstanding HStoreScanner"); - } catch (IOException e) { - doClose(); - throw e; - } - } + // Could do this pretty nicely with KeyValueHeap, but the existing + // implementation of this method only updated if no existing storefiles? + // Lets discuss. + return; } finally { this.lock.writeLock().unlock(); } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/WildcardColumnTracker.java b/src/java/org/apache/hadoop/hbase/regionserver/WildcardColumnTracker.java new file mode 100644 index 00000000000..5e7417d722f --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/regionserver/WildcardColumnTracker.java @@ -0,0 +1,314 @@ +/** + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * This class is used for the tracking and enforcement of columns and numbers + * of versions during the course of a Get or Scan operation, when all available + * column qualifiers have been asked for in the query. + *

                + * This class is utilized by {@link QueryMatcher} through two methods: + *

                • {@link checkColumn} is called when a Put satisfies all other + * conditions of the query. This method returns a {@link MatchCode} to define + * what action should be taken. + *
                • {@link update} is called at the end of every StoreFile or Memcache. + *

                  + * This class is NOT thread-safe as queries are never multi-threaded + */ +public class WildcardColumnTracker implements ColumnTracker { + + private int maxVersions; + + protected List columns; + private int index; + private ColumnCount column; + + private List newColumns; + private int newIndex; + private ColumnCount newColumn; + + /** + * Default constructor. + * @param maxVersions maximum versions to return per columns + */ + public WildcardColumnTracker(int maxVersions) { + this.maxVersions = maxVersions; + reset(); + } + + public void reset() { + this.index = 0; + this.column = null; + this.columns = null; + this.newColumns = new ArrayList(); + this.newIndex = 0; + this.newColumn = null; + } + + /** + * Can never early-out from reading more storefiles in Wildcard case. + */ + public boolean done() { + return false; + } + + // wildcard scanners never have column hints. + public ColumnCount getColumnHint() { + return null; + } + + /** + * Checks against the parameters of the query and the columns which have + * already been processed by this query. + * @param bytes KeyValue buffer + * @param offset offset to the start of the qualifier + * @param length length of the qualifier + * @return MatchCode telling QueryMatcher what action to take + */ + public MatchCode checkColumn(byte [] bytes, int offset, int length) { + + // Nothing to match against, add to new and include + if(this.column == null && this.newColumn == null) { + newColumns.add(new ColumnCount(bytes, offset, length, 1)); + this.newColumn = newColumns.get(newIndex); + return MatchCode.INCLUDE; + } + + // Nothing old, compare against new + if(this.column == null && this.newColumn != null) { + int ret = Bytes.compareTo(newColumn.getBuffer(), newColumn.getOffset(), + newColumn.getLength(), bytes, offset, length); + + // Same column + if(ret == 0) { + if(newColumn.increment() > this.maxVersions) { + return MatchCode.SKIP; + } + return MatchCode.INCLUDE; + } + + // Specified column is bigger than current column + // Move down current column and check again + if(ret <= -1) { + if(++newIndex == newColumns.size()) { + // No more, add to end and include + newColumns.add(new ColumnCount(bytes, offset, length, 1)); + this.newColumn = newColumns.get(newIndex); + return MatchCode.INCLUDE; + } + this.newColumn = newColumns.get(newIndex); + return checkColumn(bytes, offset, length); + } + + // ret >= 1 + // Specified column is smaller than current column + // Nothing to match against, add to new and include + newColumns.add(new ColumnCount(bytes, offset, length, 1)); + this.newColumn = newColumns.get(++newIndex); + return MatchCode.INCLUDE; + } + + // Nothing new, compare against old + if(this.newColumn == null && this.column != null) { + int ret = Bytes.compareTo(column.getBuffer(), column.getOffset(), + column.getLength(), bytes, offset, length); + + // Same column + if(ret == 0) { + if(column.increment() > this.maxVersions) { + return MatchCode.SKIP; + } + return MatchCode.INCLUDE; + } + + // Specified column is bigger than current column + // Move down current column and check again + if(ret <= -1) { + if(++index == columns.size()) { + // No more, add to new and include (new was empty prior to this) + newColumns.add(new ColumnCount(bytes, offset, length, 1)); + this.newColumn = newColumns.get(newIndex); + this.column = null; + return MatchCode.INCLUDE; + } + this.column = columns.get(index); + return checkColumn(bytes, offset, length); + } + + // ret >= 1 + // Specified column is smaller than current column + // Nothing to match against, add to new and include + newColumns.add(new ColumnCount(bytes, offset, length, 1)); + this.newColumn = newColumns.get(newIndex); + return MatchCode.INCLUDE; + } + + + // There are new and old, figure which to check first + int ret = Bytes.compareTo(column.getBuffer(), column.getOffset(), + column.getLength(), newColumn.getBuffer(), newColumn.getOffset(), + newColumn.getLength()); + + // Old is smaller than new, compare against old + if(ret <= -1) { + ret = Bytes.compareTo(column.getBuffer(), column.getOffset(), + column.getLength(), bytes, offset, length); + + // Same column + if(ret == 0) { + if(column.increment() > this.maxVersions) { + return MatchCode.SKIP; + } + return MatchCode.INCLUDE; + } + + // Specified column is bigger than current column + // Move down current column and check again + if(ret <= -1) { + if(++index == columns.size()) { + this.column = null; + } else { + this.column = columns.get(index); + } + return checkColumn(bytes, offset, length); + } + + // ret >= 1 + // Specified column is smaller than current column + // Nothing to match against, add to new and include + newColumns.add(new ColumnCount(bytes, offset, length, 1)); + return MatchCode.INCLUDE; + } + + // Cannot be equal, so ret >= 1 + // New is smaller than old, compare against new + + ret = Bytes.compareTo(newColumn.getBuffer(), newColumn.getOffset(), + newColumn.getLength(), bytes, offset, length); + + // Same column + if(ret == 0) { + if(newColumn.increment() > this.maxVersions) { + return MatchCode.SKIP; + } + return MatchCode.INCLUDE; + } + + // Specified column is bigger than current column + // Move down current column and check again + if(ret <= -1) { + if(++newIndex == newColumns.size()) { + this.newColumn = null; + } else { + this.newColumn = newColumns.get(newIndex); + } + return checkColumn(bytes, offset, length); + } + + // ret >= 1 + // Specified column is smaller than current column + // Nothing to match against, add to new and include + newColumns.add(new ColumnCount(bytes, offset, length, 1)); + return MatchCode.INCLUDE; + } + + /** + * Called at the end of every StoreFile or Memcache. + */ + public void update() { + // If no previous columns, use new columns and return + if(this.columns == null || this.columns.size() == 0) { + if(this.newColumns.size() > 0){ + finalize(newColumns); + } + return; + } + + // If no new columns, retain previous columns and return + if(this.newColumns.size() == 0) { + this.index = 0; + this.column = this.columns.get(index); + return; + } + + // Merge previous columns with new columns + // There will be no overlapping + List mergeColumns = new ArrayList( + columns.size() + newColumns.size()); + index = 0; + newIndex = 0; + column = columns.get(0); + newColumn = newColumns.get(0); + while(true) { + int ret = Bytes.compareTo( + column.getBuffer(), column.getOffset(),column.getLength(), + newColumn.getBuffer(), newColumn.getOffset(), newColumn.getLength()); + + // Existing is smaller than new, add existing and iterate it + if(ret <= -1) { + mergeColumns.add(column); + if(++index == columns.size()) { + // No more existing left, merge down rest of new and return + mergeDown(mergeColumns, newColumns, newIndex); + finalize(mergeColumns); + return; + } + column = columns.get(index); + continue; + } + + // New is smaller than existing, add new and iterate it + mergeColumns.add(newColumn); + if(++newIndex == newColumns.size()) { + // No more new left, merge down rest of existing and return + mergeDown(mergeColumns, columns, index); + finalize(mergeColumns); + return; + } + newColumn = newColumns.get(newIndex); + continue; + } + } + + private void mergeDown(List mergeColumns, + List srcColumns, int srcIndex) { + int index = srcIndex; + while(index < srcColumns.size()) { + mergeColumns.add(srcColumns.get(index++)); + } + } + + private void finalize(List mergeColumns) { + this.columns = mergeColumns; + this.index = 0; + this.column = this.columns.size() > 0? columns.get(index) : null; + + this.newColumns = new ArrayList(); + this.newIndex = 0; + this.newColumn = null; + } + +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexMaintenanceUtils.java b/src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexMaintenanceUtils.java deleted file mode 100644 index 2f418635801..00000000000 --- a/src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexMaintenanceUtils.java +++ /dev/null @@ -1,73 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this work for additional information regarding - * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, - * either express or implied. See the License for the specific language governing permissions and limitations under the - * License. - */ -package org.apache.hadoop.hbase.regionserver.tableindexed; - -import java.util.SortedMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification; -import org.apache.hadoop.hbase.client.tableindexed.IndexedTable; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Singleton class for index maintence logic. - */ -public class IndexMaintenanceUtils { - - private static final Log LOG = LogFactory.getLog(IndexMaintenanceUtils.class); - - public static BatchUpdate createIndexUpdate(final IndexSpecification indexSpec, final byte[] row, - final SortedMap columnValues) { - byte[] indexRow = indexSpec.getKeyGenerator().createIndexKey(row, columnValues); - BatchUpdate update = new BatchUpdate(indexRow); - - update.put(IndexedTable.INDEX_BASE_ROW_COLUMN, row); - - for (byte[] col : indexSpec.getIndexedColumns()) { - byte[] val = columnValues.get(col); - if (val == null) { - throw new RuntimeException("Unexpected missing column value. [" + Bytes.toString(col) + "]"); - } - update.put(col, val); - } - - for (byte[] col : indexSpec.getAdditionalColumns()) { - byte[] val = columnValues.get(col); - if (val != null) { - update.put(col, val); - } - } - - return update; - } - - /** - * Ask if this update does apply to the index. - * - * @param indexSpec - * @param b - * @return true if possibly apply. - */ - public static boolean doesApplyToIndex(final IndexSpecification indexSpec, - final SortedMap columnValues) { - - for (byte[] neededCol : indexSpec.getIndexedColumns()) { - if (!columnValues.containsKey(neededCol)) { - LOG.debug("Index [" + indexSpec.getIndexId() + "] can't be updated because [" - + Bytes.toString(neededCol) + "] is missing"); - return false; - } - } - return true; - } -} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexedRegion.java b/src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexedRegion.java deleted file mode 100644 index 1399bd4e1c4..00000000000 --- a/src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexedRegion.java +++ /dev/null @@ -1,305 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver.tableindexed; - -import java.io.IOException; -import java.util.Collection; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.NavigableSet; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.TreeSet; -import java.util.Map.Entry; - -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.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.tableindexed.IndexSpecification; -import org.apache.hadoop.hbase.client.tableindexed.IndexedTable; -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.regionserver.FlushRequester; -import org.apache.hadoop.hbase.regionserver.HLog; -import org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegion; -import org.apache.hadoop.hbase.util.Bytes; - -class IndexedRegion extends TransactionalRegion { - - private static final Log LOG = LogFactory.getLog(IndexedRegion.class); - - private final HBaseConfiguration conf; - private Map indexSpecToTable = new HashMap(); - - public IndexedRegion(final Path basedir, final HLog log, final FileSystem fs, - final HBaseConfiguration conf, final HRegionInfo regionInfo, - final FlushRequester flushListener) { - super(basedir, log, fs, conf, regionInfo, flushListener); - this.conf = conf; - } - - private synchronized HTable getIndexTable(IndexSpecification index) - throws IOException { - HTable indexTable = indexSpecToTable.get(index); - if (indexTable == null) { - indexTable = new HTable(conf, index.getIndexedTableName(super - .getRegionInfo().getTableDesc().getName())); - indexSpecToTable.put(index, indexTable); - } - return indexTable; - } - - private Collection getIndexes() { - return super.getRegionInfo().getTableDesc().getIndexes(); - } - - /** - * @param batchUpdate - * @param lockid - * @param writeToWAL if true, then we write this update to the log - * @throws IOException - */ - @Override - public void batchUpdate(BatchUpdate batchUpdate, Integer lockid, boolean writeToWAL) - throws IOException { - updateIndexes(batchUpdate); // Do this first because will want to see the old row - super.batchUpdate(batchUpdate, lockid, writeToWAL); - } - - private void updateIndexes(BatchUpdate batchUpdate) throws IOException { - List indexesToUpdate = new LinkedList(); - - // Find the indexes we need to update - for (IndexSpecification index : getIndexes()) { - if (possiblyAppliesToIndex(index, batchUpdate)) { - indexesToUpdate.add(index); - } - } - - if (indexesToUpdate.size() == 0) { - return; - } - - NavigableSet neededColumns = getColumnsForIndexes(indexesToUpdate); - - NavigableMap newColumnValues = - getColumnsFromBatchUpdate(batchUpdate); - Map oldColumnCells = super.getFull(batchUpdate.getRow(), - neededColumns, HConstants.LATEST_TIMESTAMP, 1, null); - - // Handle delete batch updates. Go back and get the next older values - for (BatchOperation op : batchUpdate) { - if (!op.isPut()) { - Cell current = oldColumnCells.get(op.getColumn()); - if (current != null) { - // TODO: Fix this profligacy!!! St.Ack - Cell [] older = Cell.createSingleCellArray(super.get(batchUpdate.getRow(), - op.getColumn(), current.getTimestamp(), 1)); - if (older != null && older.length > 0) { - newColumnValues.put(op.getColumn(), older[0].getValue()); - } - } - } - } - - // Add the old values to the new if they are not there - for (Entry oldEntry : oldColumnCells.entrySet()) { - if (!newColumnValues.containsKey(oldEntry.getKey())) { - newColumnValues.put(oldEntry.getKey(), oldEntry.getValue().getValue()); - } - } - - - - Iterator indexIterator = indexesToUpdate.iterator(); - while (indexIterator.hasNext()) { - IndexSpecification indexSpec = indexIterator.next(); - if (!IndexMaintenanceUtils.doesApplyToIndex(indexSpec, newColumnValues)) { - indexIterator.remove(); - } - } - - SortedMap oldColumnValues = convertToValueMap(oldColumnCells); - - for (IndexSpecification indexSpec : indexesToUpdate) { - removeOldIndexEntry(indexSpec, batchUpdate.getRow(), oldColumnValues); - updateIndex(indexSpec, batchUpdate.getRow(), newColumnValues); - } - } - - /** Return the columns needed for the update. */ - private NavigableSet getColumnsForIndexes(Collection indexes) { - NavigableSet neededColumns = new TreeSet(Bytes.BYTES_COMPARATOR); - for (IndexSpecification indexSpec : indexes) { - for (byte[] col : indexSpec.getAllColumns()) { - neededColumns.add(col); - } - } - return neededColumns; - } - - private void removeOldIndexEntry(IndexSpecification indexSpec, byte[] row, - SortedMap oldColumnValues) throws IOException { - for (byte[] indexedCol : indexSpec.getIndexedColumns()) { - if (!oldColumnValues.containsKey(indexedCol)) { - LOG.debug("Index [" + indexSpec.getIndexId() - + "] not trying to remove old entry for row [" - + Bytes.toString(row) + "] because col [" - + Bytes.toString(indexedCol) + "] is missing"); - return; - } - } - - byte[] oldIndexRow = indexSpec.getKeyGenerator().createIndexKey(row, - oldColumnValues); - LOG.debug("Index [" + indexSpec.getIndexId() + "] removing old entry [" - + Bytes.toString(oldIndexRow) + "]"); - getIndexTable(indexSpec).deleteAll(oldIndexRow); - } - - private NavigableMap getColumnsFromBatchUpdate(BatchUpdate b) { - NavigableMap columnValues = new TreeMap( - Bytes.BYTES_COMPARATOR); - for (BatchOperation op : b) { - if (op.isPut()) { - columnValues.put(op.getColumn(), op.getValue()); - } - } - return columnValues; - } - - /** Ask if this update *could* apply to the index. It may actually apply if some of the columns needed are missing. - * - * @param indexSpec - * @param b - * @return true if possibly apply. - */ - private boolean possiblyAppliesToIndex(IndexSpecification indexSpec, BatchUpdate b) { - for (BatchOperation op : b) { - if (indexSpec.containsColumn(op.getColumn())) { - return true; - } - } - return false; - } - - // FIXME: This call takes place in an RPC, and requires an RPC. This makes for - // a likely deadlock if the number of RPCs we are trying to serve is >= the - // number of handler threads. - private void updateIndex(IndexSpecification indexSpec, byte[] row, - SortedMap columnValues) throws IOException { - BatchUpdate indexUpdate = IndexMaintenanceUtils.createIndexUpdate(indexSpec, row, columnValues); - getIndexTable(indexSpec).commit(indexUpdate); // FIXME, this is synchronized - LOG.debug("Index [" + indexSpec.getIndexId() + "] adding new entry [" - + Bytes.toString(indexUpdate.getRow()) + "] for row [" - + Bytes.toString(row) + "]"); - - } - - @Override - public void deleteAll(final byte[] row, final long ts, final Integer lockid) - throws IOException { - - if (getIndexes().size() != 0) { - - // Need all columns - NavigableSet neededColumns = getColumnsForIndexes(getIndexes()); - - Map oldColumnCells = super.getFull(row, - neededColumns, HConstants.LATEST_TIMESTAMP, 1, null); - SortedMap oldColumnValues = convertToValueMap(oldColumnCells); - - - for (IndexSpecification indexSpec : getIndexes()) { - removeOldIndexEntry(indexSpec, row, oldColumnValues); - } - - // Handle if there is still a version visible. - if (ts != HConstants.LATEST_TIMESTAMP) { - Map currentColumnCells = super.getFull(row, - neededColumns, ts, 1, null); - SortedMap currentColumnValues = convertToValueMap(currentColumnCells); - - for (IndexSpecification indexSpec : getIndexes()) { - if (IndexMaintenanceUtils.doesApplyToIndex(indexSpec, currentColumnValues)) { - updateIndex(indexSpec, row, currentColumnValues); - } - } - } - } - super.deleteAll(row, ts, lockid); - } - - private SortedMap convertToValueMap( - Map cellMap) { - SortedMap currentColumnValues = new TreeMap(Bytes.BYTES_COMPARATOR); - for(Entry entry : cellMap.entrySet()) { - currentColumnValues.put(entry.getKey(), entry.getValue().getValue()); - } - return currentColumnValues; - } - - @Override - public void deleteAll(final byte[] row, byte[] column, final long ts, - final Integer lockid) throws IOException { - List indexesToUpdate = new LinkedList(); - - for(IndexSpecification indexSpec : getIndexes()) { - if (indexSpec.containsColumn(column)) { - indexesToUpdate.add(indexSpec); - } - } - - NavigableSet neededColumns = getColumnsForIndexes(indexesToUpdate); - Map oldColumnCells = super.getFull(row, - neededColumns, HConstants.LATEST_TIMESTAMP, 1, null); - SortedMap oldColumnValues = convertToValueMap(oldColumnCells); - - for (IndexSpecification indexSpec : indexesToUpdate) { - removeOldIndexEntry(indexSpec, row, oldColumnValues); - } - - // Handle if there is still a version visible. - if (ts != HConstants.LATEST_TIMESTAMP) { - Map currentColumnCells = super.getFull(row, - neededColumns, ts, 1, null); - SortedMap currentColumnValues = convertToValueMap(currentColumnCells); - - for (IndexSpecification indexSpec : getIndexes()) { - if (IndexMaintenanceUtils.doesApplyToIndex(indexSpec, currentColumnValues)) { - updateIndex(indexSpec, row, currentColumnValues); - } - } - } - - super.deleteAll(row, column, ts, lockid); - } - -} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexedRegionServer.java b/src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexedRegionServer.java deleted file mode 100644 index b383a0eaae4..00000000000 --- a/src/java/org/apache/hadoop/hbase/regionserver/tableindexed/IndexedRegionServer.java +++ /dev/null @@ -1,74 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver.tableindexed; - -import java.io.IOException; - -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.ipc.IndexedRegionInterface; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegionServer; -import org.apache.hadoop.util.Progressable; -import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion; - -/** - * RegionServer which maintains secondary indexes. - * - **/ -public class IndexedRegionServer extends TransactionalRegionServer implements - IndexedRegionInterface { - - public IndexedRegionServer(HBaseConfiguration conf) throws IOException { - this(new HServerAddress(conf.get(REGIONSERVER_ADDRESS, - DEFAULT_REGIONSERVER_ADDRESS)), conf); - } - - public IndexedRegionServer(HServerAddress serverAddress, - HBaseConfiguration conf) throws IOException { - super(serverAddress, conf); - } - - @Override - public long getProtocolVersion(final String protocol, final long clientVersion) - throws IOException { - if (protocol.equals(IndexedRegionInterface.class.getName())) { - return HBaseRPCProtocolVersion.versionID; - } - return super.getProtocolVersion(protocol, clientVersion); - } - - @Override - protected HRegion instantiateRegion(final HRegionInfo regionInfo) - throws IOException { - HRegion r = new IndexedRegion(HTableDescriptor.getTableDir(super - .getRootDir(), regionInfo.getTableDesc().getName()), super.hlog, super - .getFileSystem(), super.conf, regionInfo, super.getFlushRequester()); - r.initialize(null, new Progressable() { - public void progress() { - addProcessingMessage(regionInfo); - } - }); - return r; - } - -} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/transactional/CleanOldTransactionsChore.java b/src/java/org/apache/hadoop/hbase/regionserver/transactional/CleanOldTransactionsChore.java index a2a522a3837..e69de29bb2d 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/transactional/CleanOldTransactionsChore.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/transactional/CleanOldTransactionsChore.java @@ -1,57 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver.transactional; - -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.hadoop.hbase.Chore; -import org.apache.hadoop.hbase.regionserver.HRegion; - -/** - * Cleans up committed transactions when they are no longer needed to verify - * pending transactions. - */ -class CleanOldTransactionsChore extends Chore { - - private static final String SLEEP_CONF = "hbase.transaction.clean.sleep"; - private static final int DEFAULT_SLEEP = 60 * 1000; - - private final TransactionalRegionServer regionServer; - - /** - * @param regionServer - * @param stopRequest - */ - public CleanOldTransactionsChore( - final TransactionalRegionServer regionServer, - final AtomicBoolean stopRequest) { - super(regionServer.getConfiguration().getInt(SLEEP_CONF, DEFAULT_SLEEP), - stopRequest); - this.regionServer = regionServer; - } - - @Override - protected void chore() { - for (HRegion region : regionServer.getOnlineRegions()) { - ((TransactionalRegion) region).removeUnNeededCommitedTransactions(); - } - } - -} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionState.java b/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionState.java deleted file mode 100644 index 28c0a56eae1..00000000000 --- a/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionState.java +++ /dev/null @@ -1,362 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver.transactional; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeMap; -import java.util.TreeSet; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.filter.RowFilterInterface; -import org.apache.hadoop.hbase.filter.RowFilterSet; -import org.apache.hadoop.hbase.filter.StopRowFilter; -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.util.Bytes; - -/** - * Holds the state of a transaction. - */ -class TransactionState { - - private static final Log LOG = LogFactory.getLog(TransactionState.class); - - /** Current status. */ - public enum Status { - /** Initial status, still performing operations. */ - PENDING, - /** - * Checked if we can commit, and said yes. Still need to determine the - * global decision. - */ - COMMIT_PENDING, - /** Committed. */ - COMMITED, - /** Aborted. */ - ABORTED - } - - /** - * Simple container of the range of the scanners we've opened. Used to check - * for conflicting writes. - */ - private static class ScanRange { - protected byte[] startRow; - protected byte[] endRow; - - public ScanRange(byte[] startRow, byte[] endRow) { - this.startRow = startRow; - this.endRow = endRow; - } - - /** - * Check if this scan range contains the given key. - * - * @param rowKey - * @return boolean - */ - public boolean contains(byte[] rowKey) { - if (startRow != null && Bytes.compareTo(rowKey, startRow) < 0) { - return false; - } - if (endRow != null && Bytes.compareTo(endRow, rowKey) < 0) { - return false; - } - return true; - } - } - - private final HRegionInfo regionInfo; - private final long hLogStartSequenceId; - private final long transactionId; - private Status status; - private SortedSet readSet = new TreeSet( - Bytes.BYTES_COMPARATOR); - private List writeSet = new LinkedList(); - private List scanSet = new LinkedList(); - private Set transactionsToCheck = new HashSet(); - private int startSequenceNumber; - private Integer sequenceNumber; - - TransactionState(final long transactionId, final long rLogStartSequenceId, - HRegionInfo regionInfo) { - this.transactionId = transactionId; - this.hLogStartSequenceId = rLogStartSequenceId; - this.regionInfo = regionInfo; - this.status = Status.PENDING; - } - - void addRead(final byte[] rowKey) { - readSet.add(rowKey); - } - - Set getReadSet() { - return readSet; - } - - void addWrite(final BatchUpdate write) { - writeSet.add(write); - } - - List getWriteSet() { - return writeSet; - } - - /** - * GetFull from the writeSet. - * - * @param row - * @param columns - * @param timestamp - * @return - */ - Map localGetFull(final byte[] row, final Set columns, - final long timestamp) { - Map results = new TreeMap( - Bytes.BYTES_COMPARATOR); // Must use the Bytes Conparator because - for (BatchUpdate b : writeSet) { - if (!Bytes.equals(row, b.getRow())) { - continue; - } - if (b.getTimestamp() > timestamp) { - continue; - } - for (BatchOperation op : b) { - if (!op.isPut() - || (columns != null && !columns.contains(op.getColumn()))) { - continue; - } - results.put(op.getColumn(), new Cell(op.getValue(), b.getTimestamp())); - } - } - return results.size() == 0 ? null : results; - } - - /** - * Get from the writeSet. - * - * @param row - * @param column - * @param timestamp - * @return - */ - Cell[] localGet(final byte[] row, final byte[] column, final long timestamp) { - ArrayList results = new ArrayList(); - - // Go in reverse order to put newest updates first in list - for (int i = writeSet.size() - 1; i >= 0; i--) { - BatchUpdate b = writeSet.get(i); - - if (!Bytes.equals(row, b.getRow())) { - continue; - } - if (b.getTimestamp() > timestamp) { - continue; - } - for (BatchOperation op : b) { - if (!op.isPut() || !Bytes.equals(column, op.getColumn())) { - continue; - } - results.add(new Cell(op.getValue(), b.getTimestamp())); - } - } - return results.size() == 0 ? null : results - .toArray(new Cell[results.size()]); - } - - void addTransactionToCheck(final TransactionState transaction) { - transactionsToCheck.add(transaction); - } - - boolean hasConflict() { - for (TransactionState transactionState : transactionsToCheck) { - if (hasConflict(transactionState)) { - return true; - } - } - return false; - } - - private boolean hasConflict(final TransactionState checkAgainst) { - if (checkAgainst.getStatus().equals(TransactionState.Status.ABORTED)) { - return false; // Cannot conflict with aborted transactions - } - - for (BatchUpdate otherUpdate : checkAgainst.getWriteSet()) { - if (this.getReadSet().contains(otherUpdate.getRow())) { - LOG.debug("Transaction [" + this.toString() - + "] has read which conflicts with [" + checkAgainst.toString() - + "]: region [" + regionInfo.getRegionNameAsString() + "], row[" - + Bytes.toString(otherUpdate.getRow()) + "]"); - return true; - } - for (ScanRange scanRange : this.scanSet) { - if (scanRange.contains(otherUpdate.getRow())) { - LOG.debug("Transaction [" + this.toString() - + "] has scan which conflicts with [" + checkAgainst.toString() - + "]: region [" + regionInfo.getRegionNameAsString() + "], row[" - + Bytes.toString(otherUpdate.getRow()) + "]"); - return true; - } - } - } - return false; - } - - /** - * Get the status. - * - * @return Return the status. - */ - Status getStatus() { - return status; - } - - /** - * Set the status. - * - * @param status The status to set. - */ - void setStatus(final Status status) { - this.status = status; - } - - /** - * Get the startSequenceNumber. - * - * @return Return the startSequenceNumber. - */ - int getStartSequenceNumber() { - return startSequenceNumber; - } - - /** - * Set the startSequenceNumber. - * - * @param startSequenceNumber - */ - void setStartSequenceNumber(final int startSequenceNumber) { - this.startSequenceNumber = startSequenceNumber; - } - - /** - * Get the sequenceNumber. - * - * @return Return the sequenceNumber. - */ - Integer getSequenceNumber() { - return sequenceNumber; - } - - /** - * Set the sequenceNumber. - * - * @param sequenceNumber The sequenceNumber to set. - */ - void setSequenceNumber(final Integer sequenceNumber) { - this.sequenceNumber = sequenceNumber; - } - - @Override - public String toString() { - StringBuilder result = new StringBuilder(); - result.append("[transactionId: "); - result.append(transactionId); - result.append(" status: "); - result.append(status.name()); - result.append(" read Size: "); - result.append(readSet.size()); - result.append(" scan Size: "); - result.append(scanSet.size()); - result.append(" write Size: "); - result.append(writeSet.size()); - result.append(" startSQ: "); - result.append(startSequenceNumber); - if (sequenceNumber != null) { - result.append(" commitedSQ:"); - result.append(sequenceNumber); - } - result.append("]"); - - return result.toString(); - } - - /** - * Get the transactionId. - * - * @return Return the transactionId. - */ - long getTransactionId() { - return transactionId; - } - - /** - * Get the startSequenceId. - * - * @return Return the startSequenceId. - */ - long getHLogStartSequenceId() { - return hLogStartSequenceId; - } - - void addScan(byte[] firstRow, RowFilterInterface filter) { - ScanRange scanRange = new ScanRange(firstRow, getEndRow(filter)); - LOG.trace(String.format( - "Adding scan for transcaction [%s], from [%s] to [%s]", transactionId, - scanRange.startRow == null ? "null" : Bytes - .toString(scanRange.startRow), scanRange.endRow == null ? "null" - : Bytes.toString(scanRange.endRow))); - scanSet.add(scanRange); - } - - private byte[] getEndRow(RowFilterInterface filter) { - if (filter instanceof WhileMatchRowFilter) { - WhileMatchRowFilter wmrFilter = (WhileMatchRowFilter) filter; - if (wmrFilter.getInternalFilter() instanceof StopRowFilter) { - StopRowFilter stopFilter = (StopRowFilter) wmrFilter - .getInternalFilter(); - return stopFilter.getStopRowKey(); - } - } else if (filter instanceof RowFilterSet) { - RowFilterSet rowFilterSet = (RowFilterSet) filter; - if (rowFilterSet.getOperator() - .equals(RowFilterSet.Operator.MUST_PASS_ALL)) { - for (RowFilterInterface subFilter : rowFilterSet.getFilters()) { - byte[] endRow = getEndRow(subFilter); - if (endRow != null) { - return endRow; - } - } - } - } - return null; - } - -} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalHLogManager.java b/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalHLogManager.java deleted file mode 100644 index bdfc5930ef4..00000000000 --- a/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalHLogManager.java +++ /dev/null @@ -1,307 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver.transactional; - -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.SortedMap; -import java.util.TreeMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.io.BatchOperation; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.regionserver.HLog; -import org.apache.hadoop.hbase.regionserver.HLogKey; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.util.Progressable; - -/** - * Responsible for writing and reading (recovering) transactional information - * to/from the HLog. - */ -class TransactionalHLogManager { - /** If transactional log entry, these are the op codes */ - // TODO: Make these into types on the KeyValue!!! -- St.Ack - public enum TransactionalOperation { - /** start transaction */ - START, - /** Equivalent to append in non-transactional environment */ - WRITE, - /** Transaction commit entry */ - COMMIT, - /** Abort transaction entry */ - ABORT - } - - private static final Log LOG = LogFactory - .getLog(TransactionalHLogManager.class); - - private final HLog hlog; - private final FileSystem fileSystem; - private final HRegionInfo regionInfo; - private final HBaseConfiguration conf; - - /** - * @param region - */ - public TransactionalHLogManager(final TransactionalRegion region) { - this.hlog = region.getLog(); - this.fileSystem = region.getFilesystem(); - this.regionInfo = region.getRegionInfo(); - this.conf = region.getConf(); - } - - // For Testing - TransactionalHLogManager(final HLog hlog, final FileSystem fileSystem, - final HRegionInfo regionInfo, final HBaseConfiguration conf) { - this.hlog = hlog; - this.fileSystem = fileSystem; - this.regionInfo = regionInfo; - this.conf = conf; - } - - /** - * @param transactionId - * @throws IOException - */ - public void writeStartToLog(final long transactionId) throws IOException { - /* - HLogEdit logEdit; - logEdit = new HLogEdit(transactionId, TransactionalOperation.START); -*/ - hlog.append(regionInfo, null/*logEdit*/, System.currentTimeMillis()); - } - - /** - * @param transactionId - * @param update - * @throws IOException - */ - public void writeUpdateToLog(final long transactionId, - final BatchUpdate update) throws IOException { - - long commitTime = update.getTimestamp() == HConstants.LATEST_TIMESTAMP ? System - .currentTimeMillis() - : update.getTimestamp(); - - for (BatchOperation op : update) { - // COMMENTED OUT HLogEdit logEdit = new HLogEdit(transactionId, update.getRow(), op, commitTime); - hlog.append(regionInfo, update.getRow(), null /*logEdit*/, System.currentTimeMillis()); - } - } - - /** - * @param transactionId - * @throws IOException - */ - public void writeCommitToLog(final long transactionId) throws IOException { - /*HLogEdit logEdit; - logEdit = new HLogEdit(transactionId, - HLogEdit.TransactionalOperation.COMMIT); -*/ - hlog.append(regionInfo, null /*logEdit*/, System.currentTimeMillis()); - } - - /** - * @param transactionId - * @throws IOException - */ - public void writeAbortToLog(final long transactionId) throws IOException { - /*HLogEdit logEdit; - logEdit = new HLogEdit(transactionId, HLogEdit.TransactionalOperation.ABORT); -*/ - hlog.append(regionInfo, null /*logEdit*/, System.currentTimeMillis()); - } - - /** - * @param reconstructionLog - * @param maxSeqID - * @param reporter - * @return map of batch updates - * @throws UnsupportedEncodingException - * @throws IOException - */ - public Map> getCommitsFromLog( - final Path reconstructionLog, final long maxSeqID, - final Progressable reporter) throws UnsupportedEncodingException, - IOException { - if (reconstructionLog == null || !fileSystem.exists(reconstructionLog)) { - // Nothing to do. - return null; - } - // Check its not empty. - FileStatus[] stats = fileSystem.listStatus(reconstructionLog); - if (stats == null || stats.length == 0) { - LOG.warn("Passed reconstruction log " + reconstructionLog - + " is zero-length"); - return null; - } - - SortedMap> pendingTransactionsById = new TreeMap>(); - SortedMap> commitedTransactionsById = new TreeMap>(); - Set abortedTransactions = new HashSet(); - - SequenceFile.Reader logReader = new SequenceFile.Reader(fileSystem, - reconstructionLog, conf); - /* - try { - HLogKey key = new HLogKey(); - KeyValue val = new KeyValue(); - long skippedEdits = 0; - long totalEdits = 0; - long startCount = 0; - long writeCount = 0; - long abortCount = 0; - long commitCount = 0; - // How many edits to apply before we send a progress report. - int reportInterval = conf.getInt("hbase.hstore.report.interval.edits", - 2000); - - while (logReader.next(key, val)) { - LOG.debug("Processing edit: key: " + key.toString() + " val: " - + val.toString()); - if (key.getLogSeqNum() < maxSeqID) { - skippedEdits++; - continue; - } - // TODO: Change all below so we are not doing a getRow and getColumn - // against a KeyValue. Each invocation creates a new instance. St.Ack. - - // Check this edit is for me. - - byte[] column = val.getKeyValue().getColumn(); - Long transactionId = val.getTransactionId(); - if (!val.isTransactionEntry() || HLog.isMetaColumn(column) - || !Bytes.equals(key.getRegionName(), regionInfo.getRegionName())) { - continue; - } - - List updates = pendingTransactionsById.get(transactionId); - switch (val.getOperation()) { - - case START: - if (updates != null || abortedTransactions.contains(transactionId) - || commitedTransactionsById.containsKey(transactionId)) { - LOG.error("Processing start for transaction: " + transactionId - + ", but have already seen start message"); - throw new IOException("Corrupted transaction log"); - } - updates = new LinkedList(); - pendingTransactionsById.put(transactionId, updates); - startCount++; - break; - - case WRITE: - if (updates == null) { - LOG.error("Processing edit for transaction: " + transactionId - + ", but have not seen start message"); - throw new IOException("Corrupted transaction log"); - } - - BatchUpdate tranUpdate = new BatchUpdate(val.getKeyValue().getRow()); - if (val.getKeyValue().getValue() != null) { - tranUpdate.put(val.getKeyValue().getColumn(), - val.getKeyValue().getValue()); - } else { - tranUpdate.delete(val.getKeyValue().getColumn()); - } - updates.add(tranUpdate); - writeCount++; - break; - - case ABORT: - if (updates == null) { - LOG.error("Processing abort for transaction: " + transactionId - + ", but have not seen start message"); - throw new IOException("Corrupted transaction log"); - } - abortedTransactions.add(transactionId); - pendingTransactionsById.remove(transactionId); - abortCount++; - break; - - case COMMIT: - if (updates == null) { - LOG.error("Processing commit for transaction: " + transactionId - + ", but have not seen start message"); - throw new IOException("Corrupted transaction log"); - } - if (abortedTransactions.contains(transactionId)) { - LOG.error("Processing commit for transaction: " + transactionId - + ", but also have abort message"); - throw new IOException("Corrupted transaction log"); - } - if (updates.size() == 0) { - LOG - .warn("Transaciton " + transactionId - + " has no writes in log. "); - } - if (commitedTransactionsById.containsKey(transactionId)) { - LOG.error("Processing commit for transaction: " + transactionId - + ", but have already commited transaction with that id"); - throw new IOException("Corrupted transaction log"); - } - pendingTransactionsById.remove(transactionId); - commitedTransactionsById.put(transactionId, updates); - commitCount++; - - } - totalEdits++; - - if (reporter != null && (totalEdits % reportInterval) == 0) { - reporter.progress(); - } - } - if (LOG.isDebugEnabled()) { - LOG.debug("Read " + totalEdits + " tranasctional operations (skipped " - + skippedEdits + " because sequence id <= " + maxSeqID + "): " - + startCount + " starts, " + writeCount + " writes, " + abortCount - + " aborts, and " + commitCount + " commits."); - } - } finally { - logReader.close(); - } - - if (pendingTransactionsById.size() > 0) { - LOG - .info("Region log has " - + pendingTransactionsById.size() - + " unfinished transactions. Going to the transaction log to resolve"); - throw new RuntimeException("Transaction log not yet implemented"); - } - */ - - return commitedTransactionsById; - } -} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegion.java b/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegion.java deleted file mode 100644 index 589a1516ff8..00000000000 --- a/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegion.java +++ /dev/null @@ -1,718 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver.transactional; - -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.NavigableSet; -import java.util.Set; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.Map.Entry; -import java.util.concurrent.atomic.AtomicInteger; - -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.HBaseConfiguration; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.LeaseException; -import org.apache.hadoop.hbase.LeaseListener; -import org.apache.hadoop.hbase.Leases; -import org.apache.hadoop.hbase.Leases.LeaseStillHeldException; -import org.apache.hadoop.hbase.client.transactional.UnknownTransactionException; -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.regionserver.FlushRequester; -import org.apache.hadoop.hbase.regionserver.HLog; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.transactional.TransactionState.Status; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.util.Progressable; - -/** - * Regionserver which provides transactional support for atomic transactions. - * This is achieved with optimistic concurrency control (see - * http://www.seas.upenn.edu/~zives/cis650/papers/opt-cc.pdf). We keep track - * read and write sets for each transaction, and hold off on processing the - * writes. To decide to commit a transaction we check its read sets with all - * transactions that have committed while it was running for overlaps. - *

                  - * Because transactions can span multiple regions, all regions must agree to - * commit a transactions. The client side of this commit protocol is encoded in - * org.apache.hadoop.hbase.client.transactional.TransactionManger - *

                  - * In the event of an failure of the client mid-commit, (after we voted yes), we - * will have to consult the transaction log to determine the final decision of - * the transaction. This is not yet implemented. - */ -public class TransactionalRegion extends HRegion { - - private static final String LEASE_TIME = "hbase.transaction.leaseTime"; - private static final int DEFAULT_LEASE_TIME = 60 * 1000; - private static final int LEASE_CHECK_FREQUENCY = 1000; - - private static final String OLD_TRANSACTION_FLUSH = "hbase.transaction.flush"; - private static final int DEFAULT_OLD_TRANSACTION_FLUSH = 100; // Do a flush if we have this many old transactions.. - - - static final Log LOG = LogFactory.getLog(TransactionalRegion.class); - - // Collection of active transactions (PENDING) keyed by id. - protected Map transactionsById = new HashMap(); - - // Map of recent transactions that are COMMIT_PENDING or COMMITED keyed by - // their sequence number - private SortedMap commitedTransactionsBySequenceNumber = Collections - .synchronizedSortedMap(new TreeMap()); - - // Collection of transactions that are COMMIT_PENDING - private Set commitPendingTransactions = Collections - .synchronizedSet(new HashSet()); - - private final Leases transactionLeases; - private AtomicInteger nextSequenceId = new AtomicInteger(0); - private Object commitCheckLock = new Object(); - private TransactionalHLogManager logManager; - private final int oldTransactionFlushTrigger; - - /** - * @param basedir - * @param log - * @param fs - * @param conf - * @param regionInfo - * @param flushListener - */ - public TransactionalRegion(final Path basedir, final HLog log, - final FileSystem fs, final HBaseConfiguration conf, - final HRegionInfo regionInfo, final FlushRequester flushListener) { - super(basedir, log, fs, conf, regionInfo, flushListener); - transactionLeases = new Leases(conf.getInt(LEASE_TIME, DEFAULT_LEASE_TIME), - LEASE_CHECK_FREQUENCY); - logManager = new TransactionalHLogManager(this); - oldTransactionFlushTrigger = conf.getInt(OLD_TRANSACTION_FLUSH, DEFAULT_OLD_TRANSACTION_FLUSH); - } - - @Override - protected void doReconstructionLog(final Path oldLogFile, - final long minSeqId, final long maxSeqId, final Progressable reporter) - throws UnsupportedEncodingException, IOException { - super.doReconstructionLog(oldLogFile, minSeqId, maxSeqId, reporter); - - Map> commitedTransactionsById = logManager - .getCommitsFromLog(oldLogFile, minSeqId, reporter); - - if (commitedTransactionsById != null && commitedTransactionsById.size() > 0) { - LOG.debug("found " + commitedTransactionsById.size() - + " COMMITED transactions"); - - for (Entry> entry : commitedTransactionsById - .entrySet()) { - LOG.debug("Writing " + entry.getValue().size() - + " updates for transaction " + entry.getKey()); - for (BatchUpdate b : entry.getValue()) { - super.batchUpdate(b, true); // These are walled so they live forever - } - } - - // LOG.debug("Flushing cache"); // We must trigger a cache flush, - // otherwise - // we will would ignore the log on subsequent failure - // if (!super.flushcache()) { - // LOG.warn("Did not flush cache"); - // } - } - } - - /** - * We need to make sure that we don't complete a cache flush between running - * transactions. If we did, then we would not find all log messages needed to - * restore the transaction, as some of them would be before the last - * "complete" flush id. - */ - @Override - protected long getCompleteCacheFlushSequenceId(final long currentSequenceId) { - long minPendingStartSequenceId = currentSequenceId; - for (TransactionState transactionState : transactionsById.values()) { - minPendingStartSequenceId = Math.min(minPendingStartSequenceId, - transactionState.getHLogStartSequenceId()); - } - return minPendingStartSequenceId; - } - - /** - * @param transactionId - * @throws IOException - */ - public void beginTransaction(final long transactionId) throws IOException { - String key = String.valueOf(transactionId); - if (transactionsById.get(key) != null) { - TransactionState alias = getTransactionState(transactionId); - if (alias != null) { - alias.setStatus(Status.ABORTED); - retireTransaction(alias); - } - LOG.error("Existing trasaction with id ["+key+"] in region ["+super.getRegionInfo().getRegionNameAsString()+"]"); - throw new IOException("Already exiting transaction id: " + key); - } - - TransactionState state = new TransactionState(transactionId, super.getLog() - .getSequenceNumber(), super.getRegionInfo()); - - // Order is important here ... - List commitPendingCopy = new LinkedList(commitPendingTransactions); - for (TransactionState commitPending : commitPendingCopy) { - state.addTransactionToCheck(commitPending); - } - state.setStartSequenceNumber(nextSequenceId.get()); - - transactionsById.put(String.valueOf(key), state); - try { - transactionLeases.createLease(key, new TransactionLeaseListener(key)); - } catch (LeaseStillHeldException e) { - LOG.error("Lease still held for ["+key+"] in region ["+super.getRegionInfo().getRegionNameAsString()+"]"); - throw new RuntimeException(e); - } - LOG.debug("Begining transaction " + key + " in region " - + super.getRegionInfo().getRegionNameAsString()); - logManager.writeStartToLog(transactionId); - - maybeTriggerOldTransactionFlush(); - } - - /** - * Fetch a single data item. - * - * @param transactionId - * @param row - * @param column - * @return column value - * @throws IOException - */ - public Cell get(final long transactionId, final byte[] row, - final byte[] column) throws IOException { - Cell[] results = get(transactionId, row, column, 1); - return (results == null || results.length == 0) ? null : results[0]; - } - - /** - * Fetch multiple versions of a single data item - * - * @param transactionId - * @param row - * @param column - * @param numVersions - * @return array of values one element per version - * @throws IOException - */ - public Cell[] get(final long transactionId, final byte[] row, - final byte[] column, final int numVersions) throws IOException { - return get(transactionId, row, column, Long.MAX_VALUE, numVersions); - } - - /** - * Fetch multiple versions of a single data item, with timestamp. - * - * @param transactionId - * @param row - * @param column - * @param timestamp - * @param numVersions - * @return array of values one element per version that matches the timestamp - * @throws IOException - */ - public Cell[] get(final long transactionId, final byte[] row, - final byte[] column, final long timestamp, final int numVersions) - throws IOException { - TransactionState state = getTransactionState(transactionId); - - state.addRead(row); - - Cell[] localCells = state.localGet(row, column, timestamp); - - if (localCells != null && localCells.length > 0) { - LOG - .trace("Transactional get of something we've written in the same transaction " - + transactionId); - LOG.trace("row: " + Bytes.toString(row)); - LOG.trace("col: " + Bytes.toString(column)); - LOG.trace("numVersions: " + numVersions); - for (Cell cell : localCells) { - LOG.trace("cell: " + Bytes.toString(cell.getValue())); - } - - if (numVersions > 1) { - // FIX THIS PROFLIGACY CONVERTING RESULT OF get. - Cell[] globalCells = Cell.createSingleCellArray(get(row, column, timestamp, numVersions - 1)); - Cell[] result = new Cell[globalCells.length + localCells.length]; - System.arraycopy(localCells, 0, result, 0, localCells.length); - System.arraycopy(globalCells, 0, result, localCells.length, - globalCells.length); - return result; - } - return localCells; - } - - return Cell.createSingleCellArray(get(row, column, timestamp, numVersions)); - } - - /** - * Fetch all the columns for the indicated row at a specified timestamp. - * Returns a TreeMap that maps column names to values. - * - * @param transactionId - * @param row - * @param columns Array of columns you'd like to retrieve. When null, get all. - * @param ts - * @return Map values - * @throws IOException - */ - public Map getFull(final long transactionId, final byte[] row, - final NavigableSet columns, final long ts) throws IOException { - TransactionState state = getTransactionState(transactionId); - - state.addRead(row); - - Map localCells = state.localGetFull(row, columns, ts); - - if (localCells != null && localCells.size() > 0) { - LOG - .trace("Transactional get of something we've written in the same transaction " - + transactionId); - LOG.trace("row: " + Bytes.toString(row)); - for (Entry entry : localCells.entrySet()) { - LOG.trace("col: " + Bytes.toString(entry.getKey())); - LOG.trace("cell: " + Bytes.toString(entry.getValue().getValue())); - } - - Map internalResults = getFull(row, columns, ts, 1, null); - internalResults.putAll(localCells); - return internalResults; - } - - return getFull(row, columns, ts, 1, null); - } - - /** - * Return an iterator that scans over the HRegion, returning the indicated - * columns for only the rows that match the data filter. This Iterator must be - * closed by the caller. - * - * @param transactionId - * @param cols columns to scan. If column name is a column family, all columns - * of the specified column family are returned. Its also possible to pass a - * regex in the column qualifier. A column qualifier is judged to be a regex - * if it contains at least one of the following characters: - * \+|^&*$[]]}{)(. - * @param firstRow row which is the starting point of the scan - * @param timestamp only return rows whose timestamp is <= this value - * @param filter row filter - * @return InternalScanner - * @throws IOException - */ - public InternalScanner getScanner(final long transactionId, - final byte[][] cols, final byte[] firstRow, final long timestamp, - final RowFilterInterface filter) throws IOException { - TransactionState state = getTransactionState(transactionId); - state.addScan(firstRow, filter); - return new ScannerWrapper(transactionId, super.getScanner(cols, firstRow, - timestamp, filter)); - } - - /** - * Add a write to the transaction. Does not get applied until commit process. - * - * @param transactionId - * @param b - * @throws IOException - */ - public void batchUpdate(final long transactionId, final BatchUpdate b) - throws IOException { - TransactionState state = getTransactionState(transactionId); - state.addWrite(b); - logManager.writeUpdateToLog(transactionId, b); - } - - /** - * Add a delete to the transaction. Does not get applied until commit process. - * FIXME, not sure about this approach - * - * @param transactionId - * @param row - * @param timestamp - * @throws IOException - */ - public void deleteAll(final long transactionId, final byte[] row, - final long timestamp) throws IOException { - TransactionState state = getTransactionState(transactionId); - long now = System.currentTimeMillis(); - - for (Store store : super.stores.values()) { - List keyvalues = new ArrayList(); - store.getFull(new KeyValue(row, timestamp), - null, null, ALL_VERSIONS, null, keyvalues, now); - BatchUpdate deleteUpdate = new BatchUpdate(row, timestamp); - - for (KeyValue key : keyvalues) { - deleteUpdate.delete(key.getColumn()); - } - - state.addWrite(deleteUpdate); - logManager.writeUpdateToLog(transactionId, deleteUpdate); - - } - - } - - /** - * @param transactionId - * @return true if commit is successful - * @throws IOException - */ - public boolean commitRequest(final long transactionId) throws IOException { - synchronized (commitCheckLock) { - TransactionState state = getTransactionState(transactionId); - if (state == null) { - return false; - } - - if (hasConflict(state)) { - state.setStatus(Status.ABORTED); - retireTransaction(state); - return false; - } - - // No conflicts, we can commit. - LOG.trace("No conflicts for transaction " + transactionId - + " found in region " + super.getRegionInfo().getRegionNameAsString() - + ". Voting for commit"); - state.setStatus(Status.COMMIT_PENDING); - - // If there are writes we must keep record off the transaction - if (state.getWriteSet().size() > 0) { - // Order is important - commitPendingTransactions.add(state); - state.setSequenceNumber(nextSequenceId.getAndIncrement()); - commitedTransactionsBySequenceNumber.put(state.getSequenceNumber(), - state); - } - - return true; - } - } - - private boolean hasConflict(final TransactionState state) { - // Check transactions that were committed while we were running - for (int i = state.getStartSequenceNumber(); i < nextSequenceId.get(); i++) { - TransactionState other = commitedTransactionsBySequenceNumber.get(i); - if (other == null) { - continue; - } - state.addTransactionToCheck(other); - } - - return state.hasConflict(); - } - - /** - * Commit the transaction. - * - * @param transactionId - * @throws IOException - */ - public void commit(final long transactionId) throws IOException { - TransactionState state; - try { - state = getTransactionState(transactionId); - } catch (UnknownTransactionException e) { - LOG.fatal("Asked to commit unknown transaction: " + transactionId - + " in region " + super.getRegionInfo().getRegionNameAsString()); - // FIXME Write to the transaction log that this transaction was corrupted - throw e; - } - - if (!state.getStatus().equals(Status.COMMIT_PENDING)) { - LOG.fatal("Asked to commit a non pending transaction"); - // FIXME Write to the transaction log that this transaction was corrupted - throw new IOException("commit failure"); - } - - commit(state); - } - - /** - * Commit the transaction. - * - * @param transactionId - * @throws IOException - */ - public void abort(final long transactionId) throws IOException { - TransactionState state; - try { - state = getTransactionState(transactionId); - } catch (UnknownTransactionException e) { - LOG.error("Asked to abort unknown transaction: " + transactionId); - return; - } - - state.setStatus(Status.ABORTED); - - if (state.getWriteSet().size() > 0) { - logManager.writeAbortToLog(state.getTransactionId()); - } - - // Following removes needed if we have voted - if (state.getSequenceNumber() != null) { - commitedTransactionsBySequenceNumber.remove(state.getSequenceNumber()); - } - commitPendingTransactions.remove(state); - - retireTransaction(state); - } - - private void commit(final TransactionState state) throws IOException { - - LOG.debug("Commiting transaction: " + state.toString() + " to " - + super.getRegionInfo().getRegionNameAsString()); - - if (state.getWriteSet().size() > 0) { - logManager.writeCommitToLog(state.getTransactionId()); - } - - for (BatchUpdate update : state.getWriteSet()) { - this.batchUpdate(update, false); // Don't need to WAL these - // FIME, maybe should be walled so we don't need to look so far back. - } - - state.setStatus(Status.COMMITED); - if (state.getWriteSet().size() > 0 - && !commitPendingTransactions.remove(state)) { - LOG - .fatal("Commiting a non-query transaction that is not in commitPendingTransactions"); - throw new IOException("commit failure"); // FIXME, how to handle? - } - retireTransaction(state); - } - - // Cancel leases, and removed from lease lookup. This transaction may still - // live in commitedTransactionsBySequenceNumber and commitPendingTransactions - private void retireTransaction(final TransactionState state) { - String key = String.valueOf(state.getTransactionId()); - try { - transactionLeases.cancelLease(key); - } catch (LeaseException e) { - // Ignore - } - - transactionsById.remove(key); - } - - protected TransactionState getTransactionState(final long transactionId) - throws UnknownTransactionException { - String key = String.valueOf(transactionId); - TransactionState state = null; - - state = transactionsById.get(key); - - if (state == null) { - LOG.trace("Unknown transaction: " + key); - throw new UnknownTransactionException(key); - } - - try { - transactionLeases.renewLease(key); - } catch (LeaseException e) { - throw new RuntimeException(e); - } - - return state; - } - - private void maybeTriggerOldTransactionFlush() { - if (commitedTransactionsBySequenceNumber.size() > oldTransactionFlushTrigger) { - removeUnNeededCommitedTransactions(); - } - } - - /** - * Cleanup references to committed transactions that are no longer needed. - * - */ - synchronized void removeUnNeededCommitedTransactions() { - Integer minStartSeqNumber = getMinStartSequenceNumber(); - if (minStartSeqNumber == null) { - minStartSeqNumber = Integer.MAX_VALUE; // Remove all - } - - int numRemoved = 0; - // Copy list to avoid conc update exception - for (Entry entry : new LinkedList>( - commitedTransactionsBySequenceNumber.entrySet())) { - if (entry.getKey() >= minStartSeqNumber) { - break; - } - numRemoved = numRemoved - + (commitedTransactionsBySequenceNumber.remove(entry.getKey()) == null ? 0 - : 1); - numRemoved++; - } - - if (LOG.isDebugEnabled()) { - StringBuilder debugMessage = new StringBuilder(); - if (numRemoved > 0) { - debugMessage.append("Removed ").append(numRemoved).append( - " commited transactions"); - - if (minStartSeqNumber == Integer.MAX_VALUE) { - debugMessage.append("with any sequence number"); - } else { - debugMessage.append("with sequence lower than ").append( - minStartSeqNumber).append("."); - } - if (!commitedTransactionsBySequenceNumber.isEmpty()) { - debugMessage.append(" Still have ").append( - commitedTransactionsBySequenceNumber.size()).append(" left."); - } else { - debugMessage.append("None left."); - } - LOG.debug(debugMessage.toString()); - } else if (commitedTransactionsBySequenceNumber.size() > 0) { - debugMessage.append( - "Could not remove any transactions, and still have ").append( - commitedTransactionsBySequenceNumber.size()).append(" left"); - LOG.debug(debugMessage.toString()); - } - } - } - - private Integer getMinStartSequenceNumber() { - Integer min = null; - for (TransactionState transactionState : transactionsById.values()) { - if (min == null || transactionState.getStartSequenceNumber() < min) { - min = transactionState.getStartSequenceNumber(); - } - } - return min; - } - - // TODO, resolve from the global transaction log - protected void resolveTransactionFromLog() { - throw new RuntimeException("Globaql transaction log is not Implemented"); - } - - private class TransactionLeaseListener implements LeaseListener { - private final String transactionName; - - TransactionLeaseListener(final String n) { - this.transactionName = n; - } - - public void leaseExpired() { - LOG.info("Transaction " + this.transactionName + " lease expired"); - TransactionState s = null; - synchronized (transactionsById) { - s = transactionsById.remove(transactionName); - } - if (s == null) { - LOG.warn("Unknown transaction expired " + this.transactionName); - return; - } - - switch (s.getStatus()) { - case PENDING: - s.setStatus(Status.ABORTED); // Other transactions may have a ref - break; - case COMMIT_PENDING: - LOG.info("Transaction " + s.getTransactionId() - + " expired in COMMIT_PENDING state"); - LOG.info("Checking transaction status in transaction log"); - resolveTransactionFromLog(); - break; - default: - LOG.warn("Unexpected status on expired lease"); - } - } - } - - /** Wrapper which keeps track of rows returned by scanner. */ - private class ScannerWrapper implements InternalScanner { - private long transactionId; - private InternalScanner scanner; - - /** - * @param transactionId - * @param scanner - * @throws UnknownTransactionException - */ - public ScannerWrapper(final long transactionId, - final InternalScanner scanner) throws UnknownTransactionException { - - this.transactionId = transactionId; - this.scanner = scanner; - } - - public void close() throws IOException { - scanner.close(); - } - - public boolean isMultipleMatchScanner() { - return scanner.isMultipleMatchScanner(); - } - - public boolean isWildcardScanner() { - return scanner.isWildcardScanner(); - } - - public boolean next(List results) throws IOException { - boolean result = scanner.next(results); - TransactionState state = getTransactionState(transactionId); - - if (result) { - // TODO: Is this right???? St.Ack - byte [] row = results.get(0).getRow(); - Map localWrites = state.localGetFull(row, null, - Integer.MAX_VALUE); - if (localWrites != null) { - LOG.info("Scanning over row that has been writen to " + transactionId); - for (Entry entry : localWrites.entrySet()) { - // TODO: Is this right??? - results.add(new KeyValue(row, entry.getKey(), - entry.getValue().getTimestamp(), entry.getValue().getValue())); - } - } - } - - return result; - } - } -} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionServer.java b/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionServer.java deleted file mode 100644 index 4b43b12814e..00000000000 --- a/src/java/org/apache/hadoop/hbase/regionserver/transactional/TransactionalRegionServer.java +++ /dev/null @@ -1,304 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver.transactional; - -import java.io.IOException; -import java.lang.Thread.UncaughtExceptionHandler; -import java.util.Arrays; -import java.util.Map; -import java.util.NavigableSet; -import java.util.TreeSet; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HBaseConfiguration; -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.NotServingRegionException; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -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.HbaseMapWritable; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion; -import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.io.MapWritable; -import org.apache.hadoop.util.Progressable; - -/** - * RegionServer with support for transactions. Transactional logic is at the - * region level, so we mostly just delegate to the appropriate - * TransactionalRegion. - */ -public class TransactionalRegionServer extends HRegionServer implements - TransactionalRegionInterface { - static final Log LOG = LogFactory.getLog(TransactionalRegionServer.class); - - private final CleanOldTransactionsChore cleanOldTransactionsThread; - - /** - * @param conf - * @throws IOException - */ - public TransactionalRegionServer(final HBaseConfiguration conf) - throws IOException { - this(new HServerAddress(conf.get(REGIONSERVER_ADDRESS, - DEFAULT_REGIONSERVER_ADDRESS)), conf); - } - - /** - * @param address - * @param conf - * @throws IOException - */ - public TransactionalRegionServer(final HServerAddress address, - final HBaseConfiguration conf) throws IOException { - super(address, conf); - cleanOldTransactionsThread = new CleanOldTransactionsChore(this, - super.stopRequested); - } - - @Override - public long getProtocolVersion(final String protocol, final long clientVersion) - throws IOException { - if (protocol.equals(TransactionalRegionInterface.class.getName())) { - return HBaseRPCProtocolVersion.versionID; - } - return super.getProtocolVersion(protocol, clientVersion); - } - - @Override - protected void init(final MapWritable c) throws IOException { - super.init(c); - String n = Thread.currentThread().getName(); - UncaughtExceptionHandler handler = new UncaughtExceptionHandler() { - public void uncaughtException(final Thread t, final Throwable e) { - abort(); - LOG.fatal("Set stop flag in " + t.getName(), e); - } - }; - Threads.setDaemonThreadRunning(this.cleanOldTransactionsThread, n - + ".oldTransactionCleaner", handler); - - } - - @Override - protected HRegion instantiateRegion(final HRegionInfo regionInfo) - throws IOException { - HRegion r = new TransactionalRegion(HTableDescriptor.getTableDir(super - .getRootDir(), regionInfo.getTableDesc().getName()), super.hlog, super - .getFileSystem(), super.conf, regionInfo, super.getFlushRequester()); - r.initialize(null, new Progressable() { - public void progress() { - addProcessingMessage(regionInfo); - } - }); - return r; - } - - protected TransactionalRegion getTransactionalRegion(final byte[] regionName) - throws NotServingRegionException { - return (TransactionalRegion) super.getRegion(regionName); - } - - public void abort(final byte[] regionName, final long transactionId) - throws IOException { - checkOpen(); - super.getRequestCount().incrementAndGet(); - try { - getTransactionalRegion(regionName).abort(transactionId); - } catch (IOException e) { - checkFileSystem(); - throw e; - } - } - - public void batchUpdate(final long transactionId, final byte[] regionName, - final BatchUpdate b) throws IOException { - checkOpen(); - super.getRequestCount().incrementAndGet(); - try { - getTransactionalRegion(regionName).batchUpdate(transactionId, b); - } catch (IOException e) { - checkFileSystem(); - throw e; - } - } - - public void commit(final byte[] regionName, final long transactionId) - throws IOException { - checkOpen(); - super.getRequestCount().incrementAndGet(); - try { - getTransactionalRegion(regionName).commit(transactionId); - } catch (IOException e) { - checkFileSystem(); - throw e; - } - } - - public boolean commitRequest(final byte[] regionName, final long transactionId) - throws IOException { - checkOpen(); - super.getRequestCount().incrementAndGet(); - try { - return getTransactionalRegion(regionName).commitRequest(transactionId); - } catch (IOException e) { - checkFileSystem(); - throw e; - } - } - - public Cell get(final long transactionId, final byte[] regionName, - final byte[] row, final byte[] column) throws IOException { - checkOpen(); - super.getRequestCount().incrementAndGet(); - try { - return getTransactionalRegion(regionName).get(transactionId, row, column); - } catch (IOException e) { - checkFileSystem(); - throw e; - } - } - - public Cell[] get(final long transactionId, final byte[] regionName, - final byte[] row, final byte[] column, final int numVersions) - throws IOException { - checkOpen(); - super.getRequestCount().incrementAndGet(); - try { - return getTransactionalRegion(regionName).get(transactionId, row, column, - numVersions); - } catch (IOException e) { - checkFileSystem(); - throw e; - } - } - - public Cell[] get(final long transactionId, final byte[] regionName, - final byte[] row, final byte[] column, final long timestamp, - final int numVersions) throws IOException { - checkOpen(); - super.getRequestCount().incrementAndGet(); - try { - return getTransactionalRegion(regionName).get(transactionId, row, column, - timestamp, numVersions); - } catch (IOException e) { - checkFileSystem(); - throw e; - } - } - - public RowResult getRow(final long transactionId, final byte[] regionName, - final byte[] row, final long ts) throws IOException { - return getRow(transactionId, regionName, row, null, ts); - } - - public RowResult getRow(final long transactionId, final byte[] regionName, - final byte[] row, final byte[][] columns) throws IOException { - return getRow(transactionId, regionName, row, columns, - HConstants.LATEST_TIMESTAMP); - } - - public RowResult getRow(final long transactionId, final byte[] regionName, - final byte[] row, final byte[][] columns, final long ts) - throws IOException { - checkOpen(); - super.getRequestCount().incrementAndGet(); - try { - // convert the columns array into a set so it's easy to check later. - NavigableSet columnSet = null; - if (columns != null) { - columnSet = new TreeSet(Bytes.BYTES_COMPARATOR); - columnSet.addAll(Arrays.asList(columns)); - } - - TransactionalRegion region = getTransactionalRegion(regionName); - Map map = region.getFull(transactionId, row, columnSet, ts); - HbaseMapWritable result = new HbaseMapWritable(); - result.putAll(map); - return new RowResult(row, result); - } catch (IOException e) { - checkFileSystem(); - throw e; - } - - } - - public void deleteAll(final long transactionId, final byte[] regionName, - final byte[] row, final long timestamp) throws IOException { - checkOpen(); - super.getRequestCount().incrementAndGet(); - try { - TransactionalRegion region = getTransactionalRegion(regionName); - region.deleteAll(transactionId, row, timestamp); - } catch (IOException e) { - checkFileSystem(); - throw e; - } - } - - public long openScanner(final long transactionId, final byte[] regionName, - final byte[][] cols, final byte[] firstRow, final long timestamp, - final RowFilterInterface filter) throws IOException { - checkOpen(); - NullPointerException npe = null; - if (regionName == null) { - npe = new NullPointerException("regionName is null"); - } else if (cols == null) { - npe = new NullPointerException("columns to scan is null"); - } else if (firstRow == null) { - npe = new NullPointerException("firstRow for scanner is null"); - } - if (npe != null) { - IOException io = new IOException("Invalid arguments to openScanner"); - io.initCause(npe); - throw io; - } - super.getRequestCount().incrementAndGet(); - try { - TransactionalRegion r = getTransactionalRegion(regionName); - long scannerId = -1L; - InternalScanner s = r.getScanner(transactionId, cols, firstRow, - timestamp, filter); - scannerId = super.addScanner(s); - return scannerId; - } catch (IOException e) { - LOG.error("Error opening scanner (fsOk: " + this.fsOk + ")", - RemoteExceptionHandler.checkIOException(e)); - checkFileSystem(); - throw e; - } - } - - public void beginTransaction(final long transactionId, final byte[] regionName) - throws IOException { - getTransactionalRegion(regionName).beginTransaction(transactionId); - } - -} diff --git a/src/java/org/apache/hadoop/hbase/rest/RowController.java b/src/java/org/apache/hadoop/hbase/rest/RowController.java index 35d3c9a07a7..668319d23cc 100644 --- a/src/java/org/apache/hadoop/hbase/rest/RowController.java +++ b/src/java/org/apache/hadoop/hbase/rest/RowController.java @@ -24,7 +24,9 @@ import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.rest.descriptors.RowUpdateDescriptor; @@ -82,7 +84,6 @@ public class RowController extends AbstractController { throws HBaseRestException { RowModel innerModel = getModel(); - BatchUpdate b; RowUpdateDescriptor rud = parser .getRowUpdateDescriptor(input, pathSegments); @@ -92,14 +93,15 @@ public class RowController extends AbstractController { return; } - b = new BatchUpdate(rud.getRowName()); + Put put = new Put(Bytes.toBytes(rud.getRowName())); for (byte[] key : rud.getColVals().keySet()) { - b.put(key, rud.getColVals().get(key)); + byte [][] famAndQf = KeyValue.parseColumn(key); + put.add(famAndQf[0], famAndQf[1], rud.getColVals().get(key)); } try { - innerModel.post(rud.getTableName().getBytes(), b); + innerModel.post(rud.getTableName().getBytes(), put); s.setOK(); } catch (HBaseRestException e) { s.setUnsupportedMediaType(e.getMessage()); diff --git a/src/java/org/apache/hadoop/hbase/rest/RowModel.java b/src/java/org/apache/hadoop/hbase/rest/RowModel.java index 1b8ce8c4f80..fc7ea88764d 100644 --- a/src/java/org/apache/hadoop/hbase/rest/RowModel.java +++ b/src/java/org/apache/hadoop/hbase/rest/RowModel.java @@ -25,8 +25,13 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.rest.descriptors.TimestampsDescriptor; @@ -41,44 +46,51 @@ public class RowModel extends AbstractModel { super.initialize(conf, admin); } + @Deprecated public RowResult get(byte[] tableName, byte[] rowName) throws HBaseRestException { + return get(tableName, new Get(rowName)).getRowResult(); + } + + public Result get(byte[] tableName, Get get) + throws HBaseRestException { try { HTable table = new HTable(tableName); - return table.getRow(rowName); + return table.get(get); } catch (IOException e) { throw new HBaseRestException(e); } } - + + @Deprecated public RowResult get(byte[] tableName, byte[] rowName, byte[][] columns) throws HBaseRestException { - try { - HTable table = new HTable(tableName); - return table.getRow(rowName, columns); - } catch (IOException e) { - throw new HBaseRestException(e); + Get get = new Get(rowName); + for(byte [] column : columns) { + byte [][] famAndQf = KeyValue.parseColumn(column); + get.addColumn(famAndQf[0], famAndQf[1]); } + return get(tableName, get).getRowResult(); } + @Deprecated public RowResult get(byte[] tableName, byte[] rowName, byte[][] columns, long timestamp) throws HBaseRestException { - try { - HTable table = new HTable(tableName); - return table.getRow(rowName, columns, timestamp); - } catch (IOException e) { - throw new HBaseRestException(e); + Get get = new Get(rowName); + for(byte [] column : columns) { + byte [][] famAndQf = KeyValue.parseColumn(column); + get.addColumn(famAndQf[0], famAndQf[1]); } + get.setTimeStamp(timestamp); + return get(tableName, get).getRowResult(); } - + + @Deprecated public RowResult get(byte[] tableName, byte[] rowName, long timestamp) throws HBaseRestException { - try { - HTable table = new HTable(tableName); - return table.getRow(rowName, timestamp); - } catch (IOException e) { - throw new HBaseRestException(e); - } + Get get = new Get(rowName); + get.setTimeStamp(timestamp); + return get(tableName, get).getRowResult(); } public TimestampsDescriptor getTimestamps( @@ -98,41 +110,48 @@ public class RowModel extends AbstractModel { } - public void post(byte[] tableName, BatchUpdate b) throws HBaseRestException { + public void post(byte[] tableName, Put put) throws HBaseRestException { try { HTable table = new HTable(tableName); - table.commit(b); + table.put(put); } catch (IOException e) { throw new HBaseRestException(e); } } - public void post(byte[] tableName, List b) + public void post(byte[] tableName, List puts) throws HBaseRestException { try { HTable table = new HTable(tableName); - table.commit(b); + table.put(puts); } catch (IOException e) { throw new HBaseRestException(e); } } - + + @Deprecated public void delete(byte[] tableName, byte[] rowName) throws HBaseRestException { - try { - HTable table = new HTable(tableName); - table.deleteAll(rowName); - } catch (IOException e) { - throw new HBaseRestException(e); - } + Delete delete = new Delete(rowName); + delete(tableName, delete); } - public void delete(byte[] tableName, byte[] rowName, byte[][] columns) throws HBaseRestException { + @Deprecated + public void delete(byte[] tableName, byte[] rowName, byte[][] columns) + throws HBaseRestException { + Delete delete = new Delete(rowName); + for(byte [] column : columns) { + byte [][] famAndQf = KeyValue.parseColumn(column); + delete.deleteColumn(famAndQf[0], famAndQf[1]); + } + delete(tableName, delete); + } + + public void delete(byte[] tableName, Delete delete) + throws HBaseRestException { try { HTable table = new HTable(tableName); - for (byte[] column : columns) { - table.deleteAll(rowName, column); - } + table.delete(delete); } catch (IOException e) { throw new HBaseRestException(e); } diff --git a/src/java/org/apache/hadoop/hbase/rest/ScannerModel.java b/src/java/org/apache/hadoop/hbase/rest/ScannerModel.java index c95143080cc..aef1449bb3b 100644 --- a/src/java/org/apache/hadoop/hbase/rest/ScannerModel.java +++ b/src/java/org/apache/hadoop/hbase/rest/ScannerModel.java @@ -1,5 +1,5 @@ /** - * Copyright 2007 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -29,9 +29,10 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; 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.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.filter.RowFilterInterface; -import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.rest.descriptors.ScannerIdentifier; import org.apache.hadoop.hbase.rest.exception.HBaseRestException; @@ -49,20 +50,20 @@ public class ScannerModel extends AbstractModel { // protected static class ScannerMaster { - protected static final Map scannerMap = new ConcurrentHashMap(); + protected static final Map scannerMap = new ConcurrentHashMap(); protected static final AtomicInteger nextScannerId = new AtomicInteger(1); - public Integer addScanner(Scanner scanner) { + public Integer addScanner(ResultScanner scanner) { Integer i = Integer.valueOf(nextScannerId.getAndIncrement()); scannerMap.put(i, scanner); return i; } - public Scanner getScanner(Integer id) { + public ResultScanner getScanner(Integer id) { return scannerMap.get(id); } - public Scanner removeScanner(Integer id) { + public ResultScanner removeScanner(Integer id) { return scannerMap.remove(id); } @@ -71,7 +72,7 @@ public class ScannerModel extends AbstractModel { * id of scanner to close */ public void scannerClose(Integer id) { - Scanner s = scannerMap.remove(id); + ResultScanner s = scannerMap.remove(id); s.close(); } } @@ -79,7 +80,7 @@ public class ScannerModel extends AbstractModel { protected static final ScannerMaster scannerMaster = new ScannerMaster(); /** - * returns the next numResults RowResults from the Scaner mapped to Integer + * returns the next numResults Results from the Scaner mapped to Integer * id. If the end of the table is reached, the scanner is closed and all * succesfully retrieved rows are returned. * @@ -90,14 +91,14 @@ public class ScannerModel extends AbstractModel { * @return all successfully retrieved rows. * @throws org.apache.hadoop.hbase.rest.exception.HBaseRestException */ - public RowResult[] scannerGet(Integer id, Long numRows) + public Result[] scannerGet(Integer id, Long numRows) throws HBaseRestException { try { - ArrayList a; - Scanner s; - RowResult r; + ArrayList a; + ResultScanner s; + Result r; - a = new ArrayList(); + a = new ArrayList(); s = scannerMaster.getScanner(id); if (s == null) { @@ -114,7 +115,7 @@ public class ScannerModel extends AbstractModel { } } - return a.toArray(new RowResult[0]); + return a.toArray(new Result[0]); } catch (IOException e) { throw new HBaseRestException(e); } @@ -129,13 +130,13 @@ public class ScannerModel extends AbstractModel { * @return all rows till end of table * @throws org.apache.hadoop.hbase.rest.exception.HBaseRestException */ - public RowResult[] scannerGet(Integer id) throws HBaseRestException { + public Result[] scannerGet(Integer id) throws HBaseRestException { try { - ArrayList a; - Scanner s; - RowResult r; + ArrayList a; + ResultScanner s; + Result r; - a = new ArrayList(); + a = new ArrayList(); s = scannerMaster.getScanner(id); while ((r = s.next()) != null) { @@ -144,14 +145,14 @@ public class ScannerModel extends AbstractModel { scannerMaster.scannerClose(id); - return a.toArray(new RowResult[0]); + return a.toArray(new Result[0]); } catch (IOException e) { throw new HBaseRestException(e); } } public boolean scannerClose(Integer id) throws HBaseRestException { - Scanner s = scannerMaster.removeScanner(id); + ResultScanner s = scannerMaster.removeScanner(id); if (s == null) { throw new HBaseRestException("Scanner id: " + id + " does not exist"); @@ -208,8 +209,11 @@ public class ScannerModel extends AbstractModel { try { HTable table; table = new HTable(tableName); + Scan scan = new Scan(); + scan.addColumns(columns); + scan.setTimeRange(0, timestamp); return new ScannerIdentifier(scannerMaster.addScanner(table.getScanner( - columns, HConstants.EMPTY_START_ROW, timestamp))); + scan))); } catch (IOException e) { throw new HBaseRestException(e); } @@ -225,8 +229,11 @@ public class ScannerModel extends AbstractModel { try { HTable table; table = new HTable(tableName); + Scan scan = new Scan(startRow); + scan.addColumns(columns); + scan.setTimeRange(0, timestamp); return new ScannerIdentifier(scannerMaster.addScanner(table.getScanner( - columns, startRow, timestamp))); + scan))); } catch (IOException e) { throw new HBaseRestException(e); } @@ -243,8 +250,12 @@ public class ScannerModel extends AbstractModel { try { HTable table; table = new HTable(tableName); + Scan scan = new Scan(); + scan.addColumns(columns); + scan.setTimeRange(0, timestamp); +// scan.setFilter(filter); return new ScannerIdentifier(scannerMaster.addScanner(table.getScanner( - columns, HConstants.EMPTY_START_ROW, timestamp, filter))); + scan))); } catch (IOException e) { throw new HBaseRestException(e); } @@ -261,8 +272,12 @@ public class ScannerModel extends AbstractModel { try { HTable table; table = new HTable(tableName); + Scan scan = new Scan(startRow); + scan.addColumns(columns); + scan.setTimeRange(0, timestamp); +// scan.setFilter(filter); return new ScannerIdentifier(scannerMaster.addScanner(table.getScanner( - columns, startRow, timestamp, filter))); + scan))); } catch (IOException e) { throw new HBaseRestException(e); } diff --git a/src/java/org/apache/hadoop/hbase/rest/TableModel.java b/src/java/org/apache/hadoop/hbase/rest/TableModel.java index 9e1524acea1..7b98c551d97 100644 --- a/src/java/org/apache/hadoop/hbase/rest/TableModel.java +++ b/src/java/org/apache/hadoop/hbase/rest/TableModel.java @@ -29,8 +29,9 @@ import org.apache.hadoop.hbase.HColumnDescriptor; 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.RowResult; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.rest.exception.HBaseRestException; import org.apache.hadoop.hbase.rest.serializer.IRestSerializer; import org.apache.hadoop.hbase.rest.serializer.ISerializable; @@ -48,7 +49,7 @@ public class TableModel extends AbstractModel { } // Get Methods - public RowResult[] get(byte[] tableName) throws HBaseRestException { + public Result[] get(byte [] tableName) throws HBaseRestException { return get(tableName, getColumns(tableName)); } @@ -63,26 +64,28 @@ public class TableModel extends AbstractModel { * @return resultant rows * @throws org.apache.hadoop.hbase.rest.exception.HBaseRestException */ - public RowResult[] get(byte[] tableName, byte[][] columnNames) + public Result[] get(byte [] tableName, byte[][] columnNames) throws HBaseRestException { try { - ArrayList a = new ArrayList(); + ArrayList a = new ArrayList(); HTable table = new HTable(tableName); - Scanner s = table.getScanner(columnNames); - RowResult r; + Scan scan = new Scan(); + scan.addColumns(columnNames); + ResultScanner s = table.getScanner(scan); + Result r; while ((r = s.next()) != null) { a.add(r); } - return a.toArray(new RowResult[0]); + return a.toArray(new Result[0]); } catch (Exception e) { throw new HBaseRestException(e); } } - protected boolean doesTableExist(byte[] tableName) throws HBaseRestException { + protected boolean doesTableExist(byte [] tableName) throws HBaseRestException { try { return this.admin.tableExists(tableName); } catch (IOException e) { @@ -90,7 +93,7 @@ public class TableModel extends AbstractModel { } } - protected void disableTable(byte[] tableName) throws HBaseRestException { + protected void disableTable(byte [] tableName) throws HBaseRestException { try { this.admin.disableTable(tableName); } catch (IOException e) { @@ -98,7 +101,7 @@ public class TableModel extends AbstractModel { } } - protected void enableTable(byte[] tableName) throws HBaseRestException { + protected void enableTable(byte [] tableName) throws HBaseRestException { try { this.admin.enableTable(tableName); } catch (IOException e) { @@ -110,7 +113,7 @@ public class TableModel extends AbstractModel { ArrayList columns) throws HBaseRestException { HTableDescriptor htc = null; try { - htc = this.admin.getTableDescriptor(tableName); + htc = this.admin.getTableDescriptor(Bytes.toBytes(tableName)); } catch (IOException e) { throw new HBaseRestException("Table does not exist"); } @@ -204,7 +207,7 @@ public class TableModel extends AbstractModel { * tableName not existing. * @throws org.apache.hadoop.hbase.rest.exception.HBaseRestException */ - public boolean post(byte[] tableName, HTableDescriptor htd) + public boolean post(byte [] tableName, HTableDescriptor htd) throws HBaseRestException { try { if (!this.admin.tableExists(tableName)) { @@ -225,7 +228,7 @@ public class TableModel extends AbstractModel { * @return true if table exists and deleted, false if table does not exist. * @throws org.apache.hadoop.hbase.rest.exception.HBaseRestException */ - public boolean delete(byte[] tableName) throws HBaseRestException { + public boolean delete(byte [] tableName) throws HBaseRestException { try { if (this.admin.tableExists(tableName)) { this.admin.disableTable(tableName); @@ -241,7 +244,7 @@ public class TableModel extends AbstractModel { public static class Regions implements ISerializable { byte[][] regionKey; - public Regions(byte[][] bs) { + public Regions(byte [][] bs) { super(); this.regionKey = bs; } diff --git a/src/java/org/apache/hadoop/hbase/rest/TimestampModel.java b/src/java/org/apache/hadoop/hbase/rest/TimestampModel.java index 0e876c54dc7..c4453d95adf 100644 --- a/src/java/org/apache/hadoop/hbase/rest/TimestampModel.java +++ b/src/java/org/apache/hadoop/hbase/rest/TimestampModel.java @@ -20,11 +20,19 @@ package org.apache.hadoop.hbase.rest; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; @@ -39,56 +47,80 @@ public class TimestampModel extends AbstractModel { super.initialize(conf, admin); } + public void delete(byte [] tableName, Delete delete) + throws HBaseRestException { + try { + HTable table = new HTable(tableName); + table.delete(delete); + } catch (IOException e) { + throw new HBaseRestException(e); + } + } + + @Deprecated public void delete(byte[] tableName, byte[] rowName, long timestamp) throws HBaseRestException { - try { - HTable table = new HTable(tableName); - table.deleteAll(rowName, timestamp); - } catch (IOException e) { - throw new HBaseRestException(e); - } + Delete delete = new Delete(rowName, timestamp, null); + delete(tableName, delete); } - + + @Deprecated public void delete(byte[] tableName, byte[] rowName, byte[][] columns, long timestamp) throws HBaseRestException { + Delete delete = new Delete(rowName, timestamp, null); + for(byte [] column : columns) { + byte [][] famAndQf = KeyValue.parseColumn(column); + delete.deleteColumn(famAndQf[0], famAndQf[1]); + } + delete(tableName, delete); + } + + public Result get(final byte [] tableName, final Get get) + throws HBaseRestException { try { HTable table = new HTable(tableName); - for (byte[] column : columns) { - table.deleteAll(rowName, column, timestamp); - } + return table.get(get); } catch (IOException e) { throw new HBaseRestException(e); } } - + + @Deprecated public Cell get(byte[] tableName, byte[] rowName, byte[] columnName, long timestamp) throws HBaseRestException { - try { - HTable table = new HTable(tableName); - return table.get(rowName, columnName, timestamp, 1)[0]; - } catch (IOException e) { - throw new HBaseRestException(e); - } + Get get = new Get(rowName); + byte [][] famAndQf = KeyValue.parseColumn(columnName); + get.addColumn(famAndQf[0], famAndQf[1]); + get.setTimeStamp(timestamp); + return get(tableName, get).getCellValue(famAndQf[0], famAndQf[1]); } + @Deprecated public Cell[] get(byte[] tableName, byte[] rowName, byte[] columnName, - long timestamp, int numVersions) throws HBaseRestException { - try { - HTable table = new HTable(tableName); - return table.get(rowName, columnName, timestamp, numVersions); - } catch (IOException e) { - throw new HBaseRestException(e); + long timestamp, int numVersions) throws IOException, HBaseRestException { + Get get = new Get(rowName); + byte [][] famAndQf = KeyValue.parseColumn(columnName); + get.addColumn(famAndQf[0], famAndQf[1]); + get.setTimeStamp(timestamp); + get.setMaxVersions(numVersions); + Result result = get(tableName, get); + List cells = new ArrayList(); + for(KeyValue kv : result.sorted()) { + cells.add(new Cell(kv.getValue(), kv.getTimestamp())); } + return cells.toArray(new Cell [0]); } + @Deprecated public RowResult get(byte[] tableName, byte[] rowName, byte[][] columns, long timestamp) throws HBaseRestException { - try { - HTable table = new HTable(tableName); - return table.getRow(rowName, columns, timestamp); - } catch (IOException e) { - throw new HBaseRestException(e); + Get get = new Get(rowName); + for(byte [] column : columns) { + byte [][] famAndQf = KeyValue.parseColumn(column); + get.addColumn(famAndQf[0], famAndQf[1]); } + get.setTimeStamp(timestamp); + return get(tableName, get).getRowResult(); } /** @@ -100,25 +132,20 @@ public class TimestampModel extends AbstractModel { */ public RowResult get(byte[] tableName, byte[] rowName, long timestamp) throws HBaseRestException { - try { - HTable table = new HTable(tableName); - return table.getRow(rowName, timestamp); - } catch (IOException e) { - throw new HBaseRestException(e); - } + Get get = new Get(rowName); + get.setTimeStamp(timestamp); + return get(tableName, get).getRowResult(); } public void post(byte[] tableName, byte[] rowName, byte[] columnName, long timestamp, byte[] value) throws HBaseRestException { try { - HTable table; - BatchUpdate b; - - table = new HTable(tableName); - b = new BatchUpdate(rowName, timestamp); - - b.put(columnName, value); - table.commit(b); + HTable table = new HTable(tableName); + Put put = new Put(rowName); + put.setTimeStamp(timestamp); + byte [][] famAndQf = KeyValue.parseColumn(columnName); + put.add(famAndQf[0], famAndQf[1], value); + table.put(put); } catch (IOException e) { throw new HBaseRestException(e); } diff --git a/src/java/org/apache/hadoop/hbase/rest/parser/XMLRestParser.java b/src/java/org/apache/hadoop/hbase/rest/parser/XMLRestParser.java index abab6432374..7d6cf5dd514 100644 --- a/src/java/org/apache/hadoop/hbase/rest/parser/XMLRestParser.java +++ b/src/java/org/apache/hadoop/hbase/rest/parser/XMLRestParser.java @@ -96,7 +96,6 @@ public class XMLRestParser implements IHBaseRestParser { String compression = HColumnDescriptor.DEFAULT_COMPRESSION; boolean in_memory = HColumnDescriptor.DEFAULT_IN_MEMORY; boolean block_cache = HColumnDescriptor.DEFAULT_BLOCKCACHE; - int max_cell_size = HColumnDescriptor.DEFAULT_LENGTH; int ttl = HColumnDescriptor.DEFAULT_TTL; boolean bloomfilter = HColumnDescriptor.DEFAULT_BLOOMFILTER; @@ -108,7 +107,6 @@ public class XMLRestParser implements IHBaseRestParser { // compression = currentCDesp.getCompression(); in_memory = currentCDesp.isInMemory(); block_cache = currentCDesp.isBlockCacheEnabled(); - max_cell_size = currentCDesp.getMaxValueLength(); ttl = currentCDesp.getTimeToLive(); bloomfilter = currentCDesp.isBloomfilter(); } @@ -141,13 +139,6 @@ public class XMLRestParser implements IHBaseRestParser { .getNodeValue()); } - NodeList max_cell_size_list = columnfamily - .getElementsByTagName("max-cell-size"); - if (max_cell_size_list.getLength() > 0) { - max_cell_size = Integer.valueOf(max_cell_size_list.item(0) - .getFirstChild().getNodeValue()); - } - NodeList ttl_list = columnfamily.getElementsByTagName("time-to-live"); if (ttl_list.getLength() > 0) { ttl = Integer.valueOf(ttl_list.item(0).getFirstChild().getNodeValue()); @@ -162,7 +153,7 @@ public class XMLRestParser implements IHBaseRestParser { HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(colname), max_versions, compression, in_memory, block_cache, - max_cell_size, ttl, bloomfilter); + ttl, bloomfilter); NodeList metadataList = columnfamily.getElementsByTagName("metadata"); for (int i = 0; i < metadataList.getLength(); i++) { diff --git a/src/java/org/apache/hadoop/hbase/rest/serializer/SimpleXMLSerializer.java b/src/java/org/apache/hadoop/hbase/rest/serializer/SimpleXMLSerializer.java index 12b30a84185..2c6cd865502 100644 --- a/src/java/org/apache/hadoop/hbase/rest/serializer/SimpleXMLSerializer.java +++ b/src/java/org/apache/hadoop/hbase/rest/serializer/SimpleXMLSerializer.java @@ -163,10 +163,6 @@ public class SimpleXMLSerializer extends AbstractRestSerializer { printer.print(""); printer.print(column.getMaxVersions()); printer.print(""); - // max-length - printer.print(""); - printer.print(column.getMaxValueLength()); - printer.print(""); printer.print(""); printer.flush(); } diff --git a/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java b/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java index 2074e5357ad..9514deee4de 100644 --- a/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java +++ b/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java @@ -33,10 +33,16 @@ 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.MasterNotRunningException; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; 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.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; @@ -76,7 +82,7 @@ public class ThriftServer { // nextScannerId and scannerMap are used to manage scanner state protected int nextScannerId = 0; - protected HashMap scannerMap = null; + protected HashMap scannerMap = null; /** * Returns a list of all the column families for a given htable. @@ -115,7 +121,7 @@ public class ThriftServer { * @param scanner * @return integer scanner id */ - protected synchronized int addScanner(Scanner scanner) { + protected synchronized int addScanner(ResultScanner scanner) { int id = nextScannerId++; scannerMap.put(id, scanner); return id; @@ -127,7 +133,7 @@ public class ThriftServer { * @param id * @return a Scanner, or null if ID was invalid. */ - protected synchronized Scanner getScanner(int id) { + protected synchronized ResultScanner getScanner(int id) { return scannerMap.get(id); } @@ -138,7 +144,7 @@ public class ThriftServer { * @param id * @return a Scanner, or null if ID was invalid. */ - protected synchronized Scanner removeScanner(int id) { + protected synchronized ResultScanner removeScanner(int id) { return scannerMap.remove(id); } @@ -150,7 +156,7 @@ public class ThriftServer { HBaseHandler() throws MasterNotRunningException { conf = new HBaseConfiguration(); admin = new HBaseAdmin(conf); - scannerMap = new HashMap(); + scannerMap = new HashMap(); } public void enableTable(final byte[] tableName) throws IOError { @@ -228,35 +234,78 @@ public class ThriftServer { } } + @Deprecated public List get(byte[] tableName, byte[] row, byte[] column) throws IOError { + byte [][] famAndQf = KeyValue.parseColumn(column); + return get(tableName, row, famAndQf[0], famAndQf[1]); + } + + public List get(byte [] tableName, byte [] row, byte [] family, + byte [] qualifier) throws IOError { try { HTable table = getTable(tableName); - Cell cell = table.get(row, column); + Get get = new Get(row); + if (qualifier == null || qualifier.length == 0) { + get.addFamily(family); + } else { + get.addColumn(family, qualifier); + } + Result result = table.get(get); + Cell cell = result.getCellValue(family, qualifier); return ThriftUtilities.cellFromHBase(cell); } catch (IOException e) { throw new IOError(e.getMessage()); } } + @Deprecated public List getVer(byte[] tableName, byte[] row, byte[] column, int numVersions) throws IOError { + byte [][] famAndQf = KeyValue.parseColumn(column); + return getVer(tableName, row, famAndQf[0], famAndQf[1], numVersions); + } + + public List getVer(byte [] tableName, byte [] row, byte [] family, + byte [] qualifier, int numVersions) throws IOError { try { HTable table = getTable(tableName); - Cell[] cells = - table.get(row, column, numVersions); - return ThriftUtilities.cellFromHBase(cells); + Get get = new Get(row); + get.addColumn(family, qualifier); + get.setMaxVersions(numVersions); + Result result = table.get(get); + List cells = new ArrayList(); + for(KeyValue kv : result.sorted()) { + cells.add(new Cell(kv.getValue(), kv.getTimestamp())); + } + return ThriftUtilities.cellFromHBase(cells.toArray(new Cell[0])); } catch (IOException e) { throw new IOError(e.getMessage()); } } + @Deprecated public List getVerTs(byte[] tableName, byte[] row, byte[] column, long timestamp, int numVersions) throws IOError { + byte [][] famAndQf = KeyValue.parseColumn(column); + return getVerTs(tableName, row, famAndQf[0], famAndQf[1], timestamp, + numVersions); + } + + public List getVerTs(byte [] tableName, byte [] row, byte [] family, + byte [] qualifier, long timestamp, int numVersions) throws IOError { try { HTable table = getTable(tableName); - Cell[] cells = table.get(row, column, timestamp, numVersions); - return ThriftUtilities.cellFromHBase(cells); + Get get = new Get(row); + get.addColumn(family, qualifier); + get.setTimeStamp(timestamp); + get.setMaxVersions(numVersions); + Result result = table.get(get); + List cells = new ArrayList(); + for(KeyValue kv : result.sorted()) { + cells.add(new Cell(kv.getValue(), kv.getTimestamp())); + } + return ThriftUtilities.cellFromHBase(cells.toArray(new Cell[0])); } catch (IOException e) { throw new IOError(e.getMessage()); } @@ -285,12 +334,20 @@ public class ThriftServer { try { HTable table = getTable(tableName); if (columns == null) { - return ThriftUtilities.rowResultFromHBase(table.getRow(row, - timestamp)); + Get get = new Get(row); + get.setTimeStamp(timestamp); + Result result = table.get(get); + return ThriftUtilities.rowResultFromHBase(result.getRowResult()); } byte[][] columnArr = columns.toArray(new byte[columns.size()][]); - return ThriftUtilities.rowResultFromHBase(table.getRow(row, - columnArr, timestamp)); + Get get = new Get(row); + for(byte [] column : columnArr) { + byte [][] famAndQf = KeyValue.parseColumn(column); + get.addColumn(famAndQf[0], famAndQf[1]); + } + get.setTimeStamp(timestamp); + Result result = table.get(get); + return ThriftUtilities.rowResultFromHBase(result.getRowResult()); } catch (IOException e) { throw new IOError(e.getMessage()); } @@ -305,7 +362,15 @@ public class ThriftServer { long timestamp) throws IOError { try { HTable table = getTable(tableName); - table.deleteAll(row, column, timestamp); + Delete delete = new Delete(row, timestamp, null); + byte [][] famAndQf = KeyValue.parseColumn(column); + if(famAndQf[1].length == 0){ + delete.deleteFamily(famAndQf[0]); + } else { + delete.deleteColumns(famAndQf[0], famAndQf[1]); + } + table.delete(delete); + } catch (IOException e) { throw new IOError(e.getMessage()); } @@ -319,7 +384,8 @@ public class ThriftServer { throws IOError { try { HTable table = getTable(tableName); - table.deleteAll(row, timestamp); + Delete delete = new Delete(row, timestamp, null); + table.delete(delete); } catch (IOException e) { throw new IOError(e.getMessage()); } @@ -369,15 +435,13 @@ public class ThriftServer { HTable table = null; try { table = getTable(tableName); - BatchUpdate batchUpdate = new BatchUpdate(row, timestamp); + Put put = new Put(row); + put.setTimeStamp(timestamp); for (Mutation m : mutations) { - if (m.isDelete) { - batchUpdate.delete(m.column); - } else { - batchUpdate.put(m.column, m.value); - } + byte [][] famAndQf = KeyValue.parseColumn(m.column); + put.add(famAndQf[0], famAndQf[1], m.value); } - table.commit(batchUpdate); + table.put(put); } catch (IOException e) { throw new IOError(e.getMessage()); } catch (IllegalArgumentException e) { @@ -392,26 +456,24 @@ public class ThriftServer { public void mutateRowsTs(byte[] tableName, List rowBatches, long timestamp) throws IOError, IllegalArgument, TException { - List batchUpdates = new ArrayList(); + List puts = new ArrayList(); for (BatchMutation batch : rowBatches) { byte[] row = batch.row; List mutations = batch.mutations; - BatchUpdate batchUpdate = new BatchUpdate(row, timestamp); + Put put = new Put(row); + put.setTimeStamp(timestamp); for (Mutation m : mutations) { - if (m.isDelete) { - batchUpdate.delete(m.column); - } else { - batchUpdate.put(m.column, m.value); - } + byte [][] famAndQf = KeyValue.parseColumn(m.column); + put.add(famAndQf[0], famAndQf[1], m.value); } - batchUpdates.add(batchUpdate); + puts.add(put); } HTable table = null; try { table = getTable(tableName); - table.commit(batchUpdates); + table.put(puts); } catch (IOException e) { throw new IOError(e.getMessage()); } catch (IllegalArgumentException e) { @@ -419,19 +481,28 @@ public class ThriftServer { } } - public long atomicIncrement(byte[] tableName, byte[] row, byte[] column, long amount) throws IOError, IllegalArgument, TException { + @Deprecated + public long atomicIncrement(byte[] tableName, byte[] row, byte[] column, + long amount) throws IOError, IllegalArgument, TException { + byte [][] famAndQf = KeyValue.parseColumn(column); + return atomicIncrement(tableName, row, famAndQf[0], famAndQf[1], amount); + } + + public long atomicIncrement(byte [] tableName, byte [] row, byte [] family, + byte [] qualifier, long amount) + throws IOError, IllegalArgument, TException { HTable table; try { table = getTable(tableName); - return table.incrementColumnValue(row, column, amount); + return table.incrementColumnValue(row, family, qualifier, amount); } catch (IOException e) { throw new IOError(e.getMessage()); } } - + public void scannerClose(int id) throws IOError, IllegalArgument { LOG.debug("scannerClose: id=" + id); - Scanner scanner = getScanner(id); + ResultScanner scanner = getScanner(id); if (scanner == null) { throw new IllegalArgument("scanner ID is invalid"); } @@ -441,12 +512,12 @@ public class ThriftServer { public List scannerGetList(int id,int nbRows) throws IllegalArgument, IOError { LOG.debug("scannerGetList: id=" + id); - Scanner scanner = getScanner(id); + ResultScanner scanner = getScanner(id); if (null == scanner) { throw new IllegalArgument("scanner ID is invalid"); } - RowResult [] results = null; + Result [] results = null; try { results = scanner.next(nbRows); if (null == results) { @@ -470,7 +541,9 @@ public class ThriftServer { } else { columnsArray = columns.toArray(new byte[0][]); } - return addScanner(table.getScanner(columnsArray, startRow)); + Scan scan = new Scan(startRow); + scan.addColumns(columnsArray); + return addScanner(table.getScanner(scan)); } catch (IOException e) { throw new IOError(e.getMessage()); } @@ -486,7 +559,9 @@ public class ThriftServer { } else { columnsArray = columns.toArray(new byte[0][]); } - return addScanner(table.getScanner(columnsArray, startRow, stopRow)); + Scan scan = new Scan(startRow, stopRow); + scan.addColumns(columnsArray); + return addScanner(table.getScanner(scan)); } catch (IOException e) { throw new IOError(e.getMessage()); } @@ -502,7 +577,10 @@ public class ThriftServer { } else { columnsArray = columns.toArray(new byte[0][]); } - return addScanner(table.getScanner(columnsArray, startRow, timestamp)); + Scan scan = new Scan(startRow); + scan.addColumns(columnsArray); + scan.setTimeRange(0, timestamp); + return addScanner(table.getScanner(scan)); } catch (IOException e) { throw new IOError(e.getMessage()); } @@ -519,8 +597,10 @@ public class ThriftServer { } else { columnsArray = columns.toArray(new byte[0][]); } - return addScanner(table.getScanner(columnsArray, startRow, stopRow, - timestamp)); + Scan scan = new Scan(startRow, stopRow); + scan.addColumns(columnsArray); + scan.setTimeRange(0, timestamp); + return addScanner(table.getScanner(scan)); } catch (IOException e) { throw new IOError(e.getMessage()); } diff --git a/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java b/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java index f06e867fae8..13cbd3af9c9 100644 --- a/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java +++ b/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.TreeMap; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.hfile.Compression; @@ -58,7 +59,7 @@ public class ThriftUtilities { } HColumnDescriptor col = new HColumnDescriptor(in.name, in.maxVersions, comp.getName(), in.inMemory, in.blockCacheEnabled, - in.maxValueLength, in.timeToLive, bloom); + in.timeToLive, bloom); return col; } @@ -77,7 +78,6 @@ public class ThriftUtilities { col.compression = in.getCompression().toString(); col.inMemory = in.isInMemory(); col.blockCacheEnabled = in.isBlockCacheEnabled(); - col.maxValueLength = in.getMaxValueLength(); col.bloomFilterType = Boolean.toString(in.isBloomfilter()); return col; } @@ -150,5 +150,38 @@ public class ThriftUtilities { return rowResultFromHBase(result); } + /** + * This utility method creates a list of Thrift TRowResult "struct" based on + * an Hbase RowResult object. The empty list is returned if the input is + * null. + * + * @param in + * Hbase RowResult object + * @return Thrift TRowResult array + */ + static public List rowResultFromHBase(Result[] in) { + List results = new ArrayList(); + for ( Result result_ : in) { + if(null == result_) { + continue; + } + RowResult rowResult_ = result_.getRowResult(); + TRowResult result = new TRowResult(); + result.row = rowResult_.getRow(); + result.columns = new TreeMap(Bytes.BYTES_COMPARATOR); + for (Map.Entry entry : rowResult_.entrySet()){ + Cell cell = entry.getValue(); + result.columns.put(entry.getKey(), + new TCell(cell.getValue(), cell.getTimestamp())); + + } + results.add(result); + } + return results; + } + static public List rowResultFromHBase(Result in) { + Result [] result = { in }; + return rowResultFromHBase(result); + } } diff --git a/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java b/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java index fc0ba7b8ff9..0c97bf4199a 100644 --- a/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java @@ -45,12 +45,11 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable private static final TField MAX_VERSIONS_FIELD_DESC = new TField("maxVersions", TType.I32, (short)2); private static final TField COMPRESSION_FIELD_DESC = new TField("compression", TType.STRING, (short)3); private static final TField IN_MEMORY_FIELD_DESC = new TField("inMemory", TType.BOOL, (short)4); - private static final TField MAX_VALUE_LENGTH_FIELD_DESC = new TField("maxValueLength", TType.I32, (short)5); - private static final TField BLOOM_FILTER_TYPE_FIELD_DESC = new TField("bloomFilterType", TType.STRING, (short)6); - private static final TField BLOOM_FILTER_VECTOR_SIZE_FIELD_DESC = new TField("bloomFilterVectorSize", TType.I32, (short)7); - private static final TField BLOOM_FILTER_NB_HASHES_FIELD_DESC = new TField("bloomFilterNbHashes", TType.I32, (short)8); - private static final TField BLOCK_CACHE_ENABLED_FIELD_DESC = new TField("blockCacheEnabled", TType.BOOL, (short)9); - private static final TField TIME_TO_LIVE_FIELD_DESC = new TField("timeToLive", TType.I32, (short)10); + private static final TField BLOOM_FILTER_TYPE_FIELD_DESC = new TField("bloomFilterType", TType.STRING, (short)5); + private static final TField BLOOM_FILTER_VECTOR_SIZE_FIELD_DESC = new TField("bloomFilterVectorSize", TType.I32, (short)6); + private static final TField BLOOM_FILTER_NB_HASHES_FIELD_DESC = new TField("bloomFilterNbHashes", TType.I32, (short)7); + private static final TField BLOCK_CACHE_ENABLED_FIELD_DESC = new TField("blockCacheEnabled", TType.BOOL, (short)8); + private static final TField TIME_TO_LIVE_FIELD_DESC = new TField("timeToLive", TType.I32, (short)9); public byte[] name; public static final int NAME = 1; @@ -60,24 +59,21 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable public static final int COMPRESSION = 3; public boolean inMemory; public static final int INMEMORY = 4; - public int maxValueLength; - public static final int MAXVALUELENGTH = 5; public String bloomFilterType; - public static final int BLOOMFILTERTYPE = 6; + public static final int BLOOMFILTERTYPE = 5; public int bloomFilterVectorSize; - public static final int BLOOMFILTERVECTORSIZE = 7; + public static final int BLOOMFILTERVECTORSIZE = 6; public int bloomFilterNbHashes; - public static final int BLOOMFILTERNBHASHES = 8; + public static final int BLOOMFILTERNBHASHES = 7; public boolean blockCacheEnabled; - public static final int BLOCKCACHEENABLED = 9; + public static final int BLOCKCACHEENABLED = 8; public int timeToLive; - public static final int TIMETOLIVE = 10; + public static final int TIMETOLIVE = 9; private final Isset __isset = new Isset(); private static final class Isset implements java.io.Serializable { public boolean maxVersions = false; public boolean inMemory = false; - public boolean maxValueLength = false; public boolean bloomFilterVectorSize = false; public boolean bloomFilterNbHashes = false; public boolean blockCacheEnabled = false; @@ -93,8 +89,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable new FieldValueMetaData(TType.STRING))); put(INMEMORY, new FieldMetaData("inMemory", TFieldRequirementType.DEFAULT, new FieldValueMetaData(TType.BOOL))); - put(MAXVALUELENGTH, new FieldMetaData("maxValueLength", TFieldRequirementType.DEFAULT, - new FieldValueMetaData(TType.I32))); put(BLOOMFILTERTYPE, new FieldMetaData("bloomFilterType", TFieldRequirementType.DEFAULT, new FieldValueMetaData(TType.STRING))); put(BLOOMFILTERVECTORSIZE, new FieldMetaData("bloomFilterVectorSize", TFieldRequirementType.DEFAULT, @@ -118,8 +112,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable this.inMemory = false; - this.maxValueLength = 2147483647; - this.bloomFilterType = "NONE"; this.bloomFilterVectorSize = 0; @@ -151,8 +143,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable this.compression = compression; this.inMemory = inMemory; this.__isset.inMemory = true; - this.maxValueLength = maxValueLength; - this.__isset.maxValueLength = true; this.bloomFilterType = bloomFilterType; this.bloomFilterVectorSize = bloomFilterVectorSize; this.__isset.bloomFilterVectorSize = true; @@ -178,8 +168,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable } __isset.inMemory = other.__isset.inMemory; this.inMemory = other.inMemory; - __isset.maxValueLength = other.__isset.maxValueLength; - this.maxValueLength = other.maxValueLength; if (other.isSetBloomFilterType()) { this.bloomFilterType = other.bloomFilterType; } @@ -288,28 +276,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable this.__isset.inMemory = value; } - public int getMaxValueLength() { - return this.maxValueLength; - } - - public void setMaxValueLength(int maxValueLength) { - this.maxValueLength = maxValueLength; - this.__isset.maxValueLength = true; - } - - public void unsetMaxValueLength() { - this.__isset.maxValueLength = false; - } - - // Returns true if field maxValueLength is set (has been asigned a value) and false otherwise - public boolean isSetMaxValueLength() { - return this.__isset.maxValueLength; - } - - public void setMaxValueLengthIsSet(boolean value) { - this.__isset.maxValueLength = value; - } - public String getBloomFilterType() { return this.bloomFilterType; } @@ -455,14 +421,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable } break; - case MAXVALUELENGTH: - if (value == null) { - unsetMaxValueLength(); - } else { - setMaxValueLength((Integer)value); - } - break; - case BLOOMFILTERTYPE: if (value == null) { unsetBloomFilterType(); @@ -522,9 +480,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable case INMEMORY: return new Boolean(isInMemory()); - case MAXVALUELENGTH: - return new Integer(getMaxValueLength()); - case BLOOMFILTERTYPE: return getBloomFilterType(); @@ -556,8 +511,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable return isSetCompression(); case INMEMORY: return isSetInMemory(); - case MAXVALUELENGTH: - return isSetMaxValueLength(); case BLOOMFILTERTYPE: return isSetBloomFilterType(); case BLOOMFILTERVECTORSIZE: @@ -622,15 +575,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable return false; } - boolean this_present_maxValueLength = true; - boolean that_present_maxValueLength = true; - if (this_present_maxValueLength || that_present_maxValueLength) { - if (!(this_present_maxValueLength && that_present_maxValueLength)) - return false; - if (this.maxValueLength != that.maxValueLength) - return false; - } - boolean this_present_bloomFilterType = true && this.isSetBloomFilterType(); boolean that_present_bloomFilterType = true && that.isSetBloomFilterType(); if (this_present_bloomFilterType || that_present_bloomFilterType) { @@ -725,14 +669,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable TProtocolUtil.skip(iprot, field.type); } break; - case MAXVALUELENGTH: - if (field.type == TType.I32) { - this.maxValueLength = iprot.readI32(); - this.__isset.maxValueLength = true; - } else { - TProtocolUtil.skip(iprot, field.type); - } - break; case BLOOMFILTERTYPE: if (field.type == TType.STRING) { this.bloomFilterType = iprot.readString(); @@ -805,9 +741,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable oprot.writeFieldBegin(IN_MEMORY_FIELD_DESC); oprot.writeBool(this.inMemory); oprot.writeFieldEnd(); - oprot.writeFieldBegin(MAX_VALUE_LENGTH_FIELD_DESC); - oprot.writeI32(this.maxValueLength); - oprot.writeFieldEnd(); if (this.bloomFilterType != null) { oprot.writeFieldBegin(BLOOM_FILTER_TYPE_FIELD_DESC); oprot.writeString(this.bloomFilterType); @@ -858,10 +791,6 @@ public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable sb.append(this.inMemory); first = false; if (!first) sb.append(", "); - sb.append("maxValueLength:"); - sb.append(this.maxValueLength); - first = false; - if (!first) sb.append(", "); sb.append("bloomFilterType:"); if (this.bloomFilterType == null) { sb.append("null"); diff --git a/src/java/org/apache/hadoop/hbase/util/Bytes.java b/src/java/org/apache/hadoop/hbase/util/Bytes.java index e8d5f91ddd4..4c9d9bbe919 100644 --- a/src/java/org/apache/hadoop/hbase/util/Bytes.java +++ b/src/java/org/apache/hadoop/hbase/util/Bytes.java @@ -39,6 +39,7 @@ import org.apache.hadoop.io.WritableUtils; * HashSets, etc. */ public class Bytes { + /** * Size of long in bytes */ @@ -81,6 +82,9 @@ public class Bytes { * Byte array comparator class. */ public static class ByteArrayComparator implements RawComparator { + /** + * Constructor + */ public ByteArrayComparator() { super(); } @@ -143,13 +147,19 @@ public class Bytes { */ public static void writeByteArray(final DataOutput out, final byte [] b) throws IOException { - writeByteArray(out, b, 0, b.length); + if(b == null) { + WritableUtils.writeVInt(out, 0); + } else { + writeByteArray(out, b, 0, b.length); + } } /** * Write byte-array to out with a vint length prefix. * @param out * @param b + * @param offset + * @param length * @throws IOException */ public static void writeByteArray(final DataOutput out, final byte [] b, @@ -182,6 +192,8 @@ public class Bytes { * @param tgtBytes the byte array * @param tgtOffset position in the array * @param srcBytes byte to write out + * @param srcOffset + * @param srcLength * @return incremented offset */ public static int putBytes(byte[] tgtBytes, int tgtOffset, byte[] srcBytes, @@ -219,9 +231,18 @@ public class Bytes { * @return String made from b */ public static String toString(final byte [] b) { + if(b == null) { + return null; + } return toString(b, 0, b.length); } + public static String toString(final byte [] b1, + String sep, + final byte [] b2) { + return toString(b1, 0, b1.length) + sep + toString(b2, 0, b2.length); + } + /** * @param b Presumed UTF-8 encoded byte array. * @param off @@ -229,6 +250,12 @@ public class Bytes { * @return String made from b */ public static String toString(final byte [] b, int off, int len) { + if(b == null) { + return null; + } + if(len == 0) { + return ""; + } String result = null; try { result = new String(b, off, len, HConstants.UTF8_ENCODING); @@ -382,6 +409,10 @@ public class Bytes { return putInt(bytes, offset, i); } + /** + * @param f + * @return the float represented as byte [] + */ public static byte [] toBytes(final float f) { // Encode it as int int i = Float.floatToRawIntBits(f); @@ -417,6 +448,10 @@ public class Bytes { return putLong(bytes, offset, l); } + /** + * @param d + * @return the double represented as byte [] + */ public static byte [] toBytes(final double d) { // Encode it as a long long l = Double.doubleToRawLongBits(d); @@ -521,6 +556,7 @@ public class Bytes { /** * Converts a byte array to a short value * @param bytes + * @param offset * @return the short value */ public static short toShort(byte[] bytes, int offset) { @@ -530,6 +566,8 @@ public class Bytes { /** * Converts a byte array to a short value * @param bytes + * @param offset + * @param lengths * @return the short value */ public static short toShort(byte[] bytes, int offset, final int length) { @@ -899,12 +937,17 @@ public class Bytes { while (low <= high) { int mid = (low+high) >>> 1; - int cmp = comparator.compare(arr[mid], 0, arr[mid].length, key, offset, - length); - if (cmp < 0) + // we have to compare in this order, because the comparator order + // has special logic when the 'left side' is a special key. + int cmp = comparator.compare(key, offset, length, + arr[mid], 0, arr[mid].length); + // key lives above the midpoint + if (cmp > 0) low = mid + 1; - else if (cmp > 0) + // key lives below the midpoint + else if (cmp < 0) high = mid - 1; + // BAM. how often does this really happen? else return mid; } diff --git a/src/java/org/apache/hadoop/hbase/util/Merge.java b/src/java/org/apache/hadoop/hbase/util/Merge.java index 7abb3535de0..7edf98f7401 100644 --- a/src/java/org/apache/hadoop/hbase/util/Merge.java +++ b/src/java/org/apache/hadoop/hbase/util/Merge.java @@ -33,6 +33,8 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MasterNotRunningException; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.regionserver.HLog; import org.apache.hadoop.hbase.regionserver.HRegion; @@ -69,7 +71,7 @@ public class Merge extends Configured implements Tool { this.conf = conf; this.mergeInfo = null; } - + public int run(String[] args) throws Exception { if (parseArgs(args) != 0) { return -1; @@ -140,11 +142,14 @@ public class Merge extends Configured implements Tool { */ private void mergeTwoMetaRegions() throws IOException { HRegion rootRegion = utils.getRootRegion(); - List cells1 = - rootRegion.get(region1, HConstants.COL_REGIONINFO, -1, -1); + Get get = new Get(region1); + get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + List cells1 = rootRegion.get(get, null).list(); HRegionInfo info1 = Writables.getHRegionInfo((cells1 == null)? null: cells1.get(0).getValue()); - List cells2 = - rootRegion.get(region2, HConstants.COL_REGIONINFO, -1, -1); + + get = new Get(region2); + get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + List cells2 = rootRegion.get(get, null).list(); HRegionInfo info2 = Writables.getHRegionInfo((cells2 == null)? null: cells2.get(0).getValue()); HRegion merged = merge(info1, rootRegion, info2, rootRegion); LOG.info("Adding " + merged.getRegionInfo() + " to " + @@ -206,7 +211,9 @@ public class Merge extends Configured implements Tool { LOG.info("Found meta for region1 " + Bytes.toString(meta1.getRegionName()) + ", meta for region2 " + Bytes.toString(meta2.getRegionName())); HRegion metaRegion1 = this.utils.getMetaRegion(meta1); - List cells1 = metaRegion1.get(region1, HConstants.COL_REGIONINFO, -1, -1); + Get get = new Get(region1); + get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + List cells1 = metaRegion1.get(get, null).list(); HRegionInfo info1 = Writables.getHRegionInfo((cells1 == null)? null: cells1.get(0).getValue()); if (info1== null) { throw new NullPointerException("info1 is null using key " + @@ -219,7 +226,9 @@ public class Merge extends Configured implements Tool { } else { metaRegion2 = utils.getMetaRegion(meta2); } - List cells2 = metaRegion2.get(region2, HConstants.COL_REGIONINFO, -1, -1); + get = new Get(region2); + get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + List cells2 = metaRegion2.get(get, null).list(); HRegionInfo info2 = Writables.getHRegionInfo((cells2 == null)? null: cells2.get(0).getValue()); if (info2 == null) { throw new NullPointerException("info2 is null using key " + meta2); @@ -309,7 +318,10 @@ public class Merge extends Configured implements Tool { if (LOG.isDebugEnabled()) { LOG.debug("Removing region: " + regioninfo + " from " + meta); } - meta.deleteAll(regioninfo.getRegionName(), System.currentTimeMillis(), null); + + Delete delete = new Delete(regioninfo.getRegionName(), + System.currentTimeMillis(), null); + meta.delete(delete, null, true); } /* diff --git a/src/java/org/apache/hadoop/hbase/util/MetaUtils.java b/src/java/org/apache/hadoop/hbase/util/MetaUtils.java index 2fba461ba25..29b1bda7f31 100644 --- a/src/java/org/apache/hadoop/hbase/util/MetaUtils.java +++ b/src/java/org/apache/hadoop/hbase/util/MetaUtils.java @@ -36,7 +36,12 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.regionserver.HLog; @@ -188,28 +193,31 @@ public class MetaUtils { openRootRegion(); } - InternalScanner rootScanner = rootRegion.getScanner( - HConstants.COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW, - HConstants.LATEST_TIMESTAMP, null); + Scan scan = new Scan(); + scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + InternalScanner rootScanner = + rootRegion.getScanner(scan); try { List results = new ArrayList(); - while (rootScanner.next(results)) { + boolean hasNext = true; + do { + hasNext = rootScanner.next(results); HRegionInfo info = null; for (KeyValue kv: results) { info = Writables.getHRegionInfoOrNull(kv.getValue()); if (info == null) { LOG.warn("region info is null for row " + - Bytes.toString(kv.getRow()) + " in table " + + Bytes.toString(kv.getRow()) + " in table " + HConstants.ROOT_TABLE_NAME); - } - continue; } - if (!listener.processRow(info)) { - break; - } - results.clear(); - } + continue; + } + if (!listener.processRow(info)) { + break; + } + results.clear(); + } while (hasNext); } finally { rootScanner.close(); } @@ -243,16 +251,19 @@ public class MetaUtils { */ public void scanMetaRegion(final HRegion m, final ScannerListener listener) throws IOException { - InternalScanner metaScanner = m.getScanner(HConstants.COL_REGIONINFO_ARRAY, - HConstants.EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null); + + Scan scan = new Scan(); + scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + InternalScanner metaScanner = + m.getScanner(scan); + try { List results = new ArrayList(); while (metaScanner.next(results)) { HRegionInfo info = null; for (KeyValue kv: results) { - if (KeyValue.META_COMPARATOR.compareColumns(kv, - HConstants.COL_REGIONINFO, 0, HConstants.COL_REGIONINFO.length, - HConstants.COLUMN_FAMILY_STR.length()) == 0) { + if(kv.matchingColumn(HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER)) { info = Writables.getHRegionInfoOrNull(kv.getValue()); if (info == null) { LOG.warn("region info is null for row " + @@ -306,18 +317,30 @@ public class MetaUtils { final byte [] row, final boolean onlineOffline) throws IOException { HTable t = new HTable(c, HConstants.META_TABLE_NAME); - Cell cell = t.get(row, HConstants.COL_REGIONINFO); - if (cell == null) { + Get get = new Get(row); + get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + Result res = t.get(get); + KeyValue [] kvs = res.raw(); + if(kvs.length <= 0) { throw new IOException("no information for row " + Bytes.toString(row)); } - // Throws exception if null. - HRegionInfo info = Writables.getHRegionInfo(cell); - BatchUpdate b = new BatchUpdate(row); + byte [] value = kvs[0].getValue(); + if (value == null) { + throw new IOException("no information for row " + Bytes.toString(row)); + } + HRegionInfo info = Writables.getHRegionInfo(value); + Put put = new Put(row); info.setOffline(onlineOffline); - b.put(HConstants.COL_REGIONINFO, Writables.getBytes(info)); - b.delete(HConstants.COL_SERVER); - b.delete(HConstants.COL_STARTCODE); - t.commit(b); + put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, + Writables.getBytes(info)); + t.put(put); + + Delete delete = new Delete(row); + delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); + delete.deleteColumns(HConstants.CATALOG_FAMILY, + HConstants.STARTCODE_QUALIFIER); + + t.delete(delete); } /** @@ -402,21 +425,45 @@ public class MetaUtils { public void updateMETARegionInfo(HRegion r, final HRegionInfo hri) throws IOException { if (LOG.isDebugEnabled()) { - HRegionInfo h = Writables.getHRegionInfoOrNull( - r.get(hri.getRegionName(), HConstants.COL_REGIONINFO, -1, -1).get(0).getValue()); - LOG.debug("Old " + Bytes.toString(HConstants.COL_REGIONINFO) + - " for " + hri.toString() + " in " + r.toString() + " is: " + - h.toString()); + Get get = new Get(hri.getRegionName()); + get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + Result res = r.get(get, null); + KeyValue [] kvs = res.raw(); + if(kvs.length <= 0) { + return; + } + byte [] value = kvs[0].getValue(); + if (value == null) { + return; + } + HRegionInfo h = Writables.getHRegionInfoOrNull(value); + + LOG.debug("Old " + Bytes.toString(HConstants.CATALOG_FAMILY) + ":" + + Bytes.toString(HConstants.REGIONINFO_QUALIFIER) + " for " + + hri.toString() + " in " + r.toString() + " is: " + h.toString()); } - BatchUpdate b = new BatchUpdate(hri.getRegionName()); - b.put(HConstants.COL_REGIONINFO, Writables.getBytes(hri)); - r.batchUpdate(b, null); + + Put put = new Put(hri.getRegionName()); + put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, + Writables.getBytes(hri)); + r.put(put); + if (LOG.isDebugEnabled()) { - HRegionInfo h = Writables.getHRegionInfoOrNull( - r.get(hri.getRegionName(), HConstants.COL_REGIONINFO, -1, -1).get(0).getValue()); - LOG.debug("New " + Bytes.toString(HConstants.COL_REGIONINFO) + - " for " + hri.toString() + " in " + r.toString() + " is: " + - h.toString()); + Get get = new Get(hri.getRegionName()); + get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + Result res = r.get(get, null); + KeyValue [] kvs = res.raw(); + if(kvs.length <= 0) { + return; + } + byte [] value = kvs[0].getValue(); + if (value == null) { + return; + } + HRegionInfo h = Writables.getHRegionInfoOrNull(value); + LOG.debug("New " + Bytes.toString(HConstants.CATALOG_FAMILY) + ":" + + Bytes.toString(HConstants.REGIONINFO_QUALIFIER) + " for " + + hri.toString() + " in " + r.toString() + " is: " + h.toString()); } } diff --git a/src/java/org/apache/hadoop/hbase/util/Migrate.java b/src/java/org/apache/hadoop/hbase/util/Migrate.java index e0c7417b054..26697475c8b 100644 --- a/src/java/org/apache/hadoop/hbase/util/Migrate.java +++ b/src/java/org/apache/hadoop/hbase/util/Migrate.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.util.GenericOptionsParser; @@ -249,9 +250,10 @@ public class Migrate extends Configured implements Tool { if (!enableBlockCache(oldHri)) { return; } - BatchUpdate b = new BatchUpdate(oldHri.getRegionName()); - b.put(HConstants.COL_REGIONINFO, Writables.getBytes(oldHri)); - mr.batchUpdate(b); + Put put = new Put(oldHri.getRegionName()); + put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, + Writables.getBytes(oldHri)); + mr.put(put); LOG.info("Enabled blockcache on " + oldHri.getRegionNameAsString()); } @@ -262,7 +264,7 @@ public class Migrate extends Configured implements Tool { private boolean enableBlockCache(final HRegionInfo hri) { boolean result = false; HColumnDescriptor hcd = - hri.getTableDesc().getFamily(HConstants.COLUMN_FAMILY); + hri.getTableDesc().getFamily(HConstants.CATALOG_FAMILY); if (hcd == null) { LOG.info("No info family in: " + hri.getRegionNameAsString()); return result; @@ -283,9 +285,10 @@ public class Migrate extends Configured implements Tool { if (!updateVersions(oldHri)) { return; } - BatchUpdate b = new BatchUpdate(oldHri.getRegionName()); - b.put(HConstants.COL_REGIONINFO, Writables.getBytes(oldHri)); - mr.batchUpdate(b); + Put put = new Put(oldHri.getRegionName()); + put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, + Writables.getBytes(oldHri)); + mr.put(put); LOG.info("Upped versions on " + oldHri.getRegionNameAsString()); } @@ -296,7 +299,7 @@ public class Migrate extends Configured implements Tool { private boolean updateVersions(final HRegionInfo hri) { boolean result = false; HColumnDescriptor hcd = - hri.getTableDesc().getFamily(HConstants.COLUMN_FAMILY_HISTORIAN); + hri.getTableDesc().getFamily(HConstants.CATALOG_HISTORIAN_FAMILY); if (hcd == null) { LOG.info("No region historian family in: " + hri.getRegionNameAsString()); return result; @@ -307,7 +310,7 @@ public class Migrate extends Configured implements Tool { result = true; } // Set the versions up to 10 from old default of 1. - hcd = hri.getTableDesc().getFamily(HConstants.COLUMN_FAMILY); + hcd = hri.getTableDesc().getFamily(HConstants.CATALOG_FAMILY); if (hcd.getMaxVersions() == 1) { // Set it to 10, an arbitrary high number hcd.setMaxVersions(10); diff --git a/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java b/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java index 9f5ce23667c..85866687077 100644 --- a/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java +++ b/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.UnsupportedEncodingException; import java.util.Random; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.regionserver.HRegion; @@ -34,7 +35,7 @@ import org.apache.commons.logging.LogFactory; public abstract class AbstractMergeTestBase extends HBaseClusterTestCase { static final Log LOG = LogFactory.getLog(AbstractMergeTestBase.class.getName()); - static final byte [] COLUMN_NAME = Bytes.toBytes("contents:"); + static final byte [] COLUMN_NAME = Bytes.toBytes("contents"); protected final Random rand = new Random(); protected HTableDescriptor desc; protected ImmutableBytesWritable value; @@ -126,11 +127,10 @@ public abstract class AbstractMergeTestBase extends HBaseClusterTestCase { HRegionIncommon r = new HRegionIncommon(region); for(int i = firstRow; i < firstRow + nrows; i++) { - BatchUpdate batchUpdate = new BatchUpdate(Bytes.toBytes("row_" + Put put = new Put(Bytes.toBytes("row_" + String.format("%1$05d", i))); - - batchUpdate.put(COLUMN_NAME, value.get()); - region.batchUpdate(batchUpdate, null); + put.add(COLUMN_NAME, null, value.get()); + region.put(put); if(i % 10000 == 0) { System.out.println("Flushing write #" + i); r.flushcache(); diff --git a/src/test/org/apache/hadoop/hbase/DFSAbort.java b/src/test/org/apache/hadoop/hbase/DFSAbort.java index c2a9d87b1a0..19c2891775f 100644 --- a/src/test/org/apache/hadoop/hbase/DFSAbort.java +++ b/src/test/org/apache/hadoop/hbase/DFSAbort.java @@ -40,7 +40,7 @@ public class DFSAbort extends HBaseClusterTestCase { try { super.setUp(); HTableDescriptor desc = new HTableDescriptor(getName()); - desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY_STR)); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); HBaseAdmin admin = new HBaseAdmin(conf); admin.createTable(desc); } catch (Exception e) { diff --git a/src/test/org/apache/hadoop/hbase/HBaseTestCase.java b/src/test/org/apache/hadoop/hbase/HBaseTestCase.java index 3abe3403fe5..16b82815781 100644 --- a/src/test/org/apache/hadoop/hbase/HBaseTestCase.java +++ b/src/test/org/apache/hadoop/hbase/HBaseTestCase.java @@ -25,6 +25,7 @@ import java.io.UnsupportedEncodingException; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.NavigableMap; import java.util.SortedMap; import junit.framework.TestCase; @@ -33,8 +34,13 @@ 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.client.Delete; +import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Scanner; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; @@ -52,11 +58,11 @@ public abstract class HBaseTestCase extends TestCase { /** configuration parameter name for test directory */ public static final String TEST_DIRECTORY_KEY = "test.build.data"; - protected final static byte [] COLFAMILY_NAME1 = Bytes.toBytes("colfamily1:"); - protected final static byte [] COLFAMILY_NAME2 = Bytes.toBytes("colfamily2:"); - protected final static byte [] COLFAMILY_NAME3 = Bytes.toBytes("colfamily3:"); - protected static final byte [][] COLUMNS = {COLFAMILY_NAME1, - COLFAMILY_NAME2, COLFAMILY_NAME3}; + protected final static byte [] fam1 = Bytes.toBytes("colfamily1"); + protected final static byte [] fam2 = Bytes.toBytes("colfamily2"); + protected final static byte [] fam3 = Bytes.toBytes("colfamily3"); + protected static final byte [][] COLUMNS = {fam1, + fam2, fam3}; private boolean localfs = false; protected Path testDir = null; @@ -189,13 +195,13 @@ public abstract class HBaseTestCase extends TestCase { protected HTableDescriptor createTableDescriptor(final String name, final int versions) { HTableDescriptor htd = new HTableDescriptor(name); - htd.addFamily(new HColumnDescriptor(COLFAMILY_NAME1, versions, + htd.addFamily(new HColumnDescriptor(fam1, versions, HColumnDescriptor.DEFAULT_COMPRESSION, false, false, Integer.MAX_VALUE, HConstants.FOREVER, false)); - htd.addFamily(new HColumnDescriptor(COLFAMILY_NAME2, versions, + htd.addFamily(new HColumnDescriptor(fam2, versions, HColumnDescriptor.DEFAULT_COMPRESSION, false, false, Integer.MAX_VALUE, HConstants.FOREVER, false)); - htd.addFamily(new HColumnDescriptor(COLFAMILY_NAME3, versions, + htd.addFamily(new HColumnDescriptor(fam3, versions, HColumnDescriptor.DEFAULT_COMPRESSION, false, false, Integer.MAX_VALUE, HConstants.FOREVER, false)); return htd; @@ -284,11 +290,13 @@ public abstract class HBaseTestCase extends TestCase { break EXIT; } try { - BatchUpdate batchUpdate = ts == -1 ? - new BatchUpdate(t) : new BatchUpdate(t, ts); + Put put = new Put(t); + if(ts != -1) { + put.setTimeStamp(ts); + } try { - batchUpdate.put(column, t); - updater.commit(batchUpdate); + put.add(Bytes.toBytes(column), null, t); + updater.put(put); count++; } catch (RuntimeException ex) { ex.printStackTrace(); @@ -331,44 +339,23 @@ public abstract class HBaseTestCase extends TestCase { */ public static interface Incommon { /** - * @param row - * @param column - * @return value for row/column pair + * + * @param delete + * @param lockid + * @param writeToWAL * @throws IOException */ - public Cell get(byte [] row, byte [] column) throws IOException; - /** - * @param row - * @param column - * @param versions - * @return value for row/column pair for number of versions requested - * @throws IOException - */ - public Cell[] get(byte [] row, byte [] column, int versions) throws IOException; - /** - * @param row - * @param column - * @param ts - * @param versions - * @return value for row/column/timestamp tuple for number of versions - * @throws IOException - */ - public Cell[] get(byte [] row, byte [] column, long ts, int versions) + public void delete(Delete delete, Integer lockid, boolean writeToWAL) throws IOException; - /** - * @param row - * @param column - * @param ts - * @throws IOException - */ - public void deleteAll(byte [] row, byte [] column, long ts) throws IOException; /** - * @param batchUpdate + * @param put * @throws IOException */ - public void commit(BatchUpdate batchUpdate) throws IOException; + public void put(Put put) throws IOException; + public Result get(Get get) throws IOException; + /** * @param columns * @param firstRow @@ -393,48 +380,46 @@ public abstract class HBaseTestCase extends TestCase { this.region = HRegion; } - public void commit(BatchUpdate batchUpdate) throws IOException { - region.batchUpdate(batchUpdate, null); + public void put(Put put) throws IOException { + region.put(put); } - public void deleteAll(byte [] row, byte [] column, long ts) + public void delete(Delete delete, Integer lockid, boolean writeToWAL) throws IOException { - this.region.deleteAll(row, column, ts, null); + this.region.delete(delete, lockid, writeToWAL); } - + + public Result get(Get get) throws IOException { + return region.get(get, null); + } + public ScannerIncommon getScanner(byte [][] columns, byte [] firstRow, long ts) throws IOException { + Scan scan = new Scan(firstRow); + scan.addColumns(columns); + scan.setTimeRange(0, ts); return new - InternalScannerIncommon(region.getScanner(columns, firstRow, ts, null)); + InternalScannerIncommon(region.getScanner(scan)); } - - public Cell get(byte [] row, byte [] column) throws IOException { - // TODO: Fix profligacy converting from List to Cell []. - Cell[] result = Cell.createSingleCellArray(this.region.get(row, column, -1, -1)); - return (result == null)? null : result[0]; - } - - public Cell[] get(byte [] row, byte [] column, int versions) - throws IOException { - // TODO: Fix profligacy converting from List to Cell []. - return Cell.createSingleCellArray(this.region.get(row, column, -1, versions)); - } - - public Cell[] get(byte [] row, byte [] column, long ts, int versions) - throws IOException { - // TODO: Fix profligacy converting from List to Cell []. - return Cell.createSingleCellArray(this.region.get(row, column, ts, versions)); - } - - /** - * @param row - * @return values for each column in the specified row - * @throws IOException - */ - public Map getFull(byte [] row) throws IOException { - return region.getFull(row, null, HConstants.LATEST_TIMESTAMP, 1, null); + + //New + public ScannerIncommon getScanner(byte [] family, byte [][] qualifiers, + byte [] firstRow, long ts) + throws IOException { + Scan scan = new Scan(firstRow); + for(int i=0; i values) throws IOException { - RowResult results = scanner.next(); + Result results = scanner.next(); if (results == null) { return false; } values.clear(); - for (Map.Entry entry : results.entrySet()) { - values.add(new KeyValue(results.getRow(), entry.getKey(), - entry.getValue().getTimestamp(), entry.getValue().getValue())); - } + values.addAll(results.list()); return true; } @@ -544,25 +520,53 @@ public abstract class HBaseTestCase extends TestCase { } } - protected void assertCellEquals(final HRegion region, final byte [] row, - final byte [] column, final long timestamp, final String value) - throws IOException { - Map result = region.getFull(row, null, timestamp, 1, null); - Cell cell_value = result.get(column); - if (value == null) { - assertEquals(Bytes.toString(column) + " at timestamp " + timestamp, null, - cell_value); - } else { - if (cell_value == null) { - fail(Bytes.toString(column) + " at timestamp " + timestamp + - "\" was expected to be \"" + value + " but was null"); - } - if (cell_value != null) { - assertEquals(Bytes.toString(column) + " at timestamp " - + timestamp, value, new String(cell_value.getValue())); +// protected void assertCellEquals(final HRegion region, final byte [] row, +// final byte [] column, final long timestamp, final String value) +// throws IOException { +// Map result = region.getFull(row, null, timestamp, 1, null); +// Cell cell_value = result.get(column); +// if (value == null) { +// assertEquals(Bytes.toString(column) + " at timestamp " + timestamp, null, +// cell_value); +// } else { +// if (cell_value == null) { +// fail(Bytes.toString(column) + " at timestamp " + timestamp + +// "\" was expected to be \"" + value + " but was null"); +// } +// if (cell_value != null) { +// assertEquals(Bytes.toString(column) + " at timestamp " +// + timestamp, value, new String(cell_value.getValue())); +// } +// } +// } + + protected void assertResultEquals(final HRegion region, final byte [] row, + final byte [] family, final byte [] qualifier, final long timestamp, + final byte [] value) + throws IOException { + Get get = new Get(row); + get.setTimeStamp(timestamp); + Result res = region.get(get, null); + NavigableMap>> map = + res.getMap(); + byte [] res_value = map.get(family).get(qualifier).get(timestamp); + + if (value == null) { + assertEquals(Bytes.toString(family) + " " + Bytes.toString(qualifier) + + " at timestamp " + timestamp, null, res_value); + } else { + if (res_value == null) { + fail(Bytes.toString(family) + " " + Bytes.toString(qualifier) + + " at timestamp " + timestamp + "\" was expected to be \"" + + value + " but was null"); + } + if (res_value != null) { + assertEquals(Bytes.toString(family) + " " + Bytes.toString(qualifier) + + " at timestamp " + + timestamp, value, new String(res_value)); + } } } - } /** * Initializes parameters used in the test environment: diff --git a/src/java/org/apache/hadoop/hbase/client/tableindexed/ReverseByteArrayComparator.java b/src/test/org/apache/hadoop/hbase/KeyValueTestUtil.java similarity index 54% rename from src/java/org/apache/hadoop/hbase/client/tableindexed/ReverseByteArrayComparator.java rename to src/test/org/apache/hadoop/hbase/KeyValueTestUtil.java index 8af7b81d552..36d768a9fa1 100644 --- a/src/java/org/apache/hadoop/hbase/client/tableindexed/ReverseByteArrayComparator.java +++ b/src/test/org/apache/hadoop/hbase/KeyValueTestUtil.java @@ -1,5 +1,5 @@ -/** - * Copyright 2008 The Apache Software Foundation +/* + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -17,30 +17,38 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.client.tableindexed; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; +package org.apache.hadoop.hbase; -import org.apache.hadoop.hbase.WritableComparator; import org.apache.hadoop.hbase.util.Bytes; -public class ReverseByteArrayComparator implements WritableComparator { +public class KeyValueTestUtil { - /** {@inheritDoc} */ - public int compare(byte[] o1, byte[] o2) { - return Bytes.compareTo(o2, o1); + public static KeyValue create( + String row, + String family, + String qualifier, + long timestamp, + String value) + { + return create(row, family, qualifier, timestamp, KeyValue.Type.Put, value); } - - /** {@inheritDoc} */ - public void readFields(DataInput arg0) throws IOException { - // Nothing - } - - /** {@inheritDoc} */ - public void write(DataOutput arg0) throws IOException { - // Nothing + public static KeyValue create( + String row, + String family, + String qualifier, + long timestamp, + KeyValue.Type type, + String value) + { + return new KeyValue( + Bytes.toBytes(row), + Bytes.toBytes(family), + Bytes.toBytes(qualifier), + timestamp, + type, + Bytes.toBytes(value) + ); } } diff --git a/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java b/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java index 8ccb6a89315..02c94cf8e08 100644 --- a/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java +++ b/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java @@ -62,7 +62,7 @@ public class MiniHBaseCluster implements HConstants { } catch (BindException e) { //this port is already in use. try to use another (for multiple testing) int port = conf.getInt("hbase.master.port", DEFAULT_MASTER_PORT); - LOG.info("MiniHBaseCluster: Failed binding Master to port: " + port); + LOG.info("Failed binding Master to port: " + port, e); port++; conf.setInt("hbase.master.port", port); continue; diff --git a/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java b/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java index d2952619855..c32edf5ce98 100644 --- a/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java +++ b/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java @@ -37,13 +37,15 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.client.Get; 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.filter.PageRowFilter; -import org.apache.hadoop.hbase.filter.WhileMatchRowFilter; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.filter.PageFilter; +import org.apache.hadoop.hbase.filter.RowWhileMatchFilter; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Hash; @@ -86,12 +88,13 @@ public class PerformanceEvaluation implements HConstants { private static final int ONE_GB = 1024 * 1024 * 1000; private static final int ROWS_PER_GB = ONE_GB / ROW_LENGTH; - static final byte [] COLUMN_NAME = Bytes.toBytes(COLUMN_FAMILY_STR + "data"); + static final byte [] FAMILY_NAME = Bytes.toBytes("info"); + static final byte [] QUALIFIER_NAME = Bytes.toBytes("data"); protected static final HTableDescriptor TABLE_DESCRIPTOR; static { TABLE_DESCRIPTOR = new HTableDescriptor("TestTable"); - TABLE_DESCRIPTOR.addFamily(new HColumnDescriptor(COLUMN_FAMILY)); + TABLE_DESCRIPTOR.addFamily(new HColumnDescriptor(CATALOG_FAMILY)); } private static final String RANDOM_READ = "randomRead"; @@ -431,11 +434,12 @@ public class PerformanceEvaluation implements HConstants { @Override void testRow(final int i) throws IOException { - Scanner s = this.table.getScanner(new byte [][] {COLUMN_NAME}, - getRandomRow(this.rand, this.totalRows), - new WhileMatchRowFilter(new PageRowFilter(120))); + Scan scan = new Scan(getRandomRow(this.rand, this.totalRows)); + scan.addColumn(FAMILY_NAME, QUALIFIER_NAME); + scan.setFilter(new RowWhileMatchFilter(new PageFilter(120))); + ResultScanner s = this.table.getScanner(scan); //int count = 0; - for (RowResult rr = null; (rr = s.next()) != null;) { + for (Result rr = null; (rr = s.next()) != null;) { // LOG.info("" + count++ + " " + rr.toString()); } s.close(); @@ -461,7 +465,9 @@ public class PerformanceEvaluation implements HConstants { @Override void testRow(final int i) throws IOException { - this.table.get(getRandomRow(this.rand, this.totalRows), COLUMN_NAME); + Get get = new Get(getRandomRow(this.rand, this.totalRows)); + get.addColumn(FAMILY_NAME, QUALIFIER_NAME); + this.table.get(get); } @Override @@ -485,9 +491,9 @@ public class PerformanceEvaluation implements HConstants { @Override void testRow(final int i) throws IOException { byte [] row = getRandomRow(this.rand, this.totalRows); - BatchUpdate b = new BatchUpdate(row); - b.put(COLUMN_NAME, generateValue(this.rand)); - table.commit(b); + Put put = new Put(row); + put.add(FAMILY_NAME, QUALIFIER_NAME, generateValue(this.rand)); + table.put(put); } @Override @@ -497,7 +503,7 @@ public class PerformanceEvaluation implements HConstants { } class ScanTest extends Test { - private Scanner testScanner; + private ResultScanner testScanner; ScanTest(final HBaseConfiguration conf, final int startRow, final int perClientRunRows, final int totalRows, final Status status) { @@ -507,8 +513,9 @@ public class PerformanceEvaluation implements HConstants { @Override void testSetup() throws IOException { super.testSetup(); - this.testScanner = table.getScanner(new byte [][] {COLUMN_NAME}, - format(this.startRow)); + Scan scan = new Scan(format(this.startRow)); + scan.addColumn(FAMILY_NAME, QUALIFIER_NAME); + this.testScanner = table.getScanner(scan); } @Override @@ -539,7 +546,9 @@ public class PerformanceEvaluation implements HConstants { @Override void testRow(final int i) throws IOException { - table.get(format(i), COLUMN_NAME); + Get get = new Get(format(i)); + get.addColumn(FAMILY_NAME, QUALIFIER_NAME); + table.get(get); } @Override @@ -556,9 +565,9 @@ public class PerformanceEvaluation implements HConstants { @Override void testRow(final int i) throws IOException { - BatchUpdate b = new BatchUpdate(format(i)); - b.put(COLUMN_NAME, generateValue(this.rand)); - table.commit(b); + Put put = new Put(format(i)); + put.add(FAMILY_NAME, QUALIFIER_NAME, generateValue(this.rand)); + table.put(put); } @Override diff --git a/src/test/org/apache/hadoop/hbase/TestEmptyMetaInfo.java b/src/test/org/apache/hadoop/hbase/TestEmptyMetaInfo.java index 122d27854f8..9b5f63f9cef 100644 --- a/src/test/org/apache/hadoop/hbase/TestEmptyMetaInfo.java +++ b/src/test/org/apache/hadoop/hbase/TestEmptyMetaInfo.java @@ -23,9 +23,10 @@ package org.apache.hadoop.hbase; import java.io.IOException; 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.RowResult; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.util.Bytes; /** @@ -44,9 +45,10 @@ public class TestEmptyMetaInfo extends HBaseClusterTestCase { byte [] regionName = HRegionInfo.createRegionName(tableName, Bytes.toBytes(i == 0? "": Integer.toString(i)), Long.toString(System.currentTimeMillis())); - BatchUpdate b = new BatchUpdate(regionName); - b.put(HConstants.COL_SERVER, Bytes.toBytes("localhost:1234")); - t.commit(b); + Put put = new Put(regionName); + put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, + Bytes.toBytes("localhost:1234")); + t.put(put); } long sleepTime = conf.getLong("hbase.master.meta.thread.rescanfrequency", 10000); @@ -59,11 +61,18 @@ public class TestEmptyMetaInfo extends HBaseClusterTestCase { } catch (InterruptedException e) { // ignore } - Scanner scanner = t.getScanner(HConstants.ALL_META_COLUMNS, tableName); + Scan scan = new Scan(); + scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); + scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER); + scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER); + scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER); + ResultScanner scanner = t.getScanner(scan); try { count = 0; - for (RowResult r: scanner) { - if (r.size() > 0) { + Result r; + while((r = scanner.next()) != null) { + if (!r.isEmpty()) { count += 1; } } diff --git a/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java b/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java index 2c46619e989..07dce9ab16c 100644 --- a/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java +++ b/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java @@ -1,5 +1,5 @@ /** - * Copyright 2007 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -21,15 +21,16 @@ package org.apache.hadoop.hbase; import java.io.IOException; import java.util.Collection; -import java.util.Iterator; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.client.Get; 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.RowResult; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.util.Bytes; /** @@ -75,18 +76,19 @@ public class TestHBaseCluster extends HBaseClusterTestCase { private static final int FIRST_ROW = 1; private static final int NUM_VALS = 1000; - private static final byte [] CONTENTS = Bytes.toBytes("contents:"); - private static final String CONTENTS_BASIC_STR = "contents:basic"; - private static final byte [] CONTENTS_BASIC = Bytes.toBytes(CONTENTS_BASIC_STR); + private static final byte [] CONTENTS_CF = Bytes.toBytes("contents"); + private static final String CONTENTS_CQ_STR = "basic"; + private static final byte [] CONTENTS_CQ = Bytes.toBytes(CONTENTS_CQ_STR); private static final String CONTENTSTR = "contentstr"; - private static final byte [] ANCHOR = Bytes.toBytes("anchor:"); - private static final String ANCHORNUM = "anchor:anchornum-"; - private static final String ANCHORSTR = "anchorstr"; + // + private static final byte [] ANCHOR_CF = Bytes.toBytes("anchor"); + private static final String ANCHORNUM_CQ = "anchornum-"; + private static final String ANCHORSTR_VALUE = "anchorstr"; private void setup() throws IOException { desc = new HTableDescriptor("test"); - desc.addFamily(new HColumnDescriptor(CONTENTS)); - desc.addFamily(new HColumnDescriptor(ANCHOR)); + desc.addFamily(new HColumnDescriptor(CONTENTS_CF)); + desc.addFamily(new HColumnDescriptor(ANCHOR_CF)); admin = new HBaseAdmin(conf); admin.createTable(desc); table = new HTable(conf, desc.getName()); @@ -100,10 +102,10 @@ public class TestHBaseCluster extends HBaseClusterTestCase { // Write out a bunch of values for (int k = FIRST_ROW; k <= NUM_VALS; k++) { - BatchUpdate b = new BatchUpdate("row_" + k); - b.put(CONTENTS_BASIC, Bytes.toBytes(CONTENTSTR + k)); - b.put(ANCHORNUM + k, Bytes.toBytes(ANCHORSTR + k)); - table.commit(b); + Put put = new Put(Bytes.toBytes("row_" + k)); + put.add(CONTENTS_CF, CONTENTS_CQ, Bytes.toBytes(CONTENTSTR + k)); + put.add(ANCHOR_CF, Bytes.toBytes(ANCHORNUM_CQ + k), Bytes.toBytes(ANCHORSTR_VALUE + k)); + table.put(put); } LOG.info("Write " + NUM_VALS + " rows. Elapsed time: " + ((System.currentTimeMillis() - startTime) / 1000.0)); @@ -117,21 +119,27 @@ public class TestHBaseCluster extends HBaseClusterTestCase { String rowlabelStr = "row_" + k; byte [] rowlabel = Bytes.toBytes(rowlabelStr); - byte bodydata[] = table.get(rowlabel, CONTENTS_BASIC).getValue(); - assertNotNull("no data for row " + rowlabelStr + "/" + CONTENTS_BASIC_STR, + Get get = new Get(rowlabel); + get.addColumn(CONTENTS_CF, CONTENTS_CQ); + byte [] bodydata = table.get(get).getValue(CONTENTS_CF, CONTENTS_CQ); + assertNotNull("no data for row " + rowlabelStr + "/" + CONTENTS_CQ_STR, bodydata); String bodystr = new String(bodydata, HConstants.UTF8_ENCODING); String teststr = CONTENTSTR + k; assertTrue("Incorrect value for key: (" + rowlabelStr + "/" + - CONTENTS_BASIC_STR + "), expected: '" + teststr + "' got: '" + + CONTENTS_CQ_STR + "), expected: '" + teststr + "' got: '" + bodystr + "'", teststr.compareTo(bodystr) == 0); - String collabelStr = ANCHORNUM + k; + String collabelStr = ANCHORNUM_CQ + k; collabel = Bytes.toBytes(collabelStr); - bodydata = table.get(rowlabel, collabel).getValue(); + + get = new Get(rowlabel); + get.addColumn(ANCHOR_CF, collabel); + + bodydata = table.get(get).getValue(ANCHOR_CF, collabel); assertNotNull("no data for row " + rowlabelStr + "/" + collabelStr, bodydata); bodystr = new String(bodydata, HConstants.UTF8_ENCODING); - teststr = ANCHORSTR + k; + teststr = ANCHORSTR_VALUE + k; assertTrue("Incorrect value for key: (" + rowlabelStr + "/" + collabelStr + "), expected: '" + teststr + "' got: '" + bodystr + "'", teststr.compareTo(bodystr) == 0); @@ -142,47 +150,48 @@ public class TestHBaseCluster extends HBaseClusterTestCase { } private void scanner() throws IOException { - byte [][] cols = new byte [][] {Bytes.toBytes(ANCHORNUM + "[0-9]+"), - CONTENTS_BASIC}; long startTime = System.currentTimeMillis(); - Scanner s = table.getScanner(cols, HConstants.EMPTY_BYTE_ARRAY); + Scan scan = new Scan(); + scan.addFamily(ANCHOR_CF); + scan.addColumn(CONTENTS_CF, CONTENTS_CQ); + ResultScanner s = table.getScanner(scan); try { int contentsFetched = 0; int anchorFetched = 0; int k = 0; - for (RowResult curVals : s) { - for (Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { - byte [] col = it.next(); - byte val[] = curVals.get(col).getValue(); - String curval = Bytes.toString(val); - if (Bytes.compareTo(col, CONTENTS_BASIC) == 0) { + for (Result curVals : s) { + for(KeyValue kv : curVals.raw()) { + byte [] family = kv.getFamily(); + byte [] qualifier = kv.getQualifier(); + String strValue = new String(kv.getValue()); + if(Bytes.equals(family, CONTENTS_CF)) { assertTrue("Error at:" + Bytes.toString(curVals.getRow()) - + ", Value for " + Bytes.toString(col) + " should start with: " + CONTENTSTR - + ", but was fetched as: " + curval, - curval.startsWith(CONTENTSTR)); + + ", Value for " + Bytes.toString(qualifier) + " should start with: " + CONTENTSTR + + ", but was fetched as: " + strValue, + strValue.startsWith(CONTENTSTR)); contentsFetched++; - } else if (Bytes.toString(col).startsWith(ANCHORNUM)) { - assertTrue("Error at:" + Bytes.toString(curVals.getRow()) - + ", Value for " + Bytes.toString(col) + " should start with: " + ANCHORSTR - + ", but was fetched as: " + curval, - curval.startsWith(ANCHORSTR)); + } else if(Bytes.equals(family, ANCHOR_CF)) { + assertTrue("Error at:" + Bytes.toString(curVals.getRow()) + + ", Value for " + Bytes.toString(qualifier) + " should start with: " + ANCHORSTR_VALUE + + ", but was fetched as: " + strValue, + strValue.startsWith(ANCHORSTR_VALUE)); anchorFetched++; } else { - LOG.info(Bytes.toString(col)); + LOG.info("Family: " + Bytes.toString(family) + ", Qualifier: " + Bytes.toString(qualifier)); } } k++; } assertEquals("Expected " + NUM_VALS + " " + - Bytes.toString(CONTENTS_BASIC) + " values, but fetched " + + Bytes.toString(CONTENTS_CQ) + " values, but fetched " + contentsFetched, NUM_VALS, contentsFetched); - assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + + assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM_CQ + " values, but fetched " + anchorFetched, NUM_VALS, anchorFetched); @@ -201,7 +210,7 @@ public class TestHBaseCluster extends HBaseClusterTestCase { assertTrue(Bytes.equals(desc.getName(), tables[0].getName())); Collection families = tables[0].getFamilies(); assertEquals(2, families.size()); - assertTrue(tables[0].hasFamily(CONTENTS)); - assertTrue(tables[0].hasFamily(ANCHOR)); + assertTrue(tables[0].hasFamily(CONTENTS_CF)); + assertTrue(tables[0].hasFamily(ANCHOR_CF)); } } \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/TestKeyValue.java b/src/test/org/apache/hadoop/hbase/TestKeyValue.java index 861f4f71946..34a1ab11189 100644 --- a/src/test/org/apache/hadoop/hbase/TestKeyValue.java +++ b/src/test/org/apache/hadoop/hbase/TestKeyValue.java @@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.KVComparator; +import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.util.Bytes; public class TestKeyValue extends TestCase { @@ -39,13 +40,21 @@ public class TestKeyValue extends TestCase { final byte [] a = Bytes.toBytes("aaa"); byte [] column1 = Bytes.toBytes("abc:def"); byte [] column2 = Bytes.toBytes("abcd:ef"); - KeyValue aaa = new KeyValue(a, column1, a); - assertFalse(KeyValue.COMPARATOR. - compareColumns(aaa, column2, 0, column2.length, 4) == 0); + byte [] family2 = Bytes.toBytes("abcd"); + byte [] qualifier2 = Bytes.toBytes("ef"); + KeyValue aaa = new KeyValue(a, column1, 0L, Type.Put, a); + assertFalse(aaa.matchingColumn(column2)); + assertTrue(aaa.matchingColumn(column1)); + aaa = new KeyValue(a, column2, 0L, Type.Put, a); + assertFalse(aaa.matchingColumn(column1)); + assertTrue(aaa.matchingColumn(family2,qualifier2)); column1 = Bytes.toBytes("abcd:"); - aaa = new KeyValue(a, column1, a); - assertFalse(KeyValue.COMPARATOR. - compareColumns(aaa, column1, 0, column1.length, 4) == 0); + aaa = new KeyValue(a, column1, 0L, Type.Put, a); + assertTrue(aaa.matchingColumn(family2,null)); + assertFalse(aaa.matchingColumn(family2,qualifier2)); + // Previous test had an assertFalse that I don't understand + // assertFalse(KeyValue.COMPARATOR. + // compareColumns(aaa, column1, 0, column1.length, 4) == 0); } public void testBasics() throws Exception { @@ -111,31 +120,31 @@ public class TestKeyValue extends TestCase { public void testMoreComparisons() throws Exception { // Root compares long now = System.currentTimeMillis(); - KeyValue a = new KeyValue(".META.,,99999999999999", now); - KeyValue b = new KeyValue(".META.,,1", now); + KeyValue a = new KeyValue(Bytes.toBytes(".META.,,99999999999999"), now); + KeyValue b = new KeyValue(Bytes.toBytes(".META.,,1"), now); KVComparator c = new KeyValue.RootComparator(); assertTrue(c.compare(b, a) < 0); - KeyValue aa = new KeyValue(".META.,,1", now); - KeyValue bb = new KeyValue(".META.,,1", "info:regioninfo", - 1235943454602L); + KeyValue aa = new KeyValue(Bytes.toBytes(".META.,,1"), now); + KeyValue bb = new KeyValue(Bytes.toBytes(".META.,,1"), + Bytes.toBytes("info:regioninfo"), 1235943454602L); assertTrue(c.compare(aa, bb) < 0); // Meta compares - KeyValue aaa = - new KeyValue("TestScanMultipleVersions,row_0500,1236020145502", now); - KeyValue bbb = new KeyValue("TestScanMultipleVersions,,99999999999999", - now); + KeyValue aaa = new KeyValue( + Bytes.toBytes("TestScanMultipleVersions,row_0500,1236020145502"), now); + KeyValue bbb = new KeyValue( + Bytes.toBytes("TestScanMultipleVersions,,99999999999999"), now); c = new KeyValue.MetaComparator(); assertTrue(c.compare(bbb, aaa) < 0); - KeyValue aaaa = new KeyValue("TestScanMultipleVersions,,1236023996656", - "info:regioninfo", 1236024396271L); + KeyValue aaaa = new KeyValue(Bytes.toBytes("TestScanMultipleVersions,,1236023996656"), + Bytes.toBytes("info:regioninfo"), 1236024396271L); assertTrue(c.compare(aaaa, bbb) < 0); - KeyValue x = new KeyValue("TestScanMultipleVersions,row_0500,1236034574162", - "", 9223372036854775807L); - KeyValue y = new KeyValue("TestScanMultipleVersions,row_0500,1236034574162", - "info:regioninfo", 1236034574912L); + KeyValue x = new KeyValue(Bytes.toBytes("TestScanMultipleVersions,row_0500,1236034574162"), + Bytes.toBytes(""), 9223372036854775807L); + KeyValue y = new KeyValue(Bytes.toBytes("TestScanMultipleVersions,row_0500,1236034574162"), + Bytes.toBytes("info:regioninfo"), 1236034574912L); assertTrue(c.compare(x, y) < 0); comparisons(new KeyValue.MetaComparator()); comparisons(new KeyValue.KVComparator()); @@ -151,53 +160,53 @@ public class TestKeyValue extends TestCase { public void testKeyValueBorderCases() throws IOException { // % sorts before , so if we don't do special comparator, rowB would // come before rowA. - KeyValue rowA = new KeyValue("testtable,www.hbase.org/,1234", - "", Long.MAX_VALUE); - KeyValue rowB = new KeyValue("testtable,www.hbase.org/%20,99999", - "", Long.MAX_VALUE); + KeyValue rowA = new KeyValue(Bytes.toBytes("testtable,www.hbase.org/,1234"), + Bytes.toBytes(""), Long.MAX_VALUE); + KeyValue rowB = new KeyValue(Bytes.toBytes("testtable,www.hbase.org/%20,99999"), + Bytes.toBytes(""), Long.MAX_VALUE); assertTrue(KeyValue.META_COMPARATOR.compare(rowA, rowB) < 0); - rowA = new KeyValue("testtable,,1234", "", Long.MAX_VALUE); - rowB = new KeyValue("testtable,$www.hbase.org/,99999", "", Long.MAX_VALUE); + rowA = new KeyValue(Bytes.toBytes("testtable,,1234"), Bytes.toBytes(""), Long.MAX_VALUE); + rowB = new KeyValue(Bytes.toBytes("testtable,$www.hbase.org/,99999"), Bytes.toBytes(""), Long.MAX_VALUE); assertTrue(KeyValue.META_COMPARATOR.compare(rowA, rowB) < 0); - rowA = new KeyValue(".META.,testtable,www.hbase.org/,1234,4321", "", + rowA = new KeyValue(Bytes.toBytes(".META.,testtable,www.hbase.org/,1234,4321"), Bytes.toBytes(""), Long.MAX_VALUE); - rowB = new KeyValue(".META.,testtable,www.hbase.org/%20,99999,99999", "", + rowB = new KeyValue(Bytes.toBytes(".META.,testtable,www.hbase.org/%20,99999,99999"), Bytes.toBytes(""), Long.MAX_VALUE); assertTrue(KeyValue.ROOT_COMPARATOR.compare(rowA, rowB) < 0); } private void metacomparisons(final KeyValue.MetaComparator c) { long now = System.currentTimeMillis(); - assertTrue(c.compare(new KeyValue(".META.,a,,0,1", now), - new KeyValue(".META.,a,,0,1", now)) == 0); - KeyValue a = new KeyValue(".META.,a,,0,1", now); - KeyValue b = new KeyValue(".META.,a,,0,2", now); + assertTrue(c.compare(new KeyValue(Bytes.toBytes(".META.,a,,0,1"), now), + new KeyValue(Bytes.toBytes(".META.,a,,0,1"), now)) == 0); + KeyValue a = new KeyValue(Bytes.toBytes(".META.,a,,0,1"), now); + KeyValue b = new KeyValue(Bytes.toBytes(".META.,a,,0,2"), now); assertTrue(c.compare(a, b) < 0); - assertTrue(c.compare(new KeyValue(".META.,a,,0,2", now), - new KeyValue(".META.,a,,0,1", now)) > 0); + assertTrue(c.compare(new KeyValue(Bytes.toBytes(".META.,a,,0,2"), now), + new KeyValue(Bytes.toBytes(".META.,a,,0,1"), now)) > 0); } private void comparisons(final KeyValue.KVComparator c) { long now = System.currentTimeMillis(); - assertTrue(c.compare(new KeyValue(".META.,,1", now), - new KeyValue(".META.,,1", now)) == 0); - assertTrue(c.compare(new KeyValue(".META.,,1", now), - new KeyValue(".META.,,2", now)) < 0); - assertTrue(c.compare(new KeyValue(".META.,,2", now), - new KeyValue(".META.,,1", now)) > 0); + assertTrue(c.compare(new KeyValue(Bytes.toBytes(".META.,,1"), now), + new KeyValue(Bytes.toBytes(".META.,,1"), now)) == 0); + assertTrue(c.compare(new KeyValue(Bytes.toBytes(".META.,,1"), now), + new KeyValue(Bytes.toBytes(".META.,,2"), now)) < 0); + assertTrue(c.compare(new KeyValue(Bytes.toBytes(".META.,,2"), now), + new KeyValue(Bytes.toBytes(".META.,,1"), now)) > 0); } public void testBinaryKeys() throws Exception { Set set = new TreeSet(KeyValue.COMPARATOR); - String column = "col:umn"; - KeyValue [] keys = {new KeyValue("aaaaa,\u0000\u0000,2", column, 2), - new KeyValue("aaaaa,\u0001,3", column, 3), - new KeyValue("aaaaa,,1", column, 1), - new KeyValue("aaaaa,\u1000,5", column, 5), - new KeyValue("aaaaa,a,4", column, 4), - new KeyValue("a,a,0", column, 0), + byte [] column = Bytes.toBytes("col:umn"); + KeyValue [] keys = {new KeyValue(Bytes.toBytes("aaaaa,\u0000\u0000,2"), column, 2), + new KeyValue(Bytes.toBytes("aaaaa,\u0001,3"), column, 3), + new KeyValue(Bytes.toBytes("aaaaa,,1"), column, 1), + new KeyValue(Bytes.toBytes("aaaaa,\u1000,5"), column, 5), + new KeyValue(Bytes.toBytes("aaaaa,a,4"), column, 4), + new KeyValue(Bytes.toBytes("a,a,0"), column, 0), }; // Add to set with bad comparator for (int i = 0; i < keys.length; i++) { @@ -226,12 +235,12 @@ public class TestKeyValue extends TestCase { } // Make up -ROOT- table keys. KeyValue [] rootKeys = { - new KeyValue(".META.,aaaaa,\u0000\u0000,0,2", column, 2), - new KeyValue(".META.,aaaaa,\u0001,0,3", column, 3), - new KeyValue(".META.,aaaaa,,0,1", column, 1), - new KeyValue(".META.,aaaaa,\u1000,0,5", column, 5), - new KeyValue(".META.,aaaaa,a,0,4", column, 4), - new KeyValue(".META.,,0", column, 0), + new KeyValue(Bytes.toBytes(".META.,aaaaa,\u0000\u0000,0,2"), column, 2), + new KeyValue(Bytes.toBytes(".META.,aaaaa,\u0001,0,3"), column, 3), + new KeyValue(Bytes.toBytes(".META.,aaaaa,,0,1"), column, 1), + new KeyValue(Bytes.toBytes(".META.,aaaaa,\u1000,0,5"), column, 5), + new KeyValue(Bytes.toBytes(".META.,aaaaa,a,0,4"), column, 4), + new KeyValue(Bytes.toBytes(".META.,,0"), column, 0), }; // This will output the keys incorrectly. set = new TreeSet(new KeyValue.MetaComparator()); @@ -260,4 +269,11 @@ public class TestKeyValue extends TestCase { assertTrue(count++ == k.getTimestamp()); } } + + public void testStackedUpKeyValue() { + // Test multiple KeyValues in a single blob. + + // TODO actually write this test! + + } } \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java b/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java index 87f31281c14..22abfaaec89 100644 --- a/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java +++ b/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java @@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegion; @@ -223,9 +224,10 @@ public class TestRegionRebalancing extends HBaseClusterTestCase { throws IOException { HRegion region = createNewHRegion(desc, startKey, endKey); byte [] keyToWrite = startKey == null ? Bytes.toBytes("row_000") : startKey; - BatchUpdate bu = new BatchUpdate(keyToWrite); - bu.put(COLUMN_NAME, "test".getBytes()); - region.batchUpdate(bu, null); + Put put = new Put(keyToWrite); + byte [][] famAndQf = KeyValue.parseColumn(COLUMN_NAME); + put.add(famAndQf[0], famAndQf[1], Bytes.toBytes("test")); + region.put(put); region.close(); region.getLog().closeAndDelete(); return region; diff --git a/src/test/org/apache/hadoop/hbase/TestScanMultipleVersions.java b/src/test/org/apache/hadoop/hbase/TestScanMultipleVersions.java index 37a89ce6c8f..0edca0258e3 100644 --- a/src/test/org/apache/hadoop/hbase/TestScanMultipleVersions.java +++ b/src/test/org/apache/hadoop/hbase/TestScanMultipleVersions.java @@ -21,11 +21,12 @@ package org.apache.hadoop.hbase; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.client.Get; 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.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.util.Bytes; @@ -53,7 +54,7 @@ public class TestScanMultipleVersions extends HBaseClusterTestCase { // Create table description this.desc = new HTableDescriptor(TABLE_NAME); - this.desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY)); + this.desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); // Region 0 will contain the key range [,row_0500) INFOS[0] = new HRegionInfo(this.desc, HConstants.EMPTY_START_ROW, @@ -70,9 +71,11 @@ public class TestScanMultipleVersions extends HBaseClusterTestCase { HRegion.createHRegion(this.INFOS[i], this.testDir, this.conf); // Insert data for (int j = 0; j < TIMESTAMPS.length; j++) { - BatchUpdate b = new BatchUpdate(ROWS[i], TIMESTAMPS[j]); - b.put(HConstants.COLUMN_FAMILY, Bytes.toBytes(TIMESTAMPS[j])); - REGIONS[i].batchUpdate(b, null); + Put put = new Put(ROWS[i]); + put.setTimeStamp(TIMESTAMPS[j]); + put.add(HConstants.CATALOG_FAMILY, null, TIMESTAMPS[j], + Bytes.toBytes(TIMESTAMPS[j])); + REGIONS[i].put(put); } // Insert the region we created into the meta HRegion.addRegionToMETA(meta, REGIONS[i]); @@ -93,19 +96,25 @@ public class TestScanMultipleVersions extends HBaseClusterTestCase { HTable t = new HTable(conf, TABLE_NAME); for (int i = 0; i < ROWS.length; i++) { for (int j = 0; j < TIMESTAMPS.length; j++) { - Cell [] cells = - t.get(ROWS[i], HConstants.COLUMN_FAMILY, TIMESTAMPS[j], 1); - assertTrue(cells != null && cells.length == 1); - System.out.println("Row=" + Bytes.toString(ROWS[i]) + ", cell=" + - cells[0]); + Get get = new Get(ROWS[i]); + get.addFamily(HConstants.CATALOG_FAMILY); + get.setTimeStamp(TIMESTAMPS[j]); + Result result = t.get(get); + int cellCount = 0; + for(@SuppressWarnings("unused")KeyValue kv : result.sorted()) { + cellCount++; + } + assertTrue(cellCount == 1); } } // Case 1: scan with LATEST_TIMESTAMP. Should get two rows int count = 0; - Scanner s = t.getScanner(HConstants.COLUMN_FAMILY_ARRAY); + Scan scan = new Scan(); + scan.addFamily(HConstants.CATALOG_FAMILY); + ResultScanner s = t.getScanner(scan); try { - for (RowResult rr = null; (rr = s.next()) != null;) { + for (Result rr = null; (rr = s.next()) != null;) { System.out.println(rr.toString()); count += 1; } @@ -118,8 +127,11 @@ public class TestScanMultipleVersions extends HBaseClusterTestCase { // (in this case > 1000 and < LATEST_TIMESTAMP. Should get 2 rows. count = 0; - s = t.getScanner(HConstants.COLUMN_FAMILY_ARRAY, HConstants.EMPTY_START_ROW, - 10000L); + scan = new Scan(); + scan.setTimeRange(1000L, Long.MAX_VALUE); + scan.addFamily(HConstants.CATALOG_FAMILY); + + s = t.getScanner(scan); try { while (s.next() != null) { count += 1; @@ -133,8 +145,11 @@ public class TestScanMultipleVersions extends HBaseClusterTestCase { // (in this case == 1000. Should get 2 rows. count = 0; - s = t.getScanner(HConstants.COLUMN_FAMILY_ARRAY, HConstants.EMPTY_START_ROW, - 1000L); + scan = new Scan(); + scan.setTimeStamp(1000L); + scan.addFamily(HConstants.CATALOG_FAMILY); + + s = t.getScanner(scan); try { while (s.next() != null) { count += 1; @@ -148,8 +163,11 @@ public class TestScanMultipleVersions extends HBaseClusterTestCase { // second timestamp (100 < timestamp < 1000). Should get 2 rows. count = 0; - s = t.getScanner(HConstants.COLUMN_FAMILY_ARRAY, HConstants.EMPTY_START_ROW, - 500L); + scan = new Scan(); + scan.setTimeRange(100L, 1000L); + scan.addFamily(HConstants.CATALOG_FAMILY); + + s = t.getScanner(scan); try { while (s.next() != null) { count += 1; @@ -163,8 +181,11 @@ public class TestScanMultipleVersions extends HBaseClusterTestCase { // Should get 2 rows. count = 0; - s = t.getScanner(HConstants.COLUMN_FAMILY_ARRAY, HConstants.EMPTY_START_ROW, - 100L); + scan = new Scan(); + scan.setTimeStamp(100L); + scan.addFamily(HConstants.CATALOG_FAMILY); + + s = t.getScanner(scan); try { while (s.next() != null) { count += 1; diff --git a/src/test/org/apache/hadoop/hbase/TestScannerAPI.java b/src/test/org/apache/hadoop/hbase/TestScannerAPI.java deleted file mode 100644 index 85972a6d14d..00000000000 --- a/src/test/org/apache/hadoop/hbase/TestScannerAPI.java +++ /dev/null @@ -1,166 +0,0 @@ -/** - * 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; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; - -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.regionserver.HRegion; -import org.apache.hadoop.hbase.util.Bytes; - -/** test the scanner API at all levels */ -public class TestScannerAPI extends HBaseClusterTestCase { - private final byte [][] columns = Bytes.toByteArrays(new String[] { - "a:", "b:" - }); - private final byte [] startRow = Bytes.toBytes("0"); - - private final TreeMap> values = - new TreeMap>(Bytes.BYTES_COMPARATOR); - - /** - * @throws Exception - */ - public TestScannerAPI() throws Exception { - super(); - try { - TreeMap columns = - new TreeMap(Bytes.BYTES_COMPARATOR); - columns.put(Bytes.toBytes("a:1"), - new Cell(Bytes.toBytes("1"), HConstants.LATEST_TIMESTAMP)); - values.put(Bytes.toBytes("1"), columns); - columns = new TreeMap(Bytes.BYTES_COMPARATOR); - columns.put(Bytes.toBytes("a:2"), - new Cell(Bytes.toBytes("2"), HConstants.LATEST_TIMESTAMP)); - columns.put(Bytes.toBytes("b:2"), - new Cell(Bytes.toBytes("2"), HConstants.LATEST_TIMESTAMP)); - } catch (Exception e) { - e.printStackTrace(); - throw e; - } - } - - /** - * @throws IOException - */ - public void testApi() throws IOException { - final String tableName = getName(); - - // Create table - - HBaseAdmin admin = new HBaseAdmin(conf); - HTableDescriptor tableDesc = new HTableDescriptor(tableName); - for (int i = 0; i < columns.length; i++) { - tableDesc.addFamily(new HColumnDescriptor(columns[i])); - } - admin.createTable(tableDesc); - - // Insert values - - HTable table = new HTable(conf, getName()); - - for (Map.Entry> row: values.entrySet()) { - BatchUpdate b = new BatchUpdate(row.getKey()); - for (Map.Entry val: row.getValue().entrySet()) { - b.put(val.getKey(), val.getValue().getValue()); - } - table.commit(b); - } - - HRegion region = null; - try { - Collection regions = - cluster.getRegionThreads().get(0).getRegionServer().getOnlineRegions(); - for (HRegion r: regions) { - if (!r.getRegionInfo().isMetaRegion()) { - region = r; - } - } - } catch (Exception e) { - e.printStackTrace(); - IOException iox = new IOException("error finding region"); - iox.initCause(e); - throw iox; - } - @SuppressWarnings("null") - ScannerIncommon scanner = new InternalScannerIncommon( - region.getScanner(columns, startRow, System.currentTimeMillis(), null)); - try { - verify(scanner); - } finally { - scanner.close(); - } - - scanner = new ClientScannerIncommon(table.getScanner(columns, startRow)); - try { - verify(scanner); - } finally { - scanner.close(); - } - Scanner scanner2 = table.getScanner(columns, startRow); - try { - for (RowResult r : scanner2) { - assertTrue("row key", values.containsKey(r.getRow())); - - SortedMap columnValues = values.get(r.getRow()); - assertEquals(columnValues.size(), r.size()); - for (Map.Entry e: columnValues.entrySet()) { - byte [] column = e.getKey(); - assertTrue("column", r.containsKey(column)); - assertTrue("value", Arrays.equals(columnValues.get(column).getValue(), - r.get(column).getValue())); - } - } - } finally { - scanner.close(); - } - } - - private void verify(ScannerIncommon scanner) throws IOException { - List results = new ArrayList(); - while (scanner.next(results)) { - assertTrue("row key", values.containsKey(results.get(0).getRow())); - // TODO FIX. -// SortedMap columnValues = values.get(row); -// assertEquals(columnValues.size(), results.size()); -// for (Map.Entry e: columnValues.entrySet()) { -// byte [] column = e.getKey(); -// assertTrue("column", results.containsKey(column)); -// assertTrue("value", Arrays.equals(columnValues.get(column).getValue(), -// results.get(column).getValue())); -// } -// - results.clear(); - } - } -} \ 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 3d66b43b709..556f1bf1847 100644 --- a/src/test/org/apache/hadoop/hbase/TestSerialization.java +++ b/src/test/org/apache/hadoop/hbase/TestSerialization.java @@ -1,5 +1,5 @@ /** - * Copyright 2007 The Apache Software Foundation + * Copyright 2009 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -20,13 +20,31 @@ package org.apache.hadoop.hbase; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.NavigableSet; + +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.RowLock; +import org.apache.hadoop.hbase.client.Scan; 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.HbaseMapWritable; import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.io.DataInputBuffer; +import org.apache.hadoop.io.Writable; /** * Test HBase Writables serializations @@ -52,6 +70,7 @@ public class TestSerialization extends HBaseTestCase { assertTrue(KeyValue.COMPARATOR.compare(original, newone) == 0); } + @SuppressWarnings("unchecked") public void testHbaseMapWritable() throws Exception { HbaseMapWritable hmw = new HbaseMapWritable(); @@ -157,7 +176,7 @@ public class TestSerialization extends HBaseTestCase { assertTrue(Bytes.equals(bu.getRow(), bubu.getRow())); // Assert has same number of BatchOperations. int firstCount = 0; - for (BatchOperation bo: bubu) { + for (@SuppressWarnings("unused")BatchOperation bo: bubu) { firstCount++; } // Now deserialize again into same instance to ensure we're not @@ -166,9 +185,358 @@ public class TestSerialization extends HBaseTestCase { // Assert rows are same again. assertTrue(Bytes.equals(bu.getRow(), bububu.getRow())); int secondCount = 0; - for (BatchOperation bo: bububu) { + for (@SuppressWarnings("unused")BatchOperation bo: bububu) { secondCount++; } assertEquals(firstCount, secondCount); } + + + // + // HBASE-880 + // + + public void testPut() throws Exception{ + byte[] row = "row".getBytes(); + byte[] fam = "fam".getBytes(); + byte[] qf1 = "qf1".getBytes(); + byte[] qf2 = "qf2".getBytes(); + byte[] qf3 = "qf3".getBytes(); + byte[] qf4 = "qf4".getBytes(); + byte[] qf5 = "qf5".getBytes(); + byte[] qf6 = "qf6".getBytes(); + byte[] qf7 = "qf7".getBytes(); + byte[] qf8 = "qf8".getBytes(); + + long ts = System.currentTimeMillis(); + byte[] val = "val".getBytes(); + + Put put = new Put(row); + put.add(fam, qf1, ts, val); + put.add(fam, qf2, ts, val); + put.add(fam, qf3, ts, val); + put.add(fam, qf4, ts, val); + put.add(fam, qf5, ts, val); + put.add(fam, qf6, ts, val); + put.add(fam, qf7, ts, val); + put.add(fam, qf8, ts, val); + + byte[] sb = Writables.getBytes(put); + Put desPut = (Put)Writables.getWritable(sb, new Put()); + + //Timing test +// long start = System.nanoTime(); +// desPut = (Put)Writables.getWritable(sb, new Put()); +// long stop = System.nanoTime(); +// System.out.println("timer " +(stop-start)); + + assertTrue(Bytes.equals(put.getRow(), desPut.getRow())); + List list = null; + List desList = null; + for(Map.Entry> entry : put.getFamilyMap().entrySet()){ + assertTrue(desPut.getFamilyMap().containsKey(entry.getKey())); + list = entry.getValue(); + desList = desPut.getFamilyMap().get(entry.getKey()); + for(int i=0; i list = null; + List desList = null; + for(Map.Entry> entry : put.getFamilyMap().entrySet()){ + assertTrue(desPut.getFamilyMap().containsKey(entry.getKey())); + list = entry.getValue(); + desList = desPut.getFamilyMap().get(entry.getKey()); + for(int i=0; i list = null; + List desList = null; + for(Map.Entry> entry : + delete.getFamilyMap().entrySet()){ + assertTrue(desDelete.getFamilyMap().containsKey(entry.getKey())); + list = entry.getValue(); + desList = desDelete.getFamilyMap().get(entry.getKey()); + for(int i=0; i set = null; + Set desSet = null; + + for(Map.Entry> entry : + get.getFamilyMap().entrySet()){ + assertTrue(desGet.getFamilyMap().containsKey(entry.getKey())); + set = entry.getValue(); + desSet = desGet.getFamilyMap().get(entry.getKey()); + for(byte [] qualifier : set){ + assertTrue(desSet.contains(qualifier)); + } + } + + assertEquals(get.getLockId(), desGet.getLockId()); + assertEquals(get.getMaxVersions(), desGet.getMaxVersions()); + TimeRange tr = get.getTimeRange(); + TimeRange desTr = desGet.getTimeRange(); + assertEquals(tr.getMax(), desTr.getMax()); + assertEquals(tr.getMin(), desTr.getMin()); + } + + + public void testScan() throws Exception{ + byte[] startRow = "startRow".getBytes(); + byte[] stopRow = "stopRow".getBytes(); + byte[] fam = "fam".getBytes(); + byte[] qf1 = "qf1".getBytes(); + + long ts = System.currentTimeMillis(); + int maxVersions = 2; + + Scan scan = new Scan(startRow, stopRow); + scan.addColumn(fam, qf1); + scan.setTimeRange(ts, ts+1); + scan.setMaxVersions(maxVersions); + + byte[] sb = Writables.getBytes(scan); + Scan desScan = (Scan)Writables.getWritable(sb, new Scan()); + + assertTrue(Bytes.equals(scan.getStartRow(), desScan.getStartRow())); + assertTrue(Bytes.equals(scan.getStopRow(), desScan.getStopRow())); + Set set = null; + Set desSet = null; + + for(Map.Entry> entry : + scan.getFamilyMap().entrySet()){ + assertTrue(desScan.getFamilyMap().containsKey(entry.getKey())); + set = entry.getValue(); + desSet = desScan.getFamilyMap().get(entry.getKey()); + for(byte[] column : set){ + assertTrue(desSet.contains(column)); + } + } + + assertEquals(scan.getMaxVersions(), desScan.getMaxVersions()); + TimeRange tr = scan.getTimeRange(); + TimeRange desTr = desScan.getTimeRange(); + assertEquals(tr.getMax(), desTr.getMax()); + assertEquals(tr.getMin(), desTr.getMin()); + } + + public void testResultEmpty() throws Exception { + List keys = new ArrayList(); + Result r = new Result(keys); + assertTrue(r.isEmpty()); + byte [] rb = Writables.getBytes(r); + Result deserializedR = (Result)Writables.getWritable(rb, new Result()); + assertTrue(deserializedR.isEmpty()); + } + + + public void testResult() throws Exception { + byte [] rowA = Bytes.toBytes("rowA"); + byte [] famA = Bytes.toBytes("famA"); + byte [] qfA = Bytes.toBytes("qfA"); + byte [] valueA = Bytes.toBytes("valueA"); + + byte [] rowB = Bytes.toBytes("rowB"); + byte [] famB = Bytes.toBytes("famB"); + byte [] qfB = Bytes.toBytes("qfB"); + byte [] valueB = Bytes.toBytes("valueB"); + + KeyValue kvA = new KeyValue(rowA, famA, qfA, valueA); + KeyValue kvB = new KeyValue(rowB, famB, qfB, valueB); + + Result result = new Result(new KeyValue[]{kvA, kvB}); + + byte [] rb = Writables.getBytes(result); + Result deResult = (Result)Writables.getWritable(rb, new Result()); + + assertTrue("results are not equivalent, first key mismatch", + result.sorted()[0].equals(deResult.sorted()[0])); + + assertTrue("results are not equivalent, second key mismatch", + result.sorted()[1].equals(deResult.sorted()[1])); + + // Test empty Result + Result r = new Result(); + byte [] b = Writables.getBytes(r); + Result deserialized = (Result)Writables.getWritable(b, new Result()); + assertEquals(r.size(), deserialized.size()); + } + + public void testResultArray() throws Exception { + byte [] rowA = Bytes.toBytes("rowA"); + byte [] famA = Bytes.toBytes("famA"); + byte [] qfA = Bytes.toBytes("qfA"); + byte [] valueA = Bytes.toBytes("valueA"); + + byte [] rowB = Bytes.toBytes("rowB"); + byte [] famB = Bytes.toBytes("famB"); + byte [] qfB = Bytes.toBytes("qfB"); + byte [] valueB = Bytes.toBytes("valueB"); + + KeyValue kvA = new KeyValue(rowA, famA, qfA, valueA); + KeyValue kvB = new KeyValue(rowB, famB, qfB, valueB); + + + Result result1 = new Result(new KeyValue[]{kvA, kvB}); + Result result2 = new Result(new KeyValue[]{kvB}); + Result result3 = new Result(new KeyValue[]{kvB}); + + Result [] results = new Result [] {result1, result2, result3}; + + ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(byteStream); + Result.writeArray(out, results); + + byte [] rb = byteStream.toByteArray(); + + DataInputBuffer in = new DataInputBuffer(); + in.reset(rb, 0, rb.length); + + Result [] deResults = Result.readArray(in); + + assertTrue(results.length == deResults.length); + + for(int i=0;i keys = new ArrayList(); + Result r = new Result(keys); + Result [] results = new Result [] {r}; + + ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(byteStream); + + Result.writeArray(out, results); + + results = null; + + byteStream = new ByteArrayOutputStream(); + out = new DataOutputStream(byteStream); + Result.writeArray(out, results); + + byte [] rb = byteStream.toByteArray(); + + DataInputBuffer in = new DataInputBuffer(); + in.reset(rb, 0, rb.length); + + Result [] deResults = Result.readArray(in); + + assertTrue(deResults.length == 0); + + results = new Result[0]; + + byteStream = new ByteArrayOutputStream(); + out = new DataOutputStream(byteStream); + Result.writeArray(out, results); + + rb = byteStream.toByteArray(); + + in = new DataInputBuffer(); + in.reset(rb, 0, rb.length); + + deResults = Result.readArray(in); + + assertTrue(deResults.length == 0); + + } + + public void testTimeRange(String[] args) throws Exception{ + TimeRange tr = new TimeRange(0,5); + byte [] mb = Writables.getBytes(tr); + TimeRange deserializedTr = + (TimeRange)Writables.getWritable(mb, new TimeRange()); + + assertEquals(tr.getMax(), deserializedTr.getMax()); + assertEquals(tr.getMin(), deserializedTr.getMin()); + + } + + public void testKeyValue2() throws Exception { + byte[] row = getName().getBytes(); + byte[] fam = "fam".getBytes(); + byte[] qf = "qf".getBytes(); + long ts = System.currentTimeMillis(); + byte[] val = "val".getBytes(); + + KeyValue kv = new KeyValue(row, fam, qf, ts, val); + + byte [] mb = Writables.getBytes(kv); + KeyValue deserializedKv = + (KeyValue)Writables.getWritable(mb, new KeyValue()); + assertTrue(Bytes.equals(kv.getBuffer(), deserializedKv.getBuffer())); + assertEquals(kv.getOffset(), deserializedKv.getOffset()); + assertEquals(kv.getLength(), deserializedKv.getLength()); + } } \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/TestTable.java b/src/test/org/apache/hadoop/hbase/TestTable.java index 196c0691b7c..2100280fbda 100644 --- a/src/test/org/apache/hadoop/hbase/TestTable.java +++ b/src/test/org/apache/hadoop/hbase/TestTable.java @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.util.Bytes; @@ -57,7 +58,7 @@ public class TestTable extends HBaseClusterTestCase { // Try doing a duplicate database create. msg = null; HTableDescriptor desc = new HTableDescriptor(getName()); - desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY)); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); assertTrue("First table creation completed", admin.listTables().length == 1); boolean gotException = false; @@ -74,7 +75,7 @@ public class TestTable extends HBaseClusterTestCase { // Now try and do concurrent creation with a bunch of threads. final HTableDescriptor threadDesc = new HTableDescriptor("threaded_" + getName()); - threadDesc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY)); + threadDesc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); int count = 10; Thread [] threads = new Thread [count]; final AtomicInteger successes = new AtomicInteger(0); @@ -140,10 +141,12 @@ public class TestTable extends HBaseClusterTestCase { HTable table = new HTable(conf, getName()); try { byte[] value = Bytes.toBytes("somedata"); - BatchUpdate update = new BatchUpdate(); - update.put(colName, value); - table.commit(update); - fail("BatchUpdate on read only table succeeded"); + // This used to use an empty row... That must have been a bug + Put put = new Put(value); + byte [][] famAndQf = KeyValue.parseColumn(colName); + put.add(famAndQf[0], famAndQf[1], value); + table.put(put); + fail("Put on read only table succeeded"); } catch (Exception e) { // expected } diff --git a/src/test/org/apache/hadoop/hbase/TestZooKeeper.java b/src/test/org/apache/hadoop/hbase/TestZooKeeper.java index da49ebf784a..65beb2fc4bc 100644 --- a/src/test/org/apache/hadoop/hbase/TestZooKeeper.java +++ b/src/test/org/apache/hadoop/hbase/TestZooKeeper.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -141,9 +142,9 @@ public class TestZooKeeper extends HBaseClusterTestCase { admin.createTable(desc); HTable table = new HTable("test"); - BatchUpdate batchUpdate = new BatchUpdate("testrow"); - batchUpdate.put("fam:col", Bytes.toBytes("testdata")); - table.commit(batchUpdate); + Put put = new Put(Bytes.toBytes("testrow")); + put.add(Bytes.toBytes("fam"), Bytes.toBytes("col"), Bytes.toBytes("testdata")); + table.put(put); } catch (Exception e) { e.printStackTrace(); fail(); diff --git a/src/test/org/apache/hadoop/hbase/TimestampTestBase.java b/src/test/org/apache/hadoop/hbase/TimestampTestBase.java index 2ceca093aef..23612f6d4b0 100644 --- a/src/test/org/apache/hadoop/hbase/TimestampTestBase.java +++ b/src/test/org/apache/hadoop/hbase/TimestampTestBase.java @@ -19,8 +19,13 @@ package org.apache.hadoop.hbase; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; -import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.util.Bytes; @@ -34,7 +39,7 @@ public class TimestampTestBase extends HBaseTestCase { private static final long T1 = 100L; private static final long T2 = 200L; - private static final String COLUMN_NAME = "contents:"; + private static final String COLUMN_NAME = "contents:contents"; private static final byte [] COLUMN = Bytes.toBytes(COLUMN_NAME); private static final byte [] ROW = Bytes.toBytes("row"); @@ -55,6 +60,7 @@ public class TimestampTestBase extends HBaseTestCase { put(incommon); // Verify that returned versions match passed timestamps. assertVersions(incommon, new long [] {HConstants.LATEST_TIMESTAMP, T2, T1}); + // If I delete w/o specifying a timestamp, this means I'm deleting the // latest. delete(incommon); @@ -74,14 +80,23 @@ public class TimestampTestBase extends HBaseTestCase { // Flush everything out to disk and then retry flusher.flushcache(); assertVersions(incommon, new long [] {HConstants.LATEST_TIMESTAMP, T1, T0}); - + // Now try deleting all from T2 back inclusive (We first need to add T2 // back into the mix and to make things a little interesting, delete and // then readd T1. put(incommon, T2); delete(incommon, T1); put(incommon, T1); - incommon.deleteAll(ROW, COLUMN, T2); + + Delete delete = new Delete(ROW); + byte [][] famAndQf = KeyValue.parseColumn(COLUMN); + if (famAndQf[1].length == 0){ + delete.deleteFamily(famAndQf[0], T2); + } else { + delete.deleteColumns(famAndQf[0], famAndQf[1], T2); + } + incommon.delete(delete, null, true); + // Should only be current value in set. Assert this is so assertOnlyLatest(incommon, HConstants.LATEST_TIMESTAMP); @@ -93,12 +108,16 @@ public class TimestampTestBase extends HBaseTestCase { private static void assertOnlyLatest(final Incommon incommon, final long currentTime) throws IOException { - Cell [] cellValues = incommon.get(ROW, COLUMN, 3/*Ask for too much*/); - assertEquals(1, cellValues.length); - long time = Bytes.toLong(cellValues[0].getValue()); + Get get = null; + byte [][] famAndQf = null; + get = new Get(ROW); + famAndQf = KeyValue.parseColumn(COLUMN); + get.addColumn(famAndQf[0], famAndQf[1]); + get.setMaxVersions(3); + Result result = incommon.get(get); + assertEquals(1, result.size()); + long time = Bytes.toLong(result.sorted()[0].getValue()); assertEquals(time, currentTime); - assertNull(incommon.get(ROW, COLUMN, T1, 3 /*Too many*/)); - assertTrue(assertScanContentTimestamp(incommon, T1) == 0); } /* @@ -112,22 +131,49 @@ public class TimestampTestBase extends HBaseTestCase { public static void assertVersions(final Incommon incommon, final long [] tss) throws IOException { // Assert that 'latest' is what we expect. - byte [] bytes = incommon.get(ROW, COLUMN).getValue(); - assertEquals(Bytes.toLong(bytes), tss[0]); + Get get = null; + byte [][] famAndQf = null; + get = new Get(ROW); + famAndQf = KeyValue.parseColumn(COLUMN); + get.addColumn(famAndQf[0], famAndQf[1]); + Result r = incommon.get(get); + byte [] bytes = r.getValue(famAndQf[0], famAndQf[1]); + long t = Bytes.toLong(bytes); + assertEquals(tss[0], t); + // Now assert that if we ask for multiple versions, that they come out in // order. - Cell[] cellValues = incommon.get(ROW, COLUMN, tss.length); - assertEquals(tss.length, cellValues.length); - for (int i = 0; i < cellValues.length; i++) { - long ts = Bytes.toLong(cellValues[i].getValue()); + get = new Get(ROW); + famAndQf = KeyValue.parseColumn(COLUMN); + get.addColumn(famAndQf[0], famAndQf[1]); + get.setMaxVersions(tss.length); + Result result = incommon.get(get); + List cells = new ArrayList(); + for(KeyValue kv : result.sorted()) { + cells.add(new Cell(kv.getValue(), kv.getTimestamp())); + } + assertEquals(tss.length, cells.size()); + for (int i = 0; i < cells.size(); i++) { + long ts = Bytes.toLong(cells.get(i).getValue()); assertEquals(ts, tss[i]); } + // Specify a timestamp get multiple versions. - cellValues = incommon.get(ROW, COLUMN, tss[0], cellValues.length - 1); - for (int i = 1; i < cellValues.length; i++) { - long ts = Bytes.toLong(cellValues[i].getValue()); + get = new Get(ROW); + famAndQf = KeyValue.parseColumn(COLUMN); + get.addColumn(famAndQf[0], famAndQf[1]); + get.setTimeStamp(tss[0]); + get.setMaxVersions(cells.size() - 1); + result = incommon.get(get); + cells = new ArrayList(); + for(KeyValue kv : result.sorted()) { + cells.add(new Cell(kv.getValue(), kv.getTimestamp())); + } + for (int i = 1; i < cells.size(); i++) { + long ts = Bytes.toLong(cells.get(i).getValue()); assertEquals(ts, tss[i]); } + // Test scanner returns expected version assertScanContentTimestamp(incommon, tss[0]); } @@ -211,20 +257,44 @@ public class TimestampTestBase extends HBaseTestCase { public static void put(final Incommon loader, final byte [] bytes, final long ts) throws IOException { - BatchUpdate batchUpdate = ts == HConstants.LATEST_TIMESTAMP ? - new BatchUpdate(ROW) : new BatchUpdate(ROW, ts); - batchUpdate.put(COLUMN, bytes); - loader.commit(batchUpdate); + Put put = new Put(ROW); + if(ts != HConstants.LATEST_TIMESTAMP) { + put.setTimeStamp(ts); + } + byte [][] famAndQf = KeyValue.parseColumn(COLUMN); + put.add(famAndQf[0], famAndQf[1], bytes); + loader.put(put); } public static void delete(final Incommon loader) throws IOException { - delete(loader, HConstants.LATEST_TIMESTAMP); + delete(loader, null); } - public static void delete(final Incommon loader, final long ts) throws IOException { - BatchUpdate batchUpdate = ts == HConstants.LATEST_TIMESTAMP ? - new BatchUpdate(ROW) : new BatchUpdate(ROW, ts); - batchUpdate.delete(COLUMN); - loader.commit(batchUpdate); + public static void delete(final Incommon loader, final byte [] column) + throws IOException { + delete(loader, column, HConstants.LATEST_TIMESTAMP); } -} + + public static void delete(final Incommon loader, final long ts) + throws IOException { + delete(loader, null, ts); + } + + public static void delete(final Incommon loader, final byte [] column, + final long ts) + throws IOException { + Delete delete = ts == HConstants.LATEST_TIMESTAMP? + new Delete(ROW): new Delete(ROW, ts, null); + byte [][] famAndQf = KeyValue.parseColumn(column == null? COLUMN: column); + if (famAndQf[1].length == 0) { + delete.deleteFamily(famAndQf[0], ts); + } else { + delete.deleteColumn(famAndQf[0], famAndQf[1], ts); + } + loader.delete(delete, null, true); + } + + public static Result get(final Incommon loader) throws IOException { + return loader.get(new Get(ROW)); + } +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java b/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java index 461bdc3676b..724201f3cdc 100644 --- a/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java +++ b/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java @@ -75,85 +75,6 @@ public class TestBatchUpdate extends HBaseClusterTestCase { table = new HTable(conf, desc.getName()); } - /** - * @throws IOException - */ - public void testBatchUpdate() throws IOException { - BatchUpdate bu = new BatchUpdate("row1"); - bu.put(CONTENTS, value); - bu.delete(CONTENTS); - table.commit(bu); - - bu = new BatchUpdate("row2"); - bu.put(CONTENTS, value); - byte[][] getColumns = bu.getColumns(); - assertEquals(getColumns.length, 1); - assertTrue(Arrays.equals(getColumns[0], CONTENTS)); - assertTrue(bu.hasColumn(CONTENTS)); - assertFalse(bu.hasColumn(new byte[] {})); - byte[] getValue = bu.get(getColumns[0]); - assertTrue(Arrays.equals(getValue, value)); - table.commit(bu); - - byte [][] columns = { CONTENTS }; - Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); - for (RowResult r : scanner) { - for(Map.Entry e: r.entrySet()) { - System.out.println(Bytes.toString(r.getRow()) + ": row: " + e.getKey() + " value: " + - new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); - } - } - } - - public void testBatchUpdateMaxLength() { - // Test for a single good value - BatchUpdate batchUpdate = new BatchUpdate("row1"); - batchUpdate.put(SMALLFAM, value); - try { - table.commit(batchUpdate); - fail("Value is too long, should throw exception"); - } catch (IOException e) { - // This is expected - } - // Try to see if it's still inserted - try { - Cell cell = table.get("row1", SMALLFAM_STR); - assertNull(cell); - } catch (IOException e) { - e.printStackTrace(); - fail("This is unexpected"); - } - // Try to put a good value - batchUpdate = new BatchUpdate("row1"); - batchUpdate.put(SMALLFAM, smallValue); - try { - table.commit(batchUpdate); - } catch (IOException e) { - fail("Value is long enough, should not throw exception"); - } - } - - public void testRowsBatchUpdate() { - ArrayList rowsUpdate = new ArrayList(); - for(int i = 0; i < NB_BATCH_ROWS; i++) { - BatchUpdate batchUpdate = new BatchUpdate("row"+i); - batchUpdate.put(CONTENTS, value); - rowsUpdate.add(batchUpdate); - } - try { - table.commit(rowsUpdate); - - byte [][] columns = { CONTENTS }; - Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); - int nbRows = 0; - for(@SuppressWarnings("unused") RowResult row : scanner) - nbRows++; - assertEquals(NB_BATCH_ROWS, nbRows); - } catch (IOException e) { - fail("This is unexpected : " + e); - } - } - public void testRowsBatchUpdateBufferedOneFlush() { table.setAutoFlush(false); ArrayList rowsUpdate = new ArrayList(); @@ -168,17 +89,15 @@ public class TestBatchUpdate extends HBaseClusterTestCase { byte [][] columns = { CONTENTS }; Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); int nbRows = 0; - for(@SuppressWarnings("unused") RowResult row : scanner) - nbRows++; + for(@SuppressWarnings("unused") RowResult row : scanner) nbRows++; assertEquals(0, nbRows); scanner.close(); - + table.flushCommits(); scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); nbRows = 0; - for(@SuppressWarnings("unused") RowResult row : scanner) - nbRows++; + for(@SuppressWarnings("unused") RowResult row : scanner) nbRows++; assertEquals(NB_BATCH_ROWS*10, nbRows); } catch (IOException e) { fail("This is unexpected : " + e); @@ -209,6 +128,55 @@ public class TestBatchUpdate extends HBaseClusterTestCase { fail("This is unexpected : " + e); } } + + /** + * @throws IOException + */ + public void testBatchUpdate() throws IOException { + BatchUpdate bu = new BatchUpdate("row1"); + bu.put(CONTENTS, value); + // Can't do this in 0.20.0 mix and match put and delete -- bu.delete(CONTENTS); + table.commit(bu); + + bu = new BatchUpdate("row2"); + bu.put(CONTENTS, value); + byte[][] getColumns = bu.getColumns(); + assertEquals(getColumns.length, 1); + assertTrue(Arrays.equals(getColumns[0], CONTENTS)); + assertTrue(bu.hasColumn(CONTENTS)); + assertFalse(bu.hasColumn(new byte[] {})); + byte[] getValue = bu.get(getColumns[0]); + assertTrue(Arrays.equals(getValue, value)); + table.commit(bu); + + byte [][] columns = { CONTENTS }; + Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); + for (RowResult r : scanner) { + for(Map.Entry e: r.entrySet()) { + System.out.println(Bytes.toString(r.getRow()) + ": row: " + e.getKey() + " value: " + + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); + } + } + } - + public void testRowsBatchUpdate() { + ArrayList rowsUpdate = new ArrayList(); + for(int i = 0; i < NB_BATCH_ROWS; i++) { + BatchUpdate batchUpdate = new BatchUpdate("row"+i); + batchUpdate.put(CONTENTS, value); + rowsUpdate.add(batchUpdate); + } + try { + table.commit(rowsUpdate); + + byte [][] columns = { CONTENTS }; + Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); + int nbRows = 0; + for(@SuppressWarnings("unused") RowResult row : scanner) + nbRows++; + assertEquals(NB_BATCH_ROWS, nbRows); + } catch (IOException e) { + fail("This is unexpected : " + e); + } + } } diff --git a/src/test/org/apache/hadoop/hbase/client/TestClient.java b/src/test/org/apache/hadoop/hbase/client/TestClient.java new file mode 100644 index 00000000000..388b4508ba9 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/client/TestClient.java @@ -0,0 +1,2460 @@ +package org.apache.hadoop.hbase.client; + +import java.io.IOException; + +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.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + +public class TestClient extends HBaseClusterTestCase { + + private static byte [] ROW = Bytes.toBytes("testRow"); + private static byte [] FAMILY = Bytes.toBytes("testFamily"); + private static byte [] QUALIFIER = Bytes.toBytes("testQualifier"); + private static byte [] VALUE = Bytes.toBytes("testValue"); + + private static byte [] EMPTY = new byte[0]; + + /** + * Constructor does nothing special, start cluster. + */ + public TestClient() { + super(); + } + + public void XtestSuperSimple() throws Exception { + byte [] TABLE = Bytes.toBytes("testSuperSimple"); + HTable ht = createTable(TABLE, FAMILY); + Put put = new Put(ROW); + put.add(FAMILY, QUALIFIER, VALUE); + ht.put(put); + Scan scan = new Scan(); + scan.addColumn(FAMILY, TABLE); + ResultScanner scanner = ht.getScanner(scan); + Result result = scanner.next(); + assertTrue("Expected null result", result == null); + scanner.close(); + System.out.println("Done."); + } + + /** + * Test simple table and non-existent row cases. + */ + public void testSimpleMissing() throws Exception { + + byte [] TABLE = Bytes.toBytes("testSimpleMissing"); + + HTable ht = createTable(TABLE, FAMILY); + + byte [][] ROWS = makeN(ROW, 4); + + // Try to get a row on an empty table + + Get get = new Get(ROWS[0]); + Result result = ht.get(get); + assertEmptyResult(result); + + get = new Get(ROWS[0]); + get.addFamily(FAMILY); + result = ht.get(get); + assertEmptyResult(result); + + get = new Get(ROWS[0]); + get.addColumn(FAMILY, QUALIFIER); + result = ht.get(get); + assertEmptyResult(result); + + Scan scan = new Scan(); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + + scan = new Scan(ROWS[0]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan(ROWS[0],ROWS[1]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan(); + scan.addFamily(FAMILY); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan(); + scan.addColumn(FAMILY, QUALIFIER); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + // Insert a row + + Put put = new Put(ROWS[2]); + put.add(FAMILY, QUALIFIER, VALUE); + ht.put(put); + + // Try to get empty rows around it + + get = new Get(ROWS[1]); + result = ht.get(get); + assertEmptyResult(result); + + get = new Get(ROWS[0]); + get.addFamily(FAMILY); + result = ht.get(get); + assertEmptyResult(result); + + get = new Get(ROWS[3]); + get.addColumn(FAMILY, QUALIFIER); + result = ht.get(get); + assertEmptyResult(result); + + // Try to scan empty rows around it + + scan = new Scan(ROWS[3]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan(ROWS[0],ROWS[2]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + // Make sure we can actually get the row + + get = new Get(ROWS[2]); + result = ht.get(get); + assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); + + get = new Get(ROWS[2]); + get.addFamily(FAMILY); + result = ht.get(get); + assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); + + get = new Get(ROWS[2]); + get.addColumn(FAMILY, QUALIFIER); + result = ht.get(get); + assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); + + // Make sure we can scan the row + + scan = new Scan(); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); + + scan = new Scan(ROWS[0],ROWS[3]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); + + scan = new Scan(ROWS[2],ROWS[3]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE); + } + + /** + * Test basic puts, gets, scans, and deletes for a single row + * in a multiple family table. + */ + public void testSingleRowMultipleFamily() throws Exception { + + byte [] TABLE = Bytes.toBytes("testSingleRowMultipleFamily"); + + byte [][] ROWS = makeN(ROW, 3); + byte [][] FAMILIES = makeN(FAMILY, 10); + byte [][] QUALIFIERS = makeN(QUALIFIER, 10); + byte [][] VALUES = makeN(VALUE, 10); + + HTable ht = createTable(TABLE, FAMILIES); + + Get get; + Scan scan; + Delete delete; + Put put; + Result result; + + //////////////////////////////////////////////////////////////////////////// + // Insert one column to one family + //////////////////////////////////////////////////////////////////////////// + + put = new Put(ROWS[0]); + put.add(FAMILIES[4], QUALIFIERS[0], VALUES[0]); + ht.put(put); + + // Get the single column + getVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0); + + // Scan the single column + scanVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0); + + // Get empty results around inserted column + getVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0); + + // Scan empty results around inserted column + scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0); + + //////////////////////////////////////////////////////////////////////////// + // Flush memcache and run same tests from storefiles + //////////////////////////////////////////////////////////////////////////// + + flushMemcache(TABLE); + + // Redo get and scan tests from storefile + + getVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0); + scanVerifySingleColumn(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0, VALUES, 0); + getVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0); + scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0); + + //////////////////////////////////////////////////////////////////////////// + // Now, Test reading from memcache and storefiles at once + //////////////////////////////////////////////////////////////////////////// + + // Insert multiple columns to two other families + + put = new Put(ROWS[0]); + put.add(FAMILIES[2], QUALIFIERS[2], VALUES[2]); + put.add(FAMILIES[2], QUALIFIERS[4], VALUES[4]); + put.add(FAMILIES[4], QUALIFIERS[4], VALUES[4]); + put.add(FAMILIES[6], QUALIFIERS[6], VALUES[6]); + put.add(FAMILIES[6], QUALIFIERS[7], VALUES[7]); + put.add(FAMILIES[7], QUALIFIERS[7], VALUES[7]); + put.add(FAMILIES[9], QUALIFIERS[0], VALUES[0]); + ht.put(put); + + // Get multiple columns across multiple families and get empties around it + singleRowGetTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES); + + // Scan multiple columns across multiple families and scan empties around it + singleRowScanTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES); + + //////////////////////////////////////////////////////////////////////////// + // Flush the table again + //////////////////////////////////////////////////////////////////////////// + + flushMemcache(TABLE); + + // Redo tests again + + singleRowGetTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES); + singleRowScanTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES); + + // Insert more data to memcache + + put = new Put(ROWS[0]); + put.add(FAMILIES[6], QUALIFIERS[5], VALUES[5]); + put.add(FAMILIES[6], QUALIFIERS[8], VALUES[8]); + put.add(FAMILIES[6], QUALIFIERS[9], VALUES[9]); + put.add(FAMILIES[4], QUALIFIERS[3], VALUES[3]); + ht.put(put); + + //////////////////////////////////////////////////////////////////////////// + // Delete a storefile column + //////////////////////////////////////////////////////////////////////////// + delete = new Delete(ROWS[0]); + delete.deleteColumns(FAMILIES[6], QUALIFIERS[7]); + ht.delete(delete); + + // Try to get deleted column + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[6], QUALIFIERS[7]); + result = ht.get(get); + assertEmptyResult(result); + + // Try to scan deleted column + + scan = new Scan(); + scan.addColumn(FAMILIES[6], QUALIFIERS[7]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + // Make sure we can still get a column before it and after it + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[6], QUALIFIERS[6]); + result = ht.get(get); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[6], QUALIFIERS[8]); + result = ht.get(get); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]); + + // Make sure we can still scan a column before it and after it + + scan = new Scan(); + scan.addColumn(FAMILIES[6], QUALIFIERS[6]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); + + scan = new Scan(); + scan.addColumn(FAMILIES[6], QUALIFIERS[8]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]); + + //////////////////////////////////////////////////////////////////////////// + // Delete a memcache column + //////////////////////////////////////////////////////////////////////////// + delete = new Delete(ROWS[0]); + delete.deleteColumns(FAMILIES[6], QUALIFIERS[8]); + ht.delete(delete); + + // Try to get deleted column + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[6], QUALIFIERS[8]); + result = ht.get(get); + assertEmptyResult(result); + + // Try to scan deleted column + + scan = new Scan(); + scan.addColumn(FAMILIES[6], QUALIFIERS[8]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + // Make sure we can still get a column before it and after it + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[6], QUALIFIERS[6]); + result = ht.get(get); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[6], QUALIFIERS[9]); + result = ht.get(get); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); + + // Make sure we can still scan a column before it and after it + + scan = new Scan(); + scan.addColumn(FAMILIES[6], QUALIFIERS[6]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); + + scan = new Scan(); + scan.addColumn(FAMILIES[6], QUALIFIERS[9]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); + + //////////////////////////////////////////////////////////////////////////// + // Delete joint storefile/memcache family + //////////////////////////////////////////////////////////////////////////// + + delete = new Delete(ROWS[0]); + delete.deleteFamily(FAMILIES[4]); + ht.delete(delete); + + // Try to get storefile column in deleted family + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[4], QUALIFIERS[4]); + result = ht.get(get); + assertEmptyResult(result); + + // Try to get memcache column in deleted family + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[4], QUALIFIERS[3]); + result = ht.get(get); + assertEmptyResult(result); + + // Try to get deleted family + get = new Get(ROWS[0]); + get.addFamily(FAMILIES[4]); + result = ht.get(get); + assertEmptyResult(result); + + // Try to scan storefile column in deleted family + + scan = new Scan(); + scan.addColumn(FAMILIES[4], QUALIFIERS[4]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + // Try to scan memcache column in deleted family + scan = new Scan(); + scan.addColumn(FAMILIES[4], QUALIFIERS[3]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + // Try to scan deleted family + scan = new Scan(); + scan.addFamily(FAMILIES[4]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + // Make sure we can still get another family + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[2], QUALIFIERS[2]); + result = ht.get(get); + assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]); + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[6], QUALIFIERS[9]); + result = ht.get(get); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); + + // Make sure we can still scan another family + + scan = new Scan(); + scan.addColumn(FAMILIES[6], QUALIFIERS[6]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); + + scan = new Scan(); + scan.addColumn(FAMILIES[6], QUALIFIERS[9]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); + + //////////////////////////////////////////////////////////////////////////// + // Flush everything and rerun delete tests + //////////////////////////////////////////////////////////////////////////// + + flushMemcache(TABLE); + + // Try to get storefile column in deleted family + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[4], QUALIFIERS[4]); + result = ht.get(get); + assertEmptyResult(result); + + // Try to get memcache column in deleted family + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[4], QUALIFIERS[3]); + result = ht.get(get); + assertEmptyResult(result); + + // Try to get deleted family + get = new Get(ROWS[0]); + get.addFamily(FAMILIES[4]); + result = ht.get(get); + assertEmptyResult(result); + + // Try to scan storefile column in deleted family + + scan = new Scan(); + scan.addColumn(FAMILIES[4], QUALIFIERS[4]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + // Try to scan memcache column in deleted family + scan = new Scan(); + scan.addColumn(FAMILIES[4], QUALIFIERS[3]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + // Try to scan deleted family + scan = new Scan(); + scan.addFamily(FAMILIES[4]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + // Make sure we can still get another family + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[2], QUALIFIERS[2]); + result = ht.get(get); + assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]); + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[6], QUALIFIERS[9]); + result = ht.get(get); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); + + // Make sure we can still scan another family + + scan = new Scan(); + scan.addColumn(FAMILIES[6], QUALIFIERS[6]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]); + + scan = new Scan(); + scan.addColumn(FAMILIES[6], QUALIFIERS[9]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]); + + } + + @SuppressWarnings("unused") + public void testNull() throws Exception { + + byte [] TABLE = Bytes.toBytes("testNull"); + + // Null table name (should NOT work) + try { + HTable htFail = createTable(null, FAMILY); + assertTrue("Creating a table with null name passed, should have failed", + false); + } catch(Exception e) {} + + // Null family (should NOT work) + try { + HTable htFail = createTable(TABLE, (byte[])null); + assertTrue("Creating a table with a null family passed, should fail", + false); + } catch(Exception e) {} + + HTable ht = createTable(TABLE, FAMILY); + + // Null row (should NOT work) + try { + Put put = new Put((byte[])null); + put.add(FAMILY, QUALIFIER, VALUE); + ht.put(put); + assertTrue("Inserting a null row worked, should throw exception", + false); + } catch(Exception e) {} + + // Null qualifier (should work) + try { + + Put put = new Put(ROW); + put.add(FAMILY, null, VALUE); + ht.put(put); + + getTestNull(ht, ROW, FAMILY, VALUE); + + scanTestNull(ht, ROW, FAMILY, VALUE); + + Delete delete = new Delete(ROW); + delete.deleteColumns(FAMILY, null); + ht.delete(delete); + + Get get = new Get(ROW); + Result result = ht.get(get); + assertEmptyResult(result); + + } catch(Exception e) { + e.printStackTrace(); + assertTrue("Using a row with null qualifier threw exception, should " + + "pass", false); + } + + // Use a new table + + byte [] TABLE2 = Bytes.toBytes("testNull2"); + ht = createTable(TABLE2, FAMILY); + + // Empty qualifier, byte[0] instead of null (should work) + try { + + Put put = new Put(ROW); + put.add(FAMILY, EMPTY, VALUE); + ht.put(put); + + getTestNull(ht, ROW, FAMILY, VALUE); + + scanTestNull(ht, ROW, FAMILY, VALUE); + + // Flush and try again + + flushMemcache(TABLE2); + + getTestNull(ht, ROW, FAMILY, VALUE); + + scanTestNull(ht, ROW, FAMILY, VALUE); + + Delete delete = new Delete(ROW); + delete.deleteColumns(FAMILY, EMPTY); + ht.delete(delete); + + Get get = new Get(ROW); + Result result = ht.get(get); + assertEmptyResult(result); + + } catch(Exception e) { + e.printStackTrace(); + assertTrue("Using a row with null qualifier threw exception, should " + + "pass", false); + } + + // Null value + try { + + Put put = new Put(ROW); + put.add(FAMILY, QUALIFIER, null); + ht.put(put); + + Get get = new Get(ROW); + get.addColumn(FAMILY, QUALIFIER); + Result result = ht.get(get); + assertSingleResult(result, ROW, FAMILY, QUALIFIER, null); + + Scan scan = new Scan(); + scan.addColumn(FAMILY, QUALIFIER); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROW, FAMILY, QUALIFIER, null); + + Delete delete = new Delete(ROW); + delete.deleteColumns(FAMILY, QUALIFIER); + ht.delete(delete); + + get = new Get(ROW); + result = ht.get(get); + assertEmptyResult(result); + + } catch(Exception e) { + e.printStackTrace(); + assertTrue("Null values should be allowed, but threw exception", + false); + } + + } + + public void testVersions() throws Exception { + + byte [] TABLE = Bytes.toBytes("testSimpleVersions"); + + long [] STAMPS = makeStamps(20); + byte [][] VALUES = makeNAscii(VALUE, 20); + + HTable ht = createTable(TABLE, FAMILY, 10); + + // Insert 4 versions of same column + Put put = new Put(ROW); + put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); + put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); + put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); + put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); + ht.put(put); + + // Verify we can get each one properly + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); + + // Verify we don't accidentally get others + getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); + getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); + getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); + scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); + scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); + scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); + + // Ensure maxVersions in query is respected + Get get = new Get(ROW); + get.addColumn(FAMILY, QUALIFIER); + get.setMaxVersions(2); + Result result = ht.get(get); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[4], STAMPS[5]}, + new byte[][] {VALUES[4], VALUES[5]}, + 0, 1); + + Scan scan = new Scan(ROW); + scan.addColumn(FAMILY, QUALIFIER); + scan.setMaxVersions(2); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[4], STAMPS[5]}, + new byte[][] {VALUES[4], VALUES[5]}, + 0, 1); + + // Flush and redo + + flushMemcache(TABLE); + + // Verify we can get each one properly + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); + + // Verify we don't accidentally get others + getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); + getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); + getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); + scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); + scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[3]); + scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]); + + // Ensure maxVersions in query is respected + get = new Get(ROW); + get.addColumn(FAMILY, QUALIFIER); + get.setMaxVersions(2); + result = ht.get(get); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[4], STAMPS[5]}, + new byte[][] {VALUES[4], VALUES[5]}, + 0, 1); + + scan = new Scan(ROW); + scan.addColumn(FAMILY, QUALIFIER); + scan.setMaxVersions(2); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[4], STAMPS[5]}, + new byte[][] {VALUES[4], VALUES[5]}, + 0, 1); + + + // Add some memcache and retest + + // Insert 4 more versions of same column and a dupe + put = new Put(ROW); + put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]); + put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]); + put.add(FAMILY, QUALIFIER, STAMPS[7], VALUES[7]); + put.add(FAMILY, QUALIFIER, STAMPS[8], VALUES[8]); + ht.put(put); + + // Ensure maxVersions in query is respected + get = new Get(ROW); + get.addColumn(FAMILY, QUALIFIER); + get.setMaxVersions(); + result = ht.get(get); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]}, + new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]}, + 0, 7); + + scan = new Scan(ROW); + scan.addColumn(FAMILY, QUALIFIER); + scan.setMaxVersions(); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]}, + new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]}, + 0, 7); + + get = new Get(ROW); + get.setMaxVersions(); + result = ht.get(get); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]}, + new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]}, + 0, 7); + + scan = new Scan(ROW); + scan.setMaxVersions(); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]}, + new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]}, + 0, 7); + + // Verify we can get each one properly + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); + getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); + scanVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[7], VALUES[7]); + + // Verify we don't accidentally get others + getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); + getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]); + scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[0]); + scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[9]); + + // Ensure maxVersions of table is respected + + flushMemcache(TABLE); + + // Insert 4 more versions of same column and a dupe + put = new Put(ROW); + put.add(FAMILY, QUALIFIER, STAMPS[9], VALUES[9]); + put.add(FAMILY, QUALIFIER, STAMPS[11], VALUES[11]); + put.add(FAMILY, QUALIFIER, STAMPS[13], VALUES[13]); + put.add(FAMILY, QUALIFIER, STAMPS[15], VALUES[15]); + ht.put(put); + + get = new Get(ROW); + get.addColumn(FAMILY, QUALIFIER); + get.setMaxVersions(Integer.MAX_VALUE); + result = ht.get(get); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]}, + new byte[][] {VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]}, + 0, 9); + + scan = new Scan(ROW); + scan.addColumn(FAMILY, QUALIFIER); + scan.setMaxVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]}, + new byte[][] {VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]}, + 0, 9); + + // Delete a version in the memcache and a version in a storefile + Delete delete = new Delete(ROW); + delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[11]); + delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[7]); + ht.delete(delete); + + // Test that it's gone + get = new Get(ROW); + get.addColumn(FAMILY, QUALIFIER); + get.setMaxVersions(Integer.MAX_VALUE); + result = ht.get(get); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]}, + new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]}, + 0, 9); + + scan = new Scan(ROW); + scan.addColumn(FAMILY, QUALIFIER); + scan.setMaxVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILY, QUALIFIER, + new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]}, + new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]}, + 0, 9); + + } + + public void testVersionLimits() throws Exception { + byte [] TABLE = Bytes.toBytes("testVersionLimits"); + byte [][] FAMILIES = makeNAscii(FAMILY, 3); + int [] LIMITS = {1,3,5}; + long [] STAMPS = makeStamps(10); + byte [][] VALUES = makeNAscii(VALUE, 10); + HTable ht = createTable(TABLE, FAMILIES, LIMITS); + + // Insert limit + 1 on each family + Put put = new Put(ROW); + put.add(FAMILIES[0], QUALIFIER, STAMPS[0], VALUES[0]); + put.add(FAMILIES[0], QUALIFIER, STAMPS[1], VALUES[1]); + put.add(FAMILIES[1], QUALIFIER, STAMPS[0], VALUES[0]); + put.add(FAMILIES[1], QUALIFIER, STAMPS[1], VALUES[1]); + put.add(FAMILIES[1], QUALIFIER, STAMPS[2], VALUES[2]); + put.add(FAMILIES[1], QUALIFIER, STAMPS[3], VALUES[3]); + put.add(FAMILIES[2], QUALIFIER, STAMPS[0], VALUES[0]); + put.add(FAMILIES[2], QUALIFIER, STAMPS[1], VALUES[1]); + put.add(FAMILIES[2], QUALIFIER, STAMPS[2], VALUES[2]); + put.add(FAMILIES[2], QUALIFIER, STAMPS[3], VALUES[3]); + put.add(FAMILIES[2], QUALIFIER, STAMPS[4], VALUES[4]); + put.add(FAMILIES[2], QUALIFIER, STAMPS[5], VALUES[5]); + put.add(FAMILIES[2], QUALIFIER, STAMPS[6], VALUES[6]); + ht.put(put); + + // Verify we only get the right number out of each + + // Family0 + + Get get = new Get(ROW); + get.addColumn(FAMILIES[0], QUALIFIER); + get.setMaxVersions(Integer.MAX_VALUE); + Result result = ht.get(get); + assertNResult(result, ROW, FAMILIES[0], QUALIFIER, + new long [] {STAMPS[1]}, + new byte[][] {VALUES[1]}, + 0, 0); + + get = new Get(ROW); + get.addFamily(FAMILIES[0]); + get.setMaxVersions(Integer.MAX_VALUE); + result = ht.get(get); + assertNResult(result, ROW, FAMILIES[0], QUALIFIER, + new long [] {STAMPS[1]}, + new byte[][] {VALUES[1]}, + 0, 0); + + Scan scan = new Scan(ROW); + scan.addColumn(FAMILIES[0], QUALIFIER); + scan.setMaxVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILIES[0], QUALIFIER, + new long [] {STAMPS[1]}, + new byte[][] {VALUES[1]}, + 0, 0); + + scan = new Scan(ROW); + scan.addFamily(FAMILIES[0]); + scan.setMaxVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILIES[0], QUALIFIER, + new long [] {STAMPS[1]}, + new byte[][] {VALUES[1]}, + 0, 0); + + // Family1 + + get = new Get(ROW); + get.addColumn(FAMILIES[1], QUALIFIER); + get.setMaxVersions(Integer.MAX_VALUE); + result = ht.get(get); + assertNResult(result, ROW, FAMILIES[1], QUALIFIER, + new long [] {STAMPS[1], STAMPS[2], STAMPS[3]}, + new byte[][] {VALUES[1], VALUES[2], VALUES[3]}, + 0, 2); + + get = new Get(ROW); + get.addFamily(FAMILIES[1]); + get.setMaxVersions(Integer.MAX_VALUE); + result = ht.get(get); + assertNResult(result, ROW, FAMILIES[1], QUALIFIER, + new long [] {STAMPS[1], STAMPS[2], STAMPS[3]}, + new byte[][] {VALUES[1], VALUES[2], VALUES[3]}, + 0, 2); + + scan = new Scan(ROW); + scan.addColumn(FAMILIES[1], QUALIFIER); + scan.setMaxVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILIES[1], QUALIFIER, + new long [] {STAMPS[1], STAMPS[2], STAMPS[3]}, + new byte[][] {VALUES[1], VALUES[2], VALUES[3]}, + 0, 2); + + scan = new Scan(ROW); + scan.addFamily(FAMILIES[1]); + scan.setMaxVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILIES[1], QUALIFIER, + new long [] {STAMPS[1], STAMPS[2], STAMPS[3]}, + new byte[][] {VALUES[1], VALUES[2], VALUES[3]}, + 0, 2); + + // Family2 + + get = new Get(ROW); + get.addColumn(FAMILIES[2], QUALIFIER); + get.setMaxVersions(Integer.MAX_VALUE); + result = ht.get(get); + assertNResult(result, ROW, FAMILIES[2], QUALIFIER, + new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]}, + new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]}, + 0, 4); + + get = new Get(ROW); + get.addFamily(FAMILIES[2]); + get.setMaxVersions(Integer.MAX_VALUE); + result = ht.get(get); + assertNResult(result, ROW, FAMILIES[2], QUALIFIER, + new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]}, + new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]}, + 0, 4); + + scan = new Scan(ROW); + scan.addColumn(FAMILIES[2], QUALIFIER); + scan.setMaxVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILIES[2], QUALIFIER, + new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]}, + new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]}, + 0, 4); + + scan = new Scan(ROW); + scan.addFamily(FAMILIES[2]); + scan.setMaxVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROW, FAMILIES[2], QUALIFIER, + new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]}, + new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]}, + 0, 4); + + // Try all families + + get = new Get(ROW); + get.setMaxVersions(Integer.MAX_VALUE); + result = ht.get(get); + assertTrue("Expected 9 keys but received " + result.size(), + result.size() == 9); + + get = new Get(ROW); + get.addFamily(FAMILIES[0]); + get.addFamily(FAMILIES[1]); + get.addFamily(FAMILIES[2]); + get.setMaxVersions(Integer.MAX_VALUE); + result = ht.get(get); + assertTrue("Expected 9 keys but received " + result.size(), + result.size() == 9); + + get = new Get(ROW); + get.addColumn(FAMILIES[0], QUALIFIER); + get.addColumn(FAMILIES[1], QUALIFIER); + get.addColumn(FAMILIES[2], QUALIFIER); + get.setMaxVersions(Integer.MAX_VALUE); + result = ht.get(get); + assertTrue("Expected 9 keys but received " + result.size(), + result.size() == 9); + + scan = new Scan(ROW); + scan.setMaxVersions(Integer.MAX_VALUE); + result = getSingleScanResult(ht, scan); + assertTrue("Expected 9 keys but received " + result.size(), + result.size() == 9); + + scan = new Scan(ROW); + scan.setMaxVersions(Integer.MAX_VALUE); + scan.addFamily(FAMILIES[0]); + scan.addFamily(FAMILIES[1]); + scan.addFamily(FAMILIES[2]); + result = getSingleScanResult(ht, scan); + assertTrue("Expected 9 keys but received " + result.size(), + result.size() == 9); + + scan = new Scan(ROW); + scan.setMaxVersions(Integer.MAX_VALUE); + scan.addColumn(FAMILIES[0], QUALIFIER); + scan.addColumn(FAMILIES[1], QUALIFIER); + scan.addColumn(FAMILIES[2], QUALIFIER); + result = getSingleScanResult(ht, scan); + assertTrue("Expected 9 keys but received " + result.size(), + result.size() == 9); + + } + + public void XtestDeletes() throws Exception { + + // Test delete latest version + } + + /** + * Baseline "scalability" test. + * + * Tests one hundred families, one million columns, one million versions + */ + public void XtestMillions() throws Exception { + + // 100 families + + // millions of columns + + // millions of versions + + } + + public void XtestMultipleRegionsAndBatchPuts() throws Exception { + // Two family table + + // Insert lots of rows + + // Insert to the same row with batched puts + + // Insert to multiple rows with batched puts + + // Split the table + + // Get row from first region + + // Get row from second region + + // Scan all rows + + // Insert to multiple regions with batched puts + + // Get row from first region + + // Get row from second region + + // Scan all rows + + + } + + public void XtestMultipleRowMultipleFamily() throws Exception { + + } + + /** + * Explicitly test JIRAs related to HBASE-880 / Client API + */ + public void testJIRAs() throws Exception { + jiraTest867(); + jiraTest861(); + jiraTest33(); + jiraTest1014(); + jiraTest1182(); + jiraTest52(); + } + + // + // JIRA Testers + // + + /** + * HBASE-867 + * If millions of columns in a column family, hbase scanner won't come up + * + * Test will create numRows rows, each with numColsPerRow columns + * (1 version each), and attempt to scan them all. + * + * To test at scale, up numColsPerRow to the millions + * (have not gotten that to work running as junit though) + */ + private void jiraTest867() throws Exception { + + int numRows = 10; + int numColsPerRow = 2000; + + byte [] TABLE = Bytes.toBytes("jiraTest867"); + + byte [][] ROWS = makeN(ROW, numRows); + byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow); + + HTable ht = createTable(TABLE, FAMILY); + + // Insert rows + + for(int i=0;i some timestamp + */ + private void jiraTest1182() throws Exception { + + byte [] TABLE = Bytes.toBytes("jiraTest1182"); + byte [][] VALUES = makeNAscii(VALUE, 7); + long [] STAMPS = makeStamps(7); + + HTable ht = createTable(TABLE, FAMILY, 10); + + // Insert lots versions + + Put put = new Put(ROW); + put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]); + put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); + put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); + put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]); + put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); + put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); + ht.put(put); + + getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); + getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5); + getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); + + scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); + scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5); + scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); + + // Try same from storefile + flushMemcache(TABLE); + + getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); + getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5); + getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); + + scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); + scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5); + scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5); + + + } + + /** + * HBASE-52 + * Add a means of scanning over all versions + */ + private void jiraTest52() throws Exception { + + byte [] TABLE = Bytes.toBytes("jiraTest52"); + byte [][] VALUES = makeNAscii(VALUE, 7); + long [] STAMPS = makeStamps(7); + + HTable ht = createTable(TABLE, FAMILY, 10); + + // Insert lots versions + + Put put = new Put(ROW); + put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]); + put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]); + put.add(FAMILY, QUALIFIER, STAMPS[2], VALUES[2]); + put.add(FAMILY, QUALIFIER, STAMPS[3], VALUES[3]); + put.add(FAMILY, QUALIFIER, STAMPS[4], VALUES[4]); + put.add(FAMILY, QUALIFIER, STAMPS[5], VALUES[5]); + ht.put(put); + + getAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); + + scanAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); + + // Try same from storefile + flushMemcache(TABLE); + + getAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); + + scanAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5); + + + } + + // + // Bulk Testers + // + + private void getVersionRangeAndVerifyGreaterThan(HTable ht, byte [] row, + byte [] family, byte [] qualifier, long [] stamps, byte [][] values, + int start, int end) + throws IOException { + Get get = new Get(row); + get.addColumn(family, qualifier); + get.setMaxVersions(Integer.MAX_VALUE); + get.setTimeRange(stamps[start+1], Long.MAX_VALUE); + Result result = ht.get(get); + assertNResult(result, row, family, qualifier, stamps, values, start+1, end); + } + + private void getVersionRangeAndVerify(HTable ht, byte [] row, byte [] family, + byte [] qualifier, long [] stamps, byte [][] values, int start, int end) + throws IOException { + Get get = new Get(row); + get.addColumn(family, qualifier); + get.setMaxVersions(Integer.MAX_VALUE); + get.setTimeRange(stamps[start], stamps[end]+1); + Result result = ht.get(get); + assertNResult(result, row, family, qualifier, stamps, values, start, end); + } + + private void getAllVersionsAndVerify(HTable ht, byte [] row, byte [] family, + byte [] qualifier, long [] stamps, byte [][] values, int start, int end) + throws IOException { + Get get = new Get(row); + get.addColumn(family, qualifier); + get.setMaxVersions(Integer.MAX_VALUE); + Result result = ht.get(get); + assertNResult(result, row, family, qualifier, stamps, values, start, end); + } + + private void scanVersionRangeAndVerifyGreaterThan(HTable ht, byte [] row, + byte [] family, byte [] qualifier, long [] stamps, byte [][] values, + int start, int end) + throws IOException { + Scan scan = new Scan(row); + scan.addColumn(family, qualifier); + scan.setMaxVersions(Integer.MAX_VALUE); + scan.setTimeRange(stamps[start+1], Long.MAX_VALUE); + Result result = getSingleScanResult(ht, scan); + assertNResult(result, row, family, qualifier, stamps, values, start+1, end); + } + + private void scanVersionRangeAndVerify(HTable ht, byte [] row, byte [] family, + byte [] qualifier, long [] stamps, byte [][] values, int start, int end) + throws IOException { + Scan scan = new Scan(row); + scan.addColumn(family, qualifier); + scan.setMaxVersions(Integer.MAX_VALUE); + scan.setTimeRange(stamps[start], stamps[end]+1); + Result result = getSingleScanResult(ht, scan); + assertNResult(result, row, family, qualifier, stamps, values, start, end); + } + + private void scanAllVersionsAndVerify(HTable ht, byte [] row, byte [] family, + byte [] qualifier, long [] stamps, byte [][] values, int start, int end) + throws IOException { + Scan scan = new Scan(row); + scan.addColumn(family, qualifier); + scan.setMaxVersions(Integer.MAX_VALUE); + Result result = getSingleScanResult(ht, scan); + assertNResult(result, row, family, qualifier, stamps, values, start, end); + } + + private void getVersionAndVerify(HTable ht, byte [] row, byte [] family, + byte [] qualifier, long stamp, byte [] value) + throws Exception { + Get get = new Get(row); + get.addColumn(family, qualifier); + get.setTimeStamp(stamp); + get.setMaxVersions(Integer.MAX_VALUE); + Result result = ht.get(get); + assertSingleResult(result, row, family, qualifier, stamp, value); + } + + private void getVersionAndVerifyMissing(HTable ht, byte [] row, byte [] family, + byte [] qualifier, long stamp) + throws Exception { + Get get = new Get(row); + get.addColumn(family, qualifier); + get.setTimeStamp(stamp); + get.setMaxVersions(Integer.MAX_VALUE); + Result result = ht.get(get); + assertEmptyResult(result); + } + + private void scanVersionAndVerify(HTable ht, byte [] row, byte [] family, + byte [] qualifier, long stamp, byte [] value) + throws Exception { + Scan scan = new Scan(row); + scan.addColumn(family, qualifier); + scan.setTimeStamp(stamp); + scan.setMaxVersions(Integer.MAX_VALUE); + Result result = getSingleScanResult(ht, scan); + assertSingleResult(result, row, family, qualifier, stamp, value); + } + + private void scanVersionAndVerifyMissing(HTable ht, byte [] row, + byte [] family, byte [] qualifier, long stamp) + throws Exception { + Scan scan = new Scan(row); + scan.addColumn(family, qualifier); + scan.setTimeStamp(stamp); + scan.setMaxVersions(Integer.MAX_VALUE); + Result result = getSingleScanResult(ht, scan); + assertNullResult(result); + } + + private void getTestNull(HTable ht, byte [] row, byte [] family, + byte [] value) + throws Exception { + + Get get = new Get(row); + get.addColumn(family, null); + Result result = ht.get(get); + assertSingleResult(result, row, family, null, value); + + get = new Get(row); + get.addColumn(family, EMPTY); + result = ht.get(get); + assertSingleResult(result, row, family, EMPTY, value); + + get = new Get(row); + get.addFamily(family); + result = ht.get(get); + assertSingleResult(result, row, family, EMPTY, value); + + get = new Get(row); + result = ht.get(get); + assertSingleResult(result, row, family, EMPTY, value); + + } + + private void scanTestNull(HTable ht, byte [] row, byte [] family, + byte [] value) + throws Exception { + + Scan scan = new Scan(); + scan.addColumn(family, null); + Result result = getSingleScanResult(ht, scan); + assertSingleResult(result, row, family, EMPTY, value); + + scan = new Scan(); + scan.addColumn(family, EMPTY); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, row, family, EMPTY, value); + + scan = new Scan(); + scan.addFamily(family); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, row, family, EMPTY, value); + + scan = new Scan(); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, row, family, EMPTY, value); + + } + + private void singleRowGetTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES, + byte [][] QUALIFIERS, byte [][] VALUES) + throws Exception { + + // Single column from memcache + Get get = new Get(ROWS[0]); + get.addColumn(FAMILIES[4], QUALIFIERS[0]); + Result result = ht.get(get); + assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]); + + // Single column from storefile + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[2], QUALIFIERS[2]); + result = ht.get(get); + assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]); + + // Single column from storefile, family match + get = new Get(ROWS[0]); + get.addFamily(FAMILIES[7]); + result = ht.get(get); + assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]); + + // Two columns, one from memcache one from storefile, same family, + // wildcard match + get = new Get(ROWS[0]); + get.addFamily(FAMILIES[4]); + result = ht.get(get); + assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0], + FAMILIES[4], QUALIFIERS[4], VALUES[4]); + + // Two columns, one from memcache one from storefile, same family, + // explicit match + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[4], QUALIFIERS[0]); + get.addColumn(FAMILIES[4], QUALIFIERS[4]); + result = ht.get(get); + assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0], + FAMILIES[4], QUALIFIERS[4], VALUES[4]); + + // Three column, one from memcache two from storefile, different families, + // wildcard match + get = new Get(ROWS[0]); + get.addFamily(FAMILIES[4]); + get.addFamily(FAMILIES[7]); + result = ht.get(get); + assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, + new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} }); + + // Multiple columns from everywhere storefile, many family, wildcard + get = new Get(ROWS[0]); + get.addFamily(FAMILIES[2]); + get.addFamily(FAMILIES[4]); + get.addFamily(FAMILIES[6]); + get.addFamily(FAMILIES[7]); + result = ht.get(get); + assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, + new int [][] { + {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} + }); + + // Multiple columns from everywhere storefile, many family, wildcard + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[2], QUALIFIERS[2]); + get.addColumn(FAMILIES[2], QUALIFIERS[4]); + get.addColumn(FAMILIES[4], QUALIFIERS[0]); + get.addColumn(FAMILIES[4], QUALIFIERS[4]); + get.addColumn(FAMILIES[6], QUALIFIERS[6]); + get.addColumn(FAMILIES[6], QUALIFIERS[7]); + get.addColumn(FAMILIES[7], QUALIFIERS[7]); + get.addColumn(FAMILIES[7], QUALIFIERS[8]); + result = ht.get(get); + assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, + new int [][] { + {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} + }); + + // Everything + get = new Get(ROWS[0]); + result = ht.get(get); + assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, + new int [][] { + {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0} + }); + + // Get around inserted columns + + get = new Get(ROWS[1]); + result = ht.get(get); + assertEmptyResult(result); + + get = new Get(ROWS[0]); + get.addColumn(FAMILIES[4], QUALIFIERS[3]); + get.addColumn(FAMILIES[2], QUALIFIERS[3]); + result = ht.get(get); + assertEmptyResult(result); + + } + + private void singleRowScanTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES, + byte [][] QUALIFIERS, byte [][] VALUES) + throws Exception { + + // Single column from memcache + Scan scan = new Scan(); + scan.addColumn(FAMILIES[4], QUALIFIERS[0]); + Result result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0]); + + // Single column from storefile + scan = new Scan(); + scan.addColumn(FAMILIES[2], QUALIFIERS[2]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]); + + // Single column from storefile, family match + scan = new Scan(); + scan.addFamily(FAMILIES[7]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]); + + // Two columns, one from memcache one from storefile, same family, + // wildcard match + scan = new Scan(); + scan.addFamily(FAMILIES[4]); + result = getSingleScanResult(ht, scan); + assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0], + FAMILIES[4], QUALIFIERS[4], VALUES[4]); + + // Two columns, one from memcache one from storefile, same family, + // explicit match + scan = new Scan(); + scan.addColumn(FAMILIES[4], QUALIFIERS[0]); + scan.addColumn(FAMILIES[4], QUALIFIERS[4]); + result = getSingleScanResult(ht, scan); + assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0], + FAMILIES[4], QUALIFIERS[4], VALUES[4]); + + // Three column, one from memcache two from storefile, different families, + // wildcard match + scan = new Scan(); + scan.addFamily(FAMILIES[4]); + scan.addFamily(FAMILIES[7]); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, + new int [][] { {4, 0, 0}, {4, 4, 4}, {7, 7, 7} }); + + // Multiple columns from everywhere storefile, many family, wildcard + scan = new Scan(); + scan.addFamily(FAMILIES[2]); + scan.addFamily(FAMILIES[4]); + scan.addFamily(FAMILIES[6]); + scan.addFamily(FAMILIES[7]); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, + new int [][] { + {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} + }); + + // Multiple columns from everywhere storefile, many family, wildcard + scan = new Scan(); + scan.addColumn(FAMILIES[2], QUALIFIERS[2]); + scan.addColumn(FAMILIES[2], QUALIFIERS[4]); + scan.addColumn(FAMILIES[4], QUALIFIERS[0]); + scan.addColumn(FAMILIES[4], QUALIFIERS[4]); + scan.addColumn(FAMILIES[6], QUALIFIERS[6]); + scan.addColumn(FAMILIES[6], QUALIFIERS[7]); + scan.addColumn(FAMILIES[7], QUALIFIERS[7]); + scan.addColumn(FAMILIES[7], QUALIFIERS[8]); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, + new int [][] { + {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7} + }); + + // Everything + scan = new Scan(); + result = getSingleScanResult(ht, scan); + assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES, + new int [][] { + {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0} + }); + + // Scan around inserted columns + + scan = new Scan(ROWS[1]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan(); + scan.addColumn(FAMILIES[4], QUALIFIERS[3]); + scan.addColumn(FAMILIES[2], QUALIFIERS[3]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + } + + + + /** + * Verify a single column using gets. + * Expects family and qualifier arrays to be valid for at least + * the range: idx-2 < idx < idx+2 + */ + private void getVerifySingleColumn(HTable ht, + byte [][] ROWS, int ROWIDX, + byte [][] FAMILIES, int FAMILYIDX, + byte [][] QUALIFIERS, int QUALIFIERIDX, + byte [][] VALUES, int VALUEIDX) + throws Exception { + + Get get = new Get(ROWS[ROWIDX]); + Result result = ht.get(get); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + get = new Get(ROWS[ROWIDX]); + get.addFamily(FAMILIES[FAMILYIDX]); + result = ht.get(get); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + get = new Get(ROWS[ROWIDX]); + get.addFamily(FAMILIES[FAMILYIDX-2]); + get.addFamily(FAMILIES[FAMILYIDX]); + get.addFamily(FAMILIES[FAMILYIDX+2]); + result = ht.get(get); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + get = new Get(ROWS[ROWIDX]); + get.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[0]); + result = ht.get(get); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + get = new Get(ROWS[ROWIDX]); + get.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[1]); + get.addFamily(FAMILIES[FAMILYIDX]); + result = ht.get(get); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + get = new Get(ROWS[ROWIDX]); + get.addFamily(FAMILIES[FAMILYIDX]); + get.addColumn(FAMILIES[FAMILYIDX+1], QUALIFIERS[1]); + get.addColumn(FAMILIES[FAMILYIDX-2], QUALIFIERS[1]); + get.addFamily(FAMILIES[FAMILYIDX-1]); + get.addFamily(FAMILIES[FAMILYIDX+2]); + result = ht.get(get); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + } + + + /** + * Verify a single column using scanners. + * Expects family and qualifier arrays to be valid for at least + * the range: idx-2 to idx+2 + * Expects row array to be valid for at least idx to idx+2 + */ + private void scanVerifySingleColumn(HTable ht, + byte [][] ROWS, int ROWIDX, + byte [][] FAMILIES, int FAMILYIDX, + byte [][] QUALIFIERS, int QUALIFIERIDX, + byte [][] VALUES, int VALUEIDX) + throws Exception { + + Scan scan = new Scan(); + Result result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + scan = new Scan(ROWS[ROWIDX]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + scan = new Scan(ROWS[ROWIDX], ROWS[ROWIDX+1]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX+1]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + scan = new Scan(); + scan.addFamily(FAMILIES[FAMILYIDX]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + scan = new Scan(); + scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + scan = new Scan(); + scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]); + scan.addFamily(FAMILIES[FAMILYIDX]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + scan = new Scan(); + scan.addColumn(FAMILIES[FAMILYIDX-1], QUALIFIERS[QUALIFIERIDX+1]); + scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX]); + scan.addFamily(FAMILIES[FAMILYIDX+1]); + result = getSingleScanResult(ht, scan); + assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX], + QUALIFIERS[QUALIFIERIDX], VALUES[VALUEIDX]); + + } + + /** + * Verify we do not read any values by accident around a single column + * Same requirements as getVerifySingleColumn + */ + private void getVerifySingleEmpty(HTable ht, + byte [][] ROWS, int ROWIDX, + byte [][] FAMILIES, int FAMILYIDX, + byte [][] QUALIFIERS, int QUALIFIERIDX) + throws Exception { + + Get get = new Get(ROWS[ROWIDX]); + get.addFamily(FAMILIES[4]); + get.addColumn(FAMILIES[4], QUALIFIERS[1]); + Result result = ht.get(get); + assertEmptyResult(result); + + get = new Get(ROWS[ROWIDX]); + get.addFamily(FAMILIES[4]); + get.addColumn(FAMILIES[4], QUALIFIERS[2]); + result = ht.get(get); + assertEmptyResult(result); + + get = new Get(ROWS[ROWIDX]); + get.addFamily(FAMILIES[3]); + get.addColumn(FAMILIES[4], QUALIFIERS[2]); + get.addFamily(FAMILIES[5]); + result = ht.get(get); + assertEmptyResult(result); + + get = new Get(ROWS[ROWIDX+1]); + result = ht.get(get); + assertEmptyResult(result); + + } + + private void scanVerifySingleEmpty(HTable ht, + byte [][] ROWS, int ROWIDX, + byte [][] FAMILIES, int FAMILYIDX, + byte [][] QUALIFIERS, int QUALIFIERIDX) + throws Exception { + + Scan scan = new Scan(ROWS[ROWIDX+1]); + Result result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan(ROWS[ROWIDX+1],ROWS[ROWIDX+2]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan(HConstants.EMPTY_START_ROW, ROWS[ROWIDX]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + scan = new Scan(); + scan.addColumn(FAMILIES[FAMILYIDX], QUALIFIERS[QUALIFIERIDX+1]); + scan.addFamily(FAMILIES[FAMILYIDX-1]); + result = getSingleScanResult(ht, scan); + assertNullResult(result); + + } + + // + // Verifiers + // + + private void assertKey(KeyValue key, byte [] row, byte [] family, + byte [] qualifier, byte [] value) + throws Exception { + assertTrue("Expected row [" + Bytes.toString(row) + "] " + + "Got row [" + Bytes.toString(key.getRow()) +"]", + equals(row, key.getRow())); + assertTrue("Expected family [" + Bytes.toString(family) + "] " + + "Got family [" + Bytes.toString(key.getFamily()) + "]", + equals(family, key.getFamily())); + assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " + + "Got qualifier [" + Bytes.toString(key.getQualifier()) + "]", + equals(qualifier, key.getQualifier())); + assertTrue("Expected value [" + Bytes.toString(value) + "] " + + "Got value [" + Bytes.toString(key.getValue()) + "]", + equals(value, key.getValue())); + } + + private void assertNumKeys(Result result, int n) throws Exception { + assertTrue("Expected " + n + " keys but got " + result.size(), + result.size() == n); + } + + + private void assertNResult(Result result, byte [] row, + byte [][] families, byte [][] qualifiers, byte [][] values, + int [][] idxs) + throws Exception { + assertTrue("Expected row [" + Bytes.toString(row) + "] " + + "Got row [" + Bytes.toString(result.getRow()) +"]", + equals(row, result.getRow())); + assertTrue("Expected " + idxs.length + " keys but result contains " + + result.size(), result.size() == idxs.length); + + KeyValue [] keys = result.sorted(); + + for(int i=0;i 256) { + return makeNBig(base, n); + } + byte [][] ret = new byte[n][]; + for(int i=0;i 256) { + return makeNBig(base, n); + } + byte [][] ret = new byte[n][]; + for(int i=0;i> 8); + ret[i] = Bytes.add(base, new byte[]{(byte)byteB,(byte)byteA}); + } + return ret; + } + + private long [] makeStamps(int n) { + long [] stamps = new long[n]; + for(int i=0;i e: r.entrySet()) { - // Column name -// System.err.print(" " + Bytes.toString(e.getKey())); - c = e.getValue(); - - // Need to iterate since there may be multiple versions - for (Iterator> it = c.iterator(); - it.hasNext(); ) { - Map.Entry v = it.next(); - value = Bytes.toString(v.getValue()); -// System.err.println(" = " + value); - assertTrue(VALUE2.compareTo(Bytes.toString(v.getValue())) == 0); - } - } + get = new Get(ROW); + get.setMaxVersions(); + r = table.get(get); + assertTrue(r.size() == 2); + value = r.getValue(CONTENTS, CONTENTS); + assertTrue(value.length != 0); + assertTrue(Bytes.equals(value, VALUE2)); + NavigableMap>> map = + r.getMap(); + NavigableMap> familyMap = + map.get(CONTENTS); + NavigableMap versionMap = familyMap.get(CONTENTS); + assertTrue(versionMap.size() == 2); + assertTrue(Bytes.equals(VALUE1, versionMap.get(TIMESTAMP1))); + assertTrue(Bytes.equals(VALUE2, versionMap.get(TIMESTAMP2))); } } \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/client/TestHBaseAdmin.java b/src/test/org/apache/hadoop/hbase/client/TestHBaseAdmin.java new file mode 100644 index 00000000000..cbd6436311b --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/client/TestHBaseAdmin.java @@ -0,0 +1,323 @@ +package org.apache.hadoop.hbase.client; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseClusterTestCase; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.Bytes; + + +public class TestHBaseAdmin extends HBaseClusterTestCase { + static final Log LOG = LogFactory.getLog(TestHBaseAdmin.class.getName()); + + private String TABLE_STR = "testTable"; + private byte [] TABLE = Bytes.toBytes(TABLE_STR); + private byte [] ROW = Bytes.toBytes("testRow"); + private byte [] FAMILY = Bytes.toBytes("testFamily"); + private byte [] QUALIFIER = Bytes.toBytes("testQualifier"); + private byte [] VALUE = Bytes.toBytes("testValue"); + + private HBaseAdmin admin = null; + private HConnection connection = null; + + /** + * Constructor does nothing special, start cluster. + */ + public TestHBaseAdmin() throws Exception{ + super(); + } + + + public void testCreateTable() throws IOException { + init(); + + HTableDescriptor [] tables = connection.listTables(); + int numTables = tables.length; + + createTable(TABLE, FAMILY); + tables = connection.listTables(); + + assertEquals(numTables + 1, tables.length); + } + + + public void testDisableAndEnableTable() throws IOException { + init(); + + HTable ht = createTable(TABLE, FAMILY); + + Put put = new Put(ROW); + put.add(FAMILY, QUALIFIER, VALUE); + ht.put(put); + + admin.disableTable(TABLE); + + //Test that table is disabled + Get get = new Get(ROW); + get.addColumn(FAMILY, QUALIFIER); + boolean ok = false; + try { + ht.get(get); + } catch (RetriesExhaustedException e) { + ok = true; + } + assertEquals(true, ok); + + admin.enableTable(TABLE); + + //Test that table is enabled + try { + ht.get(get); + } catch (RetriesExhaustedException e) { + ok = false; + } + assertEquals(true, ok); + } + + + public void testTableExist() throws IOException { + init(); + boolean exist = false; + + exist = admin.tableExists(TABLE); + assertEquals(false, exist); + + createTable(TABLE, FAMILY); + + exist = admin.tableExists(TABLE); + assertEquals(true, exist); + } + + +// public void testMajorCompact() throws Exception { +// init(); +// +// int testTableCount = 0; +// int flushSleep = 1000; +// int majocCompactSleep = 7000; +// +// HTable ht = createTable(TABLE, FAMILY); +// byte [][] ROWS = makeN(ROW, 5); +// +// Put put = new Put(ROWS[0]); +// put.add(FAMILY, QUALIFIER, VALUE); +// ht.put(put); +// +// admin.flush(TABLE); +// Thread.sleep(flushSleep); +// +// put = new Put(ROWS[1]); +// put.add(FAMILY, QUALIFIER, VALUE); +// ht.put(put); +// +// admin.flush(TABLE); +// Thread.sleep(flushSleep); +// +// put = new Put(ROWS[2]); +// put.add(FAMILY, QUALIFIER, VALUE); +// ht.put(put); +// +// admin.flush(TABLE); +// Thread.sleep(flushSleep); +// +// put = new Put(ROWS[3]); +// put.add(FAMILY, QUALIFIER, VALUE); +// ht.put(put); +// +// admin.majorCompact(TABLE); +// Thread.sleep(majocCompactSleep); +// +// HRegion [] regions = null; +// +// regions = connection.getRegionServerWithRetries( +// new ServerCallable(connection, TABLE, ROW) { +// public HRegion [] call() throws IOException { +// return server.getOnlineRegionsAsArray(); +// } +// } +// ); +// for(HRegion region : regions) { +// String table = Bytes.toString(region.getRegionName()).split(",")[0]; +// if(table.equals(TABLE_STR)) { +// String output = "table: " + table; +// int i = 0; +// for(int j : region.getStoresSize()) { +// output += ", files in store " + i++ + "(" + j + ")"; +// testTableCount = j; +// } +// if (LOG.isDebugEnabled()) { +// LOG.debug(output); +// } +// System.out.println(output); +// } +// } +// assertEquals(1, testTableCount); +// } +// +// +// +// public void testFlush_TableName() throws Exception { +// init(); +// +// int initTestTableCount = 0; +// int testTableCount = 0; +// +// HTable ht = createTable(TABLE, FAMILY); +// +// Put put = new Put(ROW); +// put.add(FAMILY, QUALIFIER, VALUE); +// ht.put(put); +// +// HRegion [] regions = null; +// +// regions = connection.getRegionServerWithRetries( +// new ServerCallable(connection, TABLE, ROW) { +// public HRegion [] call() throws IOException { +// return server.getOnlineRegionsAsArray(); +// } +// } +// ); +// for(HRegion region : regions) { +// String table = Bytes.toString(region.getRegionName()).split(",")[0]; +// if(table.equals(TABLE_STR)) { +// String output = "table: " + table; +// int i = 0; +// for(int j : region.getStoresSize()) { +// output += ", files in store " + i++ + "(" + j + ")"; +// initTestTableCount = j; +// } +// if (LOG.isDebugEnabled()) { +// LOG.debug(output); +// } +// } +// } +// +// //Flushing +// admin.flush(TABLE); +// Thread.sleep(2000); +// +// regions = connection.getRegionServerWithRetries( +// new ServerCallable(connection, TABLE, ROW) { +// public HRegion [] call() throws IOException { +// return server.getOnlineRegionsAsArray(); +// } +// } +// ); +// for(HRegion region : regions) { +// String table = Bytes.toString(region.getRegionName()).split(",")[0]; +// if(table.equals(TABLE_STR)) { +// String output = "table: " + table; +// int i = 0; +// for(int j : region.getStoresSize()) { +// output += ", files in store " + i++ + "(" + j + ")"; +// testTableCount = j; +// } +// if (LOG.isDebugEnabled()) { +// LOG.debug(output); +// } +// } +// } +// +// assertEquals(initTestTableCount + 1, testTableCount); +// } +// +// +// public void testFlush_RegionName() throws Exception{ +// init(); +// int initTestTableCount = 0; +// int testTableCount = 0; +// String regionName = null; +// +// HTable ht = createTable(TABLE, FAMILY); +// +// Put put = new Put(ROW); +// put.add(FAMILY, QUALIFIER, VALUE); +// ht.put(put); +// +// HRegion [] regions = null; +// +// regions = connection.getRegionServerWithRetries( +// new ServerCallable(connection, TABLE, ROW) { +// public HRegion [] call() throws IOException { +// return server.getOnlineRegionsAsArray(); +// } +// } +// ); +// for(HRegion region : regions) { +// String reg = Bytes.toString(region.getRegionName()); +// String table = reg.split(",")[0]; +// if(table.equals(TABLE_STR)) { +// regionName = reg; +// String output = "table: " + table; +// int i = 0; +// for(int j : region.getStoresSize()) { +// output += ", files in store " + i++ + "(" + j + ")"; +// initTestTableCount = j; +// } +// if (LOG.isDebugEnabled()) { +// LOG.debug(output); +// } +// } +// } +// +// //Flushing +// admin.flush(regionName); +// Thread.sleep(2000); +// +// regions = connection.getRegionServerWithRetries( +// new ServerCallable(connection, TABLE, ROW) { +// public HRegion [] call() throws IOException { +// return server.getOnlineRegionsAsArray(); +// } +// } +// ); +// for(HRegion region : regions) { +// String table = Bytes.toString(region.getRegionName()).split(",")[0]; +// if(table.equals(TABLE_STR)) { +// String output = "table: " + table; +// int i = 0; +// for(int j : region.getStoresSize()) { +// output += ", files in store " + i++ + "(" + j + ")"; +// testTableCount = j; +// } +// if (LOG.isDebugEnabled()) { +// LOG.debug(output); +// } +// } +// } +// +// assertEquals(initTestTableCount + 1, testTableCount); +// } + + ////////////////////////////////////////////////////////////////////////////// + // Helpers + ////////////////////////////////////////////////////////////////////////////// + private byte [][] makeN(byte [] base, int n) { + byte [][] ret = new byte[n][]; + for(int i=0;i e: r.entrySet()) { - batchUpdate.put(e.getKey(), e.getValue().getValue()); + for (Result r : s) { + put = new Put(r.getRow()); + for(KeyValue kv : r.sorted()) { + put.add(kv); } - b.commit(batchUpdate); + b.put(put); } } finally { s.close(); @@ -168,7 +203,9 @@ public class TestHTable extends HBaseClusterTestCase implements HConstants { try { HTable anotherA = new HTable(conf, tableAname); - anotherA.get(row, COLUMN_FAMILY); + Get get = new Get(row); + get.addFamily(CATALOG_FAMILY); + anotherA.get(get); } catch (Exception e) { e.printStackTrace(); fail(); @@ -191,7 +228,7 @@ public class TestHTable extends HBaseClusterTestCase implements HConstants { for (HColumnDescriptor c: desc.getFamilies()) c.setValue(attrName, attrValue); // update metadata for all regions of this table - admin.modifyTable(tableAname, HConstants.MODIFY_TABLE_SET_HTD, desc); + admin.modifyTable(tableAname, HConstants.Modify.TABLE_SET_HTD, desc); // enable the table admin.enableTable(tableAname); @@ -220,144 +257,6 @@ public class TestHTable extends HBaseClusterTestCase implements HConstants { } } - public void testCheckAndSave() throws IOException { - HTable table = null; - HColumnDescriptor column2 = - new HColumnDescriptor(Bytes.toBytes("info2:")); - HBaseAdmin admin = new HBaseAdmin(conf); - HTableDescriptor testTableADesc = - new HTableDescriptor(tableAname); - testTableADesc.addFamily(column); - testTableADesc.addFamily(column2); - admin.createTable(testTableADesc); - - table = new HTable(conf, tableAname); - BatchUpdate batchUpdate = new BatchUpdate(row); - BatchUpdate batchUpdate2 = new BatchUpdate(row); - BatchUpdate batchUpdate3 = new BatchUpdate(row); - - // this row doesn't exist when checkAndSave is invoked - byte [] row1 = Bytes.toBytes("row1"); - BatchUpdate batchUpdate4 = new BatchUpdate(row1); - - // to be used for a checkAndSave for expected empty columns - BatchUpdate batchUpdate5 = new BatchUpdate(row); - - HbaseMapWritable expectedValues = - new HbaseMapWritable(); - HbaseMapWritable badExpectedValues = - new HbaseMapWritable(); - HbaseMapWritable expectedNoValues = - new HbaseMapWritable(); - // the columns used here must not be updated on batchupate - HbaseMapWritable expectedNoValues1 = - new HbaseMapWritable(); - - for(int i = 0; i < 5; i++) { - // This batchupdate is our initial batch update, - // As such we also set our expected values to the same values - // since we will be comparing the two - batchUpdate.put(COLUMN_FAMILY_STR+i, Bytes.toBytes(i)); - expectedValues.put(Bytes.toBytes(COLUMN_FAMILY_STR+i), Bytes.toBytes(i)); - - badExpectedValues.put(Bytes.toBytes(COLUMN_FAMILY_STR+i), - Bytes.toBytes(500)); - - expectedNoValues.put(Bytes.toBytes(COLUMN_FAMILY_STR+i), new byte[] {}); - // the columns used here must not be updated on batchupate - expectedNoValues1.put(Bytes.toBytes(COLUMN_FAMILY_STR+i+","+i), new byte[] {}); - - - // This is our second batchupdate that we will use to update the initial - // batchupdate - batchUpdate2.put(COLUMN_FAMILY_STR+i, Bytes.toBytes(i+1)); - - // This final batch update is to check that our expected values (which - // are now wrong) - batchUpdate3.put(COLUMN_FAMILY_STR+i, Bytes.toBytes(i+2)); - - // Batch update that will not happen because it is to happen with some - // expected values, but the row doesn't exist - batchUpdate4.put(COLUMN_FAMILY_STR+i, Bytes.toBytes(i)); - - // Batch update will happen: the row exists, but the expected columns don't, - // just as the condition - batchUpdate5.put(COLUMN_FAMILY_STR+i, Bytes.toBytes(i+3)); - } - - // Initialize rows - table.commit(batchUpdate); - - // check if incorrect values are returned false - assertFalse(table.checkAndSave(batchUpdate2,badExpectedValues,null)); - - // make sure first expected values are correct - assertTrue(table.checkAndSave(batchUpdate2, expectedValues,null)); - - // make sure check and save truly saves the data after checking the expected - // values - RowResult r = table.getRow(row); - byte[][] columns = batchUpdate2.getColumns(); - for(int i = 0;i < columns.length;i++) { - assertTrue(Bytes.equals(r.get(columns[i]).getValue(),batchUpdate2.get(columns[i]))); - } - - // make sure that the old expected values fail - assertFalse(table.checkAndSave(batchUpdate3, expectedValues,null)); - - // row doesn't exist, so doesn't matter the expected - // values (unless they are empty) - assertFalse(table.checkAndSave(batchUpdate4, badExpectedValues, null)); - - assertTrue(table.checkAndSave(batchUpdate4, expectedNoValues, null)); - // make sure check and save saves the data when expected values were empty and the row - // didn't exist - r = table.getRow(row1); - columns = batchUpdate4.getColumns(); - for(int i = 0; i < columns.length;i++) { - assertTrue(Bytes.equals(r.get(columns[i]).getValue(),batchUpdate4.get(columns[i]))); - } - - // since the row isn't empty anymore, those expected (empty) values - // are not valid anymore, so check and save method doesn't save. - assertFalse(table.checkAndSave(batchUpdate4, expectedNoValues, null)); - - // the row exists, but the columns don't. since the expected values are - // for columns without value, checkAndSave must be successful. - assertTrue(table.checkAndSave(batchUpdate5, expectedNoValues1, null)); - // make sure checkAndSave saved values for batchUpdate5. - r = table.getRow(row); - columns = batchUpdate5.getColumns(); - for(int i = 0; i < columns.length;i++) { - assertTrue(Bytes.equals(r.get(columns[i]).getValue(),batchUpdate5.get(columns[i]))); - } - - // since the condition wasn't changed, the following checkAndSave - // must also be successful. - assertTrue(table.checkAndSave(batchUpdate, expectedNoValues1, null)); - // make sure checkAndSave saved values for batchUpdate1 - r = table.getRow(row); - columns = batchUpdate.getColumns(); - for(int i = 0; i < columns.length;i++) { - assertTrue(Bytes.equals(r.get(columns[i]).getValue(),batchUpdate.get(columns[i]))); - } - - // one failing condition must make the following checkAndSave fail - // the failing condition is a column to be empty, however, it has a value. - HbaseMapWritable expectedValues1 = - new HbaseMapWritable(); - expectedValues1.put(Bytes.toBytes(COLUMN_FAMILY_STR+0), new byte[] {}); - expectedValues1.put(Bytes.toBytes(COLUMN_FAMILY_STR+"EMPTY+ROW"), new byte[] {}); - assertFalse(table.checkAndSave(batchUpdate5, expectedValues1, null)); - - // assure the values on the row remain the same - r = table.getRow(row); - columns = batchUpdate.getColumns(); - for(int i = 0; i < columns.length;i++) { - assertTrue(Bytes.equals(r.get(columns[i]).getValue(),batchUpdate.get(columns[i]))); - } - } - /** * For HADOOP-2579 */ @@ -389,43 +288,42 @@ public class TestHTable extends HBaseClusterTestCase implements HConstants { byte[] beforeSecondRow = Bytes.toBytes("rov"); HTable table = new HTable(conf, tableAname); - BatchUpdate batchUpdate = new BatchUpdate(firstRow); - BatchUpdate batchUpdate2 = new BatchUpdate(row); + Put put = new Put(firstRow); + Put put2 = new Put(row); byte[] zero = new byte[]{0}; byte[] one = new byte[]{1}; - byte[] columnFamilyBytes = Bytes.toBytes(COLUMN_FAMILY_STR); - batchUpdate.put(COLUMN_FAMILY_STR,zero); - batchUpdate2.put(COLUMN_FAMILY_STR,one); + put.add(CATALOG_FAMILY, null, zero); + put2.add(CATALOG_FAMILY, null, one); - table.commit(batchUpdate); - table.commit(batchUpdate2); + table.put(put); + table.put(put2); - RowResult result = null; + Result result = null; // Test before first that null is returned - result = table.getClosestRowBefore(beforeFirstRow, columnFamilyBytes); + result = table.getRowOrBefore(beforeFirstRow, CATALOG_FAMILY); assertTrue(result == null); // Test at first that first is returned - result = table.getClosestRowBefore(firstRow, columnFamilyBytes); - assertTrue(result.containsKey(COLUMN_FAMILY_STR)); - assertTrue(Bytes.equals(result.get(COLUMN_FAMILY_STR).getValue(), zero)); + result = table.getRowOrBefore(firstRow, CATALOG_FAMILY); + assertTrue(result.containsColumn(CATALOG_FAMILY, null)); + assertTrue(Bytes.equals(result.getValue(CATALOG_FAMILY, null), zero)); - // Test inbetween first and second that first is returned - result = table.getClosestRowBefore(beforeSecondRow, columnFamilyBytes); - assertTrue(result.containsKey(COLUMN_FAMILY_STR)); - assertTrue(Bytes.equals(result.get(COLUMN_FAMILY_STR).getValue(), zero)); + // Test in between first and second that first is returned + result = table.getRowOrBefore(beforeSecondRow, CATALOG_FAMILY); + assertTrue(result.containsColumn(CATALOG_FAMILY, null)); + assertTrue(Bytes.equals(result.getValue(CATALOG_FAMILY, null), zero)); // Test at second make sure second is returned - result = table.getClosestRowBefore(row, columnFamilyBytes); - assertTrue(result.containsKey(COLUMN_FAMILY_STR)); - assertTrue(Bytes.equals(result.get(COLUMN_FAMILY_STR).getValue(), one)); + result = table.getRowOrBefore(row, CATALOG_FAMILY); + assertTrue(result.containsColumn(CATALOG_FAMILY, null)); + assertTrue(Bytes.equals(result.getValue(CATALOG_FAMILY, null), one)); // Test after second, make sure second is returned - result = table.getClosestRowBefore(Bytes.add(row,one), columnFamilyBytes); - assertTrue(result.containsKey(COLUMN_FAMILY_STR)); - assertTrue(Bytes.equals(result.get(COLUMN_FAMILY_STR).getValue(), one)); + result = table.getRowOrBefore(Bytes.add(row,one), CATALOG_FAMILY); + assertTrue(result.containsColumn(CATALOG_FAMILY, null)); + assertTrue(Bytes.equals(result.getValue(CATALOG_FAMILY, null), one)); } /** diff --git a/src/test/org/apache/hadoop/hbase/client/TestListTables.java b/src/test/org/apache/hadoop/hbase/client/TestListTables.java index 9da5ebe0b1c..e5f9e162184 100644 --- a/src/test/org/apache/hadoop/hbase/client/TestListTables.java +++ b/src/test/org/apache/hadoop/hbase/client/TestListTables.java @@ -44,7 +44,7 @@ public class TestListTables extends HBaseClusterTestCase { super.setUp(); admin = new HBaseAdmin(conf); HColumnDescriptor family = - new HColumnDescriptor(HConstants.COLUMN_FAMILY_STR); + new HColumnDescriptor(HConstants.CATALOG_FAMILY); for (int i = 0; i < TABLES.length; i++) { TABLES[i].addFamily(family); admin.createTable(TABLES[i]); diff --git a/src/test/org/apache/hadoop/hbase/client/TestOldAPIGetRowVersions.java b/src/test/org/apache/hadoop/hbase/client/TestOldAPIGetRowVersions.java new file mode 100644 index 00000000000..e7a8be10b7b --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/client/TestOldAPIGetRowVersions.java @@ -0,0 +1,107 @@ +/** + * 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.client; + +import java.util.Iterator; +import java.util.Map; + +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.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * + */ +public class TestOldAPIGetRowVersions extends HBaseClusterTestCase { + private static final Log LOG = LogFactory.getLog(TestGetRowVersions.class); + private static final String TABLE_NAME = "test"; + private static final String CONTENTS_STR = "contents:"; + private static final String ROW = "row"; + private static final String COLUMN = "contents:contents"; + private static final long TIMESTAMP = System.currentTimeMillis(); + private static final String VALUE1 = "value1"; + private static final String VALUE2 = "value2"; + private HBaseAdmin admin = null; + private HTable table = null; + + @Override + public void setUp() throws Exception { + super.setUp(); + HTableDescriptor desc = new HTableDescriptor(TABLE_NAME); + desc.addFamily(new HColumnDescriptor(CONTENTS_STR)); + this.admin = new HBaseAdmin(conf); + this.admin.createTable(desc); + this.table = new HTable(conf, TABLE_NAME); + } + + /** @throws Exception */ + public void testGetRowMultipleVersions() throws Exception { + BatchUpdate b = new BatchUpdate(ROW, TIMESTAMP); + b.put(COLUMN, Bytes.toBytes(VALUE1)); + this.table.commit(b); + /* Taking out this recycle of the mini cluster -- it don't work well + * Debug it if fails in TestGetRowVersion, not this old api version. + // Shut down and restart the HBase cluster + this.cluster.shutdown(); + this.zooKeeperCluster.shutdown(); + LOG.debug("HBase cluster shut down -- restarting"); + this.hBaseClusterSetup(); + */ + // Make a new connection + this.table = new HTable(conf, TABLE_NAME); + // Overwrite previous value + b = new BatchUpdate(ROW, TIMESTAMP); + b.put(COLUMN, Bytes.toBytes(VALUE2)); + this.table.commit(b); + // Now verify that getRow(row, column, latest) works + RowResult r = table.getRow(ROW); + assertNotNull(r); + assertTrue(r.size() != 0); + Cell c = r.get(COLUMN); + assertNotNull(c); + assertTrue(c.getValue().length != 0); + String value = Bytes.toString(c.getValue()); + assertTrue(value.compareTo(VALUE2) == 0); + // Now check getRow with multiple versions + r = table.getRow(ROW, HConstants.ALL_VERSIONS); + for (Map.Entry e: r.entrySet()) { + // Column name +// System.err.print(" " + Bytes.toString(e.getKey())); + c = e.getValue(); + + // Need to iterate since there may be multiple versions + for (Iterator> it = c.iterator(); + it.hasNext(); ) { + Map.Entry v = it.next(); + value = Bytes.toString(v.getValue()); +// System.err.println(" = " + value); + assertTrue(VALUE2.compareTo(Bytes.toString(v.getValue())) == 0); + } + } + } +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/client/TestOldAPIHTable.java b/src/test/org/apache/hadoop/hbase/client/TestOldAPIHTable.java new file mode 100644 index 00000000000..952266539ac --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/client/TestOldAPIHTable.java @@ -0,0 +1,459 @@ +/** + * 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.client; + +import java.io.IOException; +import java.util.Map; + +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.TableNotFoundException; +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.HbaseMapWritable; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Tests HTable + */ +public class TestOldAPIHTable extends HBaseClusterTestCase implements HConstants { + private static final String COLUMN_FAMILY_STR = "contents:"; + private static final byte [] COLUMN_FAMILY = Bytes.toBytes(COLUMN_FAMILY_STR); + private static final byte [][] COLUMN_FAMILY_ARRAY = {COLUMN_FAMILY}; + + private static final HColumnDescriptor column = + new HColumnDescriptor(COLUMN_FAMILY); + + private static final byte [] nosuchTable = Bytes.toBytes("nosuchTable"); + private static final byte [] tableAname = Bytes.toBytes("tableA"); + private static final byte [] tableBname = Bytes.toBytes("tableB"); + + private static final byte [] row = Bytes.toBytes("row"); + + private static final byte [] attrName = Bytes.toBytes("TESTATTR"); + private static final byte [] attrValue = Bytes.toBytes("somevalue"); + + + /** + * For HADOOP-2579 + */ + public void testTableNotFoundExceptionWithoutAnyTables() { + try { + new HTable(conf, "notATable"); + fail("Should have thrown a TableNotFoundException"); + } catch (TableNotFoundException e) { + // expected + } catch (IOException e) { + e.printStackTrace(); + fail("Should have thrown a TableNotFoundException instead of a " + + e.getClass()); + } + } + + public void testGetClosestRowBefore() throws IOException { + HColumnDescriptor column2 = + new HColumnDescriptor(Bytes.toBytes("info2:")); + HBaseAdmin admin = new HBaseAdmin(conf); + HTableDescriptor testTableADesc = + new HTableDescriptor(tableAname); + testTableADesc.addFamily(column); + testTableADesc.addFamily(column2); + admin.createTable(testTableADesc); + + byte[] firstRow = Bytes.toBytes("ro"); + byte[] beforeFirstRow = Bytes.toBytes("rn"); + byte[] beforeSecondRow = Bytes.toBytes("rov"); + + HTable table = new HTable(conf, tableAname); + BatchUpdate batchUpdate = new BatchUpdate(firstRow); + BatchUpdate batchUpdate2 = new BatchUpdate(row); + byte[] zero = new byte[]{0}; + byte[] one = new byte[]{1}; + byte[] columnFamilyBytes = Bytes.toBytes(COLUMN_FAMILY_STR); + + batchUpdate.put(COLUMN_FAMILY_STR,zero); + batchUpdate2.put(COLUMN_FAMILY_STR,one); + + table.commit(batchUpdate); + table.commit(batchUpdate2); + + RowResult result = null; + + // Test before first that null is returned + result = table.getClosestRowBefore(beforeFirstRow, columnFamilyBytes); + assertTrue(result == null); + + // Test at first that first is returned + result = table.getClosestRowBefore(firstRow, columnFamilyBytes); + assertTrue(result.containsKey(COLUMN_FAMILY_STR)); + assertTrue(Bytes.equals(result.get(COLUMN_FAMILY_STR).getValue(), zero)); + + // Test inbetween first and second that first is returned + result = table.getClosestRowBefore(beforeSecondRow, columnFamilyBytes); + assertTrue(result.containsKey(COLUMN_FAMILY_STR)); + assertTrue(Bytes.equals(result.get(COLUMN_FAMILY_STR).getValue(), zero)); + + // Test at second make sure second is returned + result = table.getClosestRowBefore(row, columnFamilyBytes); + assertTrue(result.containsKey(COLUMN_FAMILY_STR)); + assertTrue(Bytes.equals(result.get(COLUMN_FAMILY_STR).getValue(), one)); + + // Test after second, make sure second is returned + result = table.getClosestRowBefore(Bytes.add(row,one), columnFamilyBytes); + assertTrue(result.containsKey(COLUMN_FAMILY_STR)); + assertTrue(Bytes.equals(result.get(COLUMN_FAMILY_STR).getValue(), one)); + } + + /** + * For HADOOP-2579 + */ + public void testTableNotFoundExceptionWithATable() { + try { + HBaseAdmin admin = new HBaseAdmin(conf); + HTableDescriptor testTableADesc = + new HTableDescriptor("table"); + testTableADesc.addFamily(column); + admin.createTable(testTableADesc); + + // This should throw a TableNotFoundException, it has not been created + new HTable(conf, "notATable"); + + fail("Should have thrown a TableNotFoundException"); + } catch (TableNotFoundException e) { + // expected + } catch (IOException e) { + e.printStackTrace(); + fail("Should have thrown a TableNotFoundException instead of a " + + e.getClass()); + } + } + + public void testGetRow() { + HTable table = null; + try { + HColumnDescriptor column2 = + new HColumnDescriptor(Bytes.toBytes("info2:")); + HBaseAdmin admin = new HBaseAdmin(conf); + HTableDescriptor testTableADesc = + new HTableDescriptor(tableAname); + testTableADesc.addFamily(column); + testTableADesc.addFamily(column2); + admin.createTable(testTableADesc); + + table = new HTable(conf, tableAname); + BatchUpdate batchUpdate = new BatchUpdate(row); + + for(int i = 0; i < 5; i++) + batchUpdate.put(COLUMN_FAMILY_STR+i, Bytes.toBytes(i)); + + table.commit(batchUpdate); + + assertTrue(table.exists(row)); + for(int i = 0; i < 5; i++) + assertTrue(table.exists(row, Bytes.toBytes(COLUMN_FAMILY_STR+i))); + + RowResult result = null; + result = table.getRow(row, new byte[][] {COLUMN_FAMILY}); + for(int i = 0; i < 5; i++) + assertTrue(result.containsKey(Bytes.toBytes(COLUMN_FAMILY_STR+i))); + + result = table.getRow(row); + for(int i = 0; i < 5; i++) + assertTrue(result.containsKey(Bytes.toBytes(COLUMN_FAMILY_STR+i))); + + batchUpdate = new BatchUpdate(row); + batchUpdate.put("info2:a", Bytes.toBytes("a")); + table.commit(batchUpdate); + + result = table.getRow(row, new byte[][] { COLUMN_FAMILY, + Bytes.toBytes("info2:a") }); + for(int i = 0; i < 5; i++) + assertTrue(result.containsKey(Bytes.toBytes(COLUMN_FAMILY_STR+i))); + assertTrue(result.containsKey(Bytes.toBytes("info2:a"))); + } catch (IOException e) { + e.printStackTrace(); + fail("Should not have any exception " + + e.getClass()); + } + } + + /** + * the test + * @throws IOException + */ + public void testHTable() throws IOException { + byte[] value = "value".getBytes(UTF8_ENCODING); + + try { + new HTable(conf, nosuchTable); + + } catch (TableNotFoundException e) { + // expected + + } catch (IOException e) { + e.printStackTrace(); + fail(); + } + + HTableDescriptor tableAdesc = new HTableDescriptor(tableAname); + tableAdesc.addFamily(column); + + HTableDescriptor tableBdesc = new HTableDescriptor(tableBname); + tableBdesc.addFamily(column); + + // create a couple of tables + + HBaseAdmin admin = new HBaseAdmin(conf); + admin.createTable(tableAdesc); + admin.createTable(tableBdesc); + + // put some data into table A + + HTable a = new HTable(conf, tableAname); + + // Assert the metadata is good. + HTableDescriptor meta = + a.getConnection().getHTableDescriptor(tableAdesc.getName()); + assertTrue(meta.equals(tableAdesc)); + + BatchUpdate batchUpdate = new BatchUpdate(row); + batchUpdate.put(COLUMN_FAMILY, value); + a.commit(batchUpdate); + + // open a new connection to A and a connection to b + + HTable newA = new HTable(conf, tableAname); + HTable b = new HTable(conf, tableBname); + + // copy data from A to B + + Scanner s = + newA.getScanner(COLUMN_FAMILY_ARRAY, EMPTY_START_ROW); + + try { + for (RowResult r : s) { + batchUpdate = new BatchUpdate(r.getRow()); + for(Map.Entry e: r.entrySet()) { + batchUpdate.put(e.getKey(), e.getValue().getValue()); + } + b.commit(batchUpdate); + } + } finally { + s.close(); + } + + // Opening a new connection to A will cause the tables to be reloaded + + try { + HTable anotherA = new HTable(conf, tableAname); + anotherA.get(row, COLUMN_FAMILY); + } catch (Exception e) { + e.printStackTrace(); + fail(); + } + + // We can still access A through newA because it has the table information + // cached. And if it needs to recalibrate, that will cause the information + // to be reloaded. + + // Test user metadata + + try { + // make a modifiable descriptor + HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor()); + // offline the table + admin.disableTable(tableAname); + // add a user attribute to HTD + desc.setValue(attrName, attrValue); + // add a user attribute to HCD + for (HColumnDescriptor c: desc.getFamilies()) + c.setValue(attrName, attrValue); + // update metadata for all regions of this table + admin.modifyTable(tableAname, HConstants.Modify.TABLE_SET_HTD, desc); + // enable the table + admin.enableTable(tableAname); + + // test that attribute changes were applied + desc = a.getTableDescriptor(); + if (Bytes.compareTo(desc.getName(), tableAname) != 0) + fail("wrong table descriptor returned"); + // check HTD attribute + value = desc.getValue(attrName); + if (value == null) + fail("missing HTD attribute value"); + if (Bytes.compareTo(value, attrValue) != 0) + fail("HTD attribute value is incorrect"); + // check HCD attribute + for (HColumnDescriptor c: desc.getFamilies()) { + value = c.getValue(attrName); + if (value == null) + fail("missing HCD attribute value"); + if (Bytes.compareTo(value, attrValue) != 0) + fail("HCD attribute value is incorrect"); + } + + } catch (Exception e) { + e.printStackTrace(); + fail(); + } + } + +// public void testCheckAndSave() throws IOException { +// HTable table = null; +// HColumnDescriptor column2 = +// new HColumnDescriptor(Bytes.toBytes("info2:")); +// HBaseAdmin admin = new HBaseAdmin(conf); +// HTableDescriptor testTableADesc = +// new HTableDescriptor(tableAname); +// testTableADesc.addFamily(column); +// testTableADesc.addFamily(column2); +// admin.createTable(testTableADesc); +// +// table = new HTable(conf, tableAname); +// BatchUpdate batchUpdate = new BatchUpdate(row); +// BatchUpdate batchUpdate2 = new BatchUpdate(row); +// BatchUpdate batchUpdate3 = new BatchUpdate(row); +// +// // this row doesn't exist when checkAndSave is invoked +// byte [] row1 = Bytes.toBytes("row1"); +// BatchUpdate batchUpdate4 = new BatchUpdate(row1); +// +// // to be used for a checkAndSave for expected empty columns +// BatchUpdate batchUpdate5 = new BatchUpdate(row); +// +// HbaseMapWritable expectedValues = +// new HbaseMapWritable(); +// HbaseMapWritable badExpectedValues = +// new HbaseMapWritable(); +// HbaseMapWritable expectedNoValues = +// new HbaseMapWritable(); +// // the columns used here must not be updated on batchupate +// HbaseMapWritable expectedNoValues1 = +// new HbaseMapWritable(); +// +// for(int i = 0; i < 5; i++) { +// // This batchupdate is our initial batch update, +// // As such we also set our expected values to the same values +// // since we will be comparing the two +// batchUpdate.put(COLUMN_FAMILY_STR+i, Bytes.toBytes(i)); +// expectedValues.put(Bytes.toBytes(COLUMN_FAMILY_STR+i), Bytes.toBytes(i)); +// +// badExpectedValues.put(Bytes.toBytes(COLUMN_FAMILY_STR+i), +// Bytes.toBytes(500)); +// +// expectedNoValues.put(Bytes.toBytes(COLUMN_FAMILY_STR+i), new byte[] {}); +// // the columns used here must not be updated on batchupate +// expectedNoValues1.put(Bytes.toBytes(COLUMN_FAMILY_STR+i+","+i), new byte[] {}); +// +// +// // This is our second batchupdate that we will use to update the initial +// // batchupdate +// batchUpdate2.put(COLUMN_FAMILY_STR+i, Bytes.toBytes(i+1)); +// +// // This final batch update is to check that our expected values (which +// // are now wrong) +// batchUpdate3.put(COLUMN_FAMILY_STR+i, Bytes.toBytes(i+2)); +// +// // Batch update that will not happen because it is to happen with some +// // expected values, but the row doesn't exist +// batchUpdate4.put(COLUMN_FAMILY_STR+i, Bytes.toBytes(i)); +// +// // Batch update will happen: the row exists, but the expected columns don't, +// // just as the condition +// batchUpdate5.put(COLUMN_FAMILY_STR+i, Bytes.toBytes(i+3)); +// } +// +// // Initialize rows +// table.commit(batchUpdate); +// +// // check if incorrect values are returned false +// assertFalse(table.checkAndSave(batchUpdate2,badExpectedValues,null)); +// +// // make sure first expected values are correct +// assertTrue(table.checkAndSave(batchUpdate2, expectedValues,null)); +// +// // make sure check and save truly saves the data after checking the expected +// // values +// RowResult r = table.getRow(row); +// byte[][] columns = batchUpdate2.getColumns(); +// for(int i = 0;i < columns.length;i++) { +// assertTrue(Bytes.equals(r.get(columns[i]).getValue(),batchUpdate2.get(columns[i]))); +// } +// +// // make sure that the old expected values fail +// assertFalse(table.checkAndSave(batchUpdate3, expectedValues,null)); +// +// // row doesn't exist, so doesn't matter the expected +// // values (unless they are empty) +// assertFalse(table.checkAndSave(batchUpdate4, badExpectedValues, null)); +// +// assertTrue(table.checkAndSave(batchUpdate4, expectedNoValues, null)); +// // make sure check and save saves the data when expected values were empty and the row +// // didn't exist +// r = table.getRow(row1); +// columns = batchUpdate4.getColumns(); +// for(int i = 0; i < columns.length;i++) { +// assertTrue(Bytes.equals(r.get(columns[i]).getValue(),batchUpdate4.get(columns[i]))); +// } +// +// // since the row isn't empty anymore, those expected (empty) values +// // are not valid anymore, so check and save method doesn't save. +// assertFalse(table.checkAndSave(batchUpdate4, expectedNoValues, null)); +// +// // the row exists, but the columns don't. since the expected values are +// // for columns without value, checkAndSave must be successful. +// assertTrue(table.checkAndSave(batchUpdate5, expectedNoValues1, null)); +// // make sure checkAndSave saved values for batchUpdate5. +// r = table.getRow(row); +// columns = batchUpdate5.getColumns(); +// for(int i = 0; i < columns.length;i++) { +// assertTrue(Bytes.equals(r.get(columns[i]).getValue(),batchUpdate5.get(columns[i]))); +// } +// +// // since the condition wasn't changed, the following checkAndSave +// // must also be successful. +// assertTrue(table.checkAndSave(batchUpdate, expectedNoValues1, null)); +// // make sure checkAndSave saved values for batchUpdate1 +// r = table.getRow(row); +// columns = batchUpdate.getColumns(); +// for(int i = 0; i < columns.length;i++) { +// assertTrue(Bytes.equals(r.get(columns[i]).getValue(),batchUpdate.get(columns[i]))); +// } +// +// // one failing condition must make the following checkAndSave fail +// // the failing condition is a column to be empty, however, it has a value. +// HbaseMapWritable expectedValues1 = +// new HbaseMapWritable(); +// expectedValues1.put(Bytes.toBytes(COLUMN_FAMILY_STR+0), new byte[] {}); +// expectedValues1.put(Bytes.toBytes(COLUMN_FAMILY_STR+"EMPTY+ROW"), new byte[] {}); +// assertFalse(table.checkAndSave(batchUpdate5, expectedValues1, null)); +// +// // assure the values on the row remain the same +// r = table.getRow(row); +// columns = batchUpdate.getColumns(); +// for(int i = 0; i < columns.length;i++) { +// assertTrue(Bytes.equals(r.get(columns[i]).getValue(),batchUpdate.get(columns[i]))); +// } +// } + +} diff --git a/src/test/org/apache/hadoop/hbase/client/TestOldAPITimestamp.java b/src/test/org/apache/hadoop/hbase/client/TestOldAPITimestamp.java new file mode 100644 index 00000000000..e9ed17f1218 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/client/TestOldAPITimestamp.java @@ -0,0 +1,71 @@ +/** + * 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.client; + +import java.io.IOException; + +import org.apache.hadoop.hbase.HBaseClusterTestCase; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TimestampTestBase; + +/** + * Tests user specifiable time stamps putting, getting and scanning. Also + * tests same in presence of deletes. Test cores are written so can be + * run against an HRegion and against an HTable: i.e. both local and remote. + */ +public class TestOldAPITimestamp extends HBaseClusterTestCase { + private static final String COLUMN_NAME = "contents:"; + + /** + * Basic test of timestamps. + * Do the above tests from client side. + * @throws IOException + */ + public void testTimestamps() throws IOException { + HTable t = createTable(); + Incommon incommon = new HTableIncommon(t); + TimestampTestBase.doTestDelete(incommon, new FlushCache() { + public void flushcache() throws IOException { + cluster.flushcache(); + } + }); + + // Perhaps drop and readd the table between tests so the former does + // not pollute this latter? Or put into separate tests. + TimestampTestBase.doTestTimestampScanning(incommon, new FlushCache() { + public void flushcache() throws IOException { + cluster.flushcache(); + } + }); + } + + /* + * Create a table named TABLE_NAME. + * @return An instance of an HTable connected to the created table. + * @throws IOException + */ + private HTable createTable() throws IOException { + HTableDescriptor desc = new HTableDescriptor(getName()); + desc.addFamily(new HColumnDescriptor(COLUMN_NAME)); + HBaseAdmin admin = new HBaseAdmin(conf); + admin.createTable(desc); + return new HTable(conf, getName()); + } +} diff --git a/src/test/org/apache/hadoop/hbase/client/TestPut.java b/src/test/org/apache/hadoop/hbase/client/TestPut.java new file mode 100644 index 00000000000..fbbad38a7a2 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/client/TestPut.java @@ -0,0 +1,202 @@ +/** + * 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.client; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Map; + +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.KeyValue; +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 puts + */ +public class TestPut extends HBaseClusterTestCase { + private static final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents"); + private static final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam"); + + private static final byte [] row1 = Bytes.toBytes("row1"); + private static final byte [] row2 = Bytes.toBytes("row2"); + + private static final int SMALL_LENGTH = 1; + private static final int NB_BATCH_ROWS = 10; + private byte [] value; + private byte [] smallValue; + + private HTableDescriptor desc = null; + private HTable table = null; + + /** + * @throws UnsupportedEncodingException + */ + public TestPut() throws UnsupportedEncodingException { + super(); + value = Bytes.toBytes("abcd"); + smallValue = Bytes.toBytes("a"); + } + + @Override + public void setUp() throws Exception { + super.setUp(); + this.desc = new HTableDescriptor("test"); + desc.addFamily(new HColumnDescriptor(CONTENTS_FAMILY)); + desc.addFamily(new HColumnDescriptor(SMALL_FAMILY, + HColumnDescriptor.DEFAULT_VERSIONS, + HColumnDescriptor.DEFAULT_COMPRESSION, + HColumnDescriptor.DEFAULT_IN_MEMORY, + HColumnDescriptor.DEFAULT_BLOCKCACHE, SMALL_LENGTH, + HColumnDescriptor.DEFAULT_TTL, HColumnDescriptor.DEFAULT_BLOOMFILTER)); + HBaseAdmin admin = new HBaseAdmin(conf); + admin.createTable(desc); + table = new HTable(conf, desc.getName()); + } + + /** + * @throws IOException + */ + public void testPut() throws IOException { + + Put put = new Put(row1); + put.add(CONTENTS_FAMILY, null, value); + table.put(put); + + put = new Put(row2); + put.add(CONTENTS_FAMILY, null, value); + + assertEquals(put.size(), 1); + assertEquals(put.getFamilyMap().get(CONTENTS_FAMILY).size(), 1); + + KeyValue kv = put.getFamilyMap().get(CONTENTS_FAMILY).get(0); + + assertTrue(Bytes.equals(kv.getFamily(), CONTENTS_FAMILY)); + // will it return null or an empty byte array? + assertTrue(Bytes.equals(kv.getQualifier(), new byte[0])); + + assertTrue(Bytes.equals(kv.getValue(), value)); + + table.put(put); + + Scan scan = new Scan(); + scan.addColumn(CONTENTS_FAMILY, null); + ResultScanner scanner = table.getScanner(scan); + for (Result r : scanner) { + for(KeyValue key : r.sorted()) { + System.out.println(Bytes.toString(r.getRow()) + ": " + key.toString()); + } + } + } + + public void testRowsPut() { + ArrayList rowsUpdate = new ArrayList(); + for(int i = 0; i < NB_BATCH_ROWS; i++) { + byte [] row = Bytes.toBytes("row" + i); + Put put = new Put(row); + put.add(CONTENTS_FAMILY, null, value); + rowsUpdate.add(put); + } + try { + table.put(rowsUpdate); + + Scan scan = new Scan(); + scan.addFamily(CONTENTS_FAMILY); + ResultScanner scanner = table.getScanner(scan); + int nbRows = 0; + for(@SuppressWarnings("unused") Result row : scanner) + nbRows++; + assertEquals(NB_BATCH_ROWS, nbRows); + } catch (IOException e) { + fail("This is unexpected : " + e); + } + } + + public void testRowsPutBufferedOneFlush() { + table.setAutoFlush(false); + ArrayList rowsUpdate = new ArrayList(); + for(int i = 0; i < NB_BATCH_ROWS*10; i++) { + byte [] row = Bytes.toBytes("row" + i); + Put put = new Put(row); + put.add(CONTENTS_FAMILY, null, value); + rowsUpdate.add(put); + } + try { + table.put(rowsUpdate); + + Scan scan = new Scan(); + scan.addFamily(CONTENTS_FAMILY); + ResultScanner scanner = table.getScanner(scan); + int nbRows = 0; + for(@SuppressWarnings("unused") Result row : scanner) + nbRows++; + assertEquals(0, nbRows); + scanner.close(); + + table.flushCommits(); + + scan = new Scan(); + scan.addFamily(CONTENTS_FAMILY); + scanner = table.getScanner(scan); + nbRows = 0; + for(@SuppressWarnings("unused") Result row : scanner) + nbRows++; + assertEquals(NB_BATCH_ROWS*10, nbRows); + } catch (IOException e) { + fail("This is unexpected : " + e); + } + } + + public void testRowsPutBufferedManyManyFlushes() { + table.setAutoFlush(false); + table.setWriteBufferSize(10); + ArrayList rowsUpdate = new ArrayList(); + for(int i = 0; i < NB_BATCH_ROWS*10; i++) { + byte [] row = Bytes.toBytes("row" + i); + Put put = new Put(row); + put.add(CONTENTS_FAMILY, null, value); + rowsUpdate.add(put); + } + try { + table.put(rowsUpdate); + + table.flushCommits(); + + Scan scan = new Scan(); + scan.addFamily(CONTENTS_FAMILY); + ResultScanner scanner = table.getScanner(scan); + int nbRows = 0; + for(@SuppressWarnings("unused") Result row : scanner) + nbRows++; + assertEquals(NB_BATCH_ROWS*10, nbRows); + } catch (IOException e) { + fail("This is unexpected : " + e); + } + } + + +} diff --git a/src/test/org/apache/hadoop/hbase/client/TestScannerTimes.java b/src/test/org/apache/hadoop/hbase/client/TestScannerTimes.java index 5f3407a9c8e..bb938de484f 100644 --- a/src/test/org/apache/hadoop/hbase/client/TestScannerTimes.java +++ b/src/test/org/apache/hadoop/hbase/client/TestScannerTimes.java @@ -26,9 +26,11 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HBaseClusterTestCase; +import org.apache.hadoop.hbase.KeyValue; 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 that verifies that scanners return a different timestamp for values that @@ -36,9 +38,9 @@ import org.apache.hadoop.hbase.io.RowResult; */ public class TestScannerTimes extends HBaseClusterTestCase { private static final String TABLE_NAME = "hbase737"; - private static final String FAM1 = "fam1:"; - private static final String FAM2 = "fam2:"; - private static final String ROW = "row"; + private static final byte [] FAM1 = Bytes.toBytes("fam1"); + private static final byte [] FAM2 = Bytes.toBytes("fam2"); + private static final byte [] ROW = Bytes.toBytes("row"); /** * test for HBASE-737 @@ -57,9 +59,9 @@ public class TestScannerTimes extends HBaseClusterTestCase { HTable table = new HTable(conf, TABLE_NAME); // Insert some values - BatchUpdate b = new BatchUpdate(ROW); - b.put(FAM1 + "letters", "abcdefg".getBytes(HConstants.UTF8_ENCODING)); - table.commit(b); + Put put = new Put(ROW); + put.add(FAM1, Bytes.toBytes("letters"), Bytes.toBytes("abcdefg")); + table.put(put); try { Thread.sleep(1000); @@ -67,35 +69,34 @@ public class TestScannerTimes extends HBaseClusterTestCase { //ignore } - b = new BatchUpdate(ROW); - b.put(FAM1 + "numbers", "123456".getBytes(HConstants.UTF8_ENCODING)); - table.commit(b); + put = new Put(ROW); + put.add(FAM1, Bytes.toBytes("numbers"), Bytes.toBytes("123456")); + table.put(put); try { Thread.sleep(1000); } catch (InterruptedException i) { //ignore } - - b = new BatchUpdate(ROW); - b.put(FAM2 + "letters", "hijklmnop".getBytes(HConstants.UTF8_ENCODING)); - table.commit(b); + + put = new Put(ROW); + put.add(FAM2, Bytes.toBytes("letters"), Bytes.toBytes("hijklmnop")); + table.put(put); long times[] = new long[3]; - byte[][] columns = new byte[][] { - FAM1.getBytes(HConstants.UTF8_ENCODING), - FAM2.getBytes(HConstants.UTF8_ENCODING) - }; // First scan the memcache - Scanner s = table.getScanner(columns); + Scan scan = new Scan(); + scan.addFamily(FAM1); + scan.addFamily(FAM2); + ResultScanner s = table.getScanner(scan); try { int index = 0; - RowResult r = null; + Result r = null; while ((r = s.next()) != null) { - for (Cell c: r.values()) { - times[index++] = c.getTimestamp(); + for(KeyValue key : r.sorted()) { + times[index++] = key.getTimestamp(); } } } finally { @@ -107,23 +108,30 @@ public class TestScannerTimes extends HBaseClusterTestCase { } } - // Fush data to disk and try again + // Flush data to disk and try again cluster.flushcache(); + // Reset times + for(int i=0;i times[i]); } } - } } diff --git a/src/test/org/apache/hadoop/hbase/client/tableindexed/TestIndexedTable.java b/src/test/org/apache/hadoop/hbase/client/tableindexed/TestIndexedTable.java deleted file mode 100644 index c949d453194..00000000000 --- a/src/test/org/apache/hadoop/hbase/client/tableindexed/TestIndexedTable.java +++ /dev/null @@ -1,131 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.tableindexed; - -import java.io.IOException; -import java.util.Random; - -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.PerformanceEvaluation; -import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegionServer; -import org.apache.hadoop.hbase.util.Bytes; - -public class TestIndexedTable extends HBaseClusterTestCase { - - private static final Log LOG = LogFactory.getLog(TestIndexedTable.class); - - private static final String TABLE_NAME = "table1"; - - private static final byte[] FAMILY = Bytes.toBytes("family:"); - private static final byte[] COL_A = Bytes.toBytes("family:a"); - private static final String INDEX_COL_A = "A"; - - private static final int NUM_ROWS = 10; - private static final int MAX_VAL = 10000; - - private IndexedTableAdmin admin; - private IndexedTable table; - private Random random = new Random(); - - /** constructor */ - public TestIndexedTable() { - conf - .set(HConstants.REGION_SERVER_IMPL, IndexedRegionServer.class.getName()); - conf.setInt("hbase.master.info.port", -1); - conf.setInt("hbase.regionserver.info.port", -1); - } - - @Override - protected void setUp() throws Exception { - super.setUp(); - - HTableDescriptor desc = new HTableDescriptor(TABLE_NAME); - desc.addFamily(new HColumnDescriptor(FAMILY)); - - // Create a new index that does lexicographic ordering on COL_A - IndexSpecification colAIndex = new IndexSpecification(INDEX_COL_A, - COL_A); - desc.addIndex(colAIndex); - - admin = new IndexedTableAdmin(conf); - admin.createTable(desc); - table = new IndexedTable(conf, desc.getName()); - } - - private void writeInitalRows() throws IOException { - for (int i = 0; i < NUM_ROWS; i++) { - BatchUpdate update = new BatchUpdate(PerformanceEvaluation.format(i)); - byte[] colA = PerformanceEvaluation.format(random.nextInt(MAX_VAL)); - update.put(COL_A, colA); - table.commit(update); - LOG.info("Inserted row [" + Bytes.toString(update.getRow()) + "] val: [" - + Bytes.toString(colA) + "]"); - } - } - - - public void testInitialWrites() throws IOException { - writeInitalRows(); - assertRowsInOrder(NUM_ROWS); - } - - private void assertRowsInOrder(int numRowsExpected) throws IndexNotFoundException, IOException { - Scanner scanner = table.getIndexedScanner(INDEX_COL_A, - HConstants.EMPTY_START_ROW, null, null, null); - int numRows = 0; - byte[] lastColA = null; - for (RowResult rowResult : scanner) { - byte[] colA = rowResult.get(COL_A).getValue(); - LOG.info("index scan : row [" + Bytes.toString(rowResult.getRow()) - + "] value [" + Bytes.toString(colA) + "]"); - if (lastColA != null) { - Assert.assertTrue(Bytes.compareTo(lastColA, colA) <= 0); - } - lastColA = colA; - numRows++; - } - Assert.assertEquals(numRowsExpected, numRows); - } - - public void testMultipleWrites() throws IOException { - writeInitalRows(); - writeInitalRows(); // Update the rows. - assertRowsInOrder(NUM_ROWS); - } - - public void testDelete() throws IOException { - writeInitalRows(); - // Delete the first row; - table.deleteAll(PerformanceEvaluation.format(0)); - - assertRowsInOrder(NUM_ROWS - 1); - } - -} diff --git a/src/test/org/apache/hadoop/hbase/client/transactional/DisabledTestTransactions.java b/src/test/org/apache/hadoop/hbase/client/transactional/DisabledTestTransactions.java deleted file mode 100644 index 0b453e026c4..00000000000 --- a/src/test/org/apache/hadoop/hbase/client/transactional/DisabledTestTransactions.java +++ /dev/null @@ -1,143 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.transactional; - -import java.io.IOException; - -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.io.BatchUpdate; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface; -import org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegionServer; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Test the transaction functionality. This requires to run an - * {@link TransactionalRegionServer}. - */ -public class DisabledTestTransactions extends HBaseClusterTestCase { - - private static final String TABLE_NAME = "table1"; - - private static final byte[] FAMILY = Bytes.toBytes("family:"); - private static final byte[] COL_A = Bytes.toBytes("family:a"); - - private static final byte[] ROW1 = Bytes.toBytes("row1"); - private static final byte[] ROW2 = Bytes.toBytes("row2"); - private static final byte[] ROW3 = Bytes.toBytes("row3"); - - private HBaseAdmin admin; - private TransactionalTable table; - private TransactionManager transactionManager; - - /** constructor */ - public DisabledTestTransactions() { - conf.set(HConstants.REGION_SERVER_CLASS, TransactionalRegionInterface.class - .getName()); - conf.set(HConstants.REGION_SERVER_IMPL, TransactionalRegionServer.class - .getName()); - } - - @Override - protected void setUp() throws Exception { - super.setUp(); - - HTableDescriptor desc = new HTableDescriptor(TABLE_NAME); - desc.addFamily(new HColumnDescriptor(FAMILY)); - admin = new HBaseAdmin(conf); - admin.createTable(desc); - table = new TransactionalTable(conf, desc.getName()); - - transactionManager = new TransactionManager(conf); - writeInitalRow(); - } - - private void writeInitalRow() throws IOException { - BatchUpdate update = new BatchUpdate(ROW1); - update.put(COL_A, Bytes.toBytes(1)); - table.commit(update); - } - - public void testSimpleTransaction() throws IOException, - CommitUnsuccessfulException { - TransactionState transactionState = makeTransaction1(); - transactionManager.tryCommit(transactionState); - } - - public void testTwoTransactionsWithoutConflict() throws IOException, - CommitUnsuccessfulException { - TransactionState transactionState1 = makeTransaction1(); - TransactionState transactionState2 = makeTransaction2(); - - transactionManager.tryCommit(transactionState1); - transactionManager.tryCommit(transactionState2); - } - - public void TestTwoTransactionsWithConflict() throws IOException, - CommitUnsuccessfulException { - TransactionState transactionState1 = makeTransaction1(); - TransactionState transactionState2 = makeTransaction2(); - - transactionManager.tryCommit(transactionState2); - - try { - transactionManager.tryCommit(transactionState1); - fail(); - } catch (CommitUnsuccessfulException e) { - // Good - } - } - - // Read from ROW1,COL_A and put it in ROW2_COLA and ROW3_COLA - private TransactionState makeTransaction1() throws IOException { - TransactionState transactionState = transactionManager.beginTransaction(); - - Cell row1_A = table.get(transactionState, ROW1, COL_A); - - BatchUpdate write1 = new BatchUpdate(ROW2); - write1.put(COL_A, row1_A.getValue()); - table.commit(transactionState, write1); - - BatchUpdate write2 = new BatchUpdate(ROW3); - write2.put(COL_A, row1_A.getValue()); - table.commit(transactionState, write2); - - return transactionState; - } - - // Read ROW1,COL_A, increment its (integer) value, write back - private TransactionState makeTransaction2() throws IOException { - TransactionState transactionState = transactionManager.beginTransaction(); - - Cell row1_A = table.get(transactionState, ROW1, COL_A); - - int value = Bytes.toInt(row1_A.getValue()); - - BatchUpdate write = new BatchUpdate(ROW1); - write.put(COL_A, Bytes.toBytes(value + 1)); - table.commit(transactionState, write); - - return transactionState; - } -} diff --git a/src/test/org/apache/hadoop/hbase/client/transactional/StressTestTransactions.java b/src/test/org/apache/hadoop/hbase/client/transactional/StressTestTransactions.java deleted file mode 100644 index 92fec03cc0c..00000000000 --- a/src/test/org/apache/hadoop/hbase/client/transactional/StressTestTransactions.java +++ /dev/null @@ -1,420 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.transactional; - -import java.io.IOException; -import java.util.LinkedList; -import java.util.List; -import java.util.Random; -import java.util.concurrent.atomic.AtomicBoolean; - -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.io.BatchUpdate; -import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface; -import org.apache.hadoop.hbase.regionserver.transactional.TransactionalRegionServer; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Stress Test the transaction functionality. This requires to run an - * {@link TransactionalRegionServer}. We run many threads doing reads/writes - * which may conflict with each other. We have two types of transactions, those - * which operate on rows of a single table, and those which operate on rows - * across multiple tables. Each transaction type has a modification operation - * which changes two values while maintaining the sum. Also each transaction - * type has a consistency-check operation which sums all rows and verifies that - * the sum is as expected. - */ -public class StressTestTransactions extends HBaseClusterTestCase { - protected static final Log LOG = LogFactory - .getLog(StressTestTransactions.class); - - private static final int NUM_TABLES = 3; - private static final int NUM_ST_ROWS = 3; - private static final int NUM_MT_ROWS = 3; - private static final int NUM_TRANSACTIONS_PER_THREAD = 100; - private static final int NUM_SINGLE_TABLE_THREADS = 6; - private static final int NUM_MULTI_TABLE_THREADS = 6; - private static final int PRE_COMMIT_SLEEP = 10; - protected static final Random RAND = new Random(); - - private static final byte[] FAMILY = Bytes.toBytes("family:"); - static final byte[] COL = Bytes.toBytes("family:a"); - - private HBaseAdmin admin; - protected TransactionalTable[] tables; - protected TransactionManager transactionManager; - - /** constructor */ - public StressTestTransactions() { - conf.set(HConstants.REGION_SERVER_CLASS, TransactionalRegionInterface.class - .getName()); - conf.set(HConstants.REGION_SERVER_IMPL, TransactionalRegionServer.class - .getName()); - } - - @Override - protected void setUp() throws Exception { - super.setUp(); - - tables = new TransactionalTable[NUM_TABLES]; - - for (int i = 0; i < tables.length; i++) { - HTableDescriptor desc = new HTableDescriptor(makeTableName(i)); - desc.addFamily(new HColumnDescriptor(FAMILY)); - admin = new HBaseAdmin(conf); - admin.createTable(desc); - tables[i] = new TransactionalTable(conf, desc.getName()); - } - - transactionManager = new TransactionManager(conf); - } - - private String makeTableName(final int i) { - return "table" + i; - } - - private void writeInitalValues() throws IOException { - for (TransactionalTable table : tables) { - for (int i = 0; i < NUM_ST_ROWS; i++) { - byte[] row = makeSTRow(i); - BatchUpdate b = new BatchUpdate(row); - b.put(COL, Bytes.toBytes(SingleTableTransactionThread.INITIAL_VALUE)); - table.commit(b); - } - for (int i = 0; i < NUM_MT_ROWS; i++) { - byte[] row = makeMTRow(i); - BatchUpdate b = new BatchUpdate(row); - b.put(COL, Bytes.toBytes(MultiTableTransactionThread.INITIAL_VALUE)); - table.commit(b); - } - } - } - - protected byte[] makeSTRow(final int i) { - return Bytes.toBytes("st" + i); - } - - protected byte[] makeMTRow(final int i) { - return Bytes.toBytes("mt" + i); - } - - static int nextThreadNum = 1; - protected static final AtomicBoolean stopRequest = new AtomicBoolean(false); - static final AtomicBoolean consistencyFailure = new AtomicBoolean(false); - - // Thread which runs transactions - abstract class TransactionThread extends Thread { - private int numRuns = 0; - private int numAborts = 0; - private int numUnknowns = 0; - - public TransactionThread(final String namePrefix) { - super.setName(namePrefix + "transaction " + nextThreadNum++); - } - - @Override - public void run() { - for (int i = 0; i < NUM_TRANSACTIONS_PER_THREAD; i++) { - if (stopRequest.get()) { - return; - } - try { - numRuns++; - transaction(); - } catch (UnknownTransactionException e) { - numUnknowns++; - } catch (IOException e) { - throw new RuntimeException(e); - } catch (CommitUnsuccessfulException e) { - numAborts++; - } - } - } - - protected abstract void transaction() throws IOException, - CommitUnsuccessfulException; - - public int getNumAborts() { - return numAborts; - } - - public int getNumUnknowns() { - return numUnknowns; - } - - protected void preCommitSleep() { - try { - Thread.sleep(PRE_COMMIT_SLEEP); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - protected void consistencyFailure() { - LOG.fatal("Consistency failure"); - stopRequest.set(true); - consistencyFailure.set(true); - } - - /** - * Get the numRuns. - * - * @return Return the numRuns. - */ - public int getNumRuns() { - return numRuns; - } - - } - - // Atomically change the value of two rows rows while maintaining the sum. - // This should preserve the global sum of the rows, which is also checked - // with a transaction. - private class SingleTableTransactionThread extends TransactionThread { - private static final int INITIAL_VALUE = 10; - public static final int TOTAL_SUM = INITIAL_VALUE * NUM_ST_ROWS; - private static final int MAX_TRANSFER_AMT = 100; - - private TransactionalTable table; - boolean doCheck = false; - - public SingleTableTransactionThread() { - super("single table "); - } - - @Override - protected void transaction() throws IOException, - CommitUnsuccessfulException { - if (doCheck) { - checkTotalSum(); - } else { - doSingleRowChange(); - } - doCheck = !doCheck; - } - - private void doSingleRowChange() throws IOException, - CommitUnsuccessfulException { - table = tables[RAND.nextInt(NUM_TABLES)]; - int transferAmount = RAND.nextInt(MAX_TRANSFER_AMT * 2) - - MAX_TRANSFER_AMT; - int row1Index = RAND.nextInt(NUM_ST_ROWS); - int row2Index; - do { - row2Index = RAND.nextInt(NUM_ST_ROWS); - } while (row2Index == row1Index); - byte[] row1 = makeSTRow(row1Index); - byte[] row2 = makeSTRow(row2Index); - - TransactionState transactionState = transactionManager.beginTransaction(); - int row1Amount = Bytes.toInt(table.get(transactionState, row1, COL) - .getValue()); - int row2Amount = Bytes.toInt(table.get(transactionState, row2, COL) - .getValue()); - - row1Amount -= transferAmount; - row2Amount += transferAmount; - - BatchUpdate update = new BatchUpdate(row1); - update.put(COL, Bytes.toBytes(row1Amount)); - table.commit(transactionState, update); - update = new BatchUpdate(row2); - update.put(COL, Bytes.toBytes(row2Amount)); - table.commit(transactionState, update); - - super.preCommitSleep(); - - transactionManager.tryCommit(transactionState); - LOG.debug("Commited"); - } - - // Check the table we last mutated - private void checkTotalSum() throws IOException, - CommitUnsuccessfulException { - TransactionState transactionState = transactionManager.beginTransaction(); - int totalSum = 0; - for (int i = 0; i < NUM_ST_ROWS; i++) { - totalSum += Bytes.toInt(table.get(transactionState, makeSTRow(i), COL) - .getValue()); - } - - transactionManager.tryCommit(transactionState); - if (TOTAL_SUM != totalSum) { - super.consistencyFailure(); - } - } - - } - - // Similar to SingleTable, but this time we maintain consistency across tables - // rather than rows - private class MultiTableTransactionThread extends TransactionThread { - private static final int INITIAL_VALUE = 1000; - public static final int TOTAL_SUM = INITIAL_VALUE * NUM_TABLES; - private static final int MAX_TRANSFER_AMT = 100; - - private byte[] row; - boolean doCheck = false; - - public MultiTableTransactionThread() { - super("multi table"); - } - - @Override - protected void transaction() throws IOException, - CommitUnsuccessfulException { - if (doCheck) { - checkTotalSum(); - } else { - doSingleRowChange(); - } - doCheck = !doCheck; - } - - private void doSingleRowChange() throws IOException, - CommitUnsuccessfulException { - row = makeMTRow(RAND.nextInt(NUM_MT_ROWS)); - int transferAmount = RAND.nextInt(MAX_TRANSFER_AMT * 2) - - MAX_TRANSFER_AMT; - int table1Index = RAND.nextInt(tables.length); - int table2Index; - do { - table2Index = RAND.nextInt(tables.length); - } while (table2Index == table1Index); - - TransactionalTable table1 = tables[table1Index]; - TransactionalTable table2 = tables[table2Index]; - - TransactionState transactionState = transactionManager.beginTransaction(); - int table1Amount = Bytes.toInt(table1.get(transactionState, row, COL) - .getValue()); - int table2Amount = Bytes.toInt(table2.get(transactionState, row, COL) - .getValue()); - - table1Amount -= transferAmount; - table2Amount += transferAmount; - - BatchUpdate update = new BatchUpdate(row); - update.put(COL, Bytes.toBytes(table1Amount)); - table1.commit(transactionState, update); - - update = new BatchUpdate(row); - update.put(COL, Bytes.toBytes(table2Amount)); - table2.commit(transactionState, update); - - super.preCommitSleep(); - - transactionManager.tryCommit(transactionState); - - LOG.trace(Bytes.toString(table1.getTableName()) + ": " + table1Amount); - LOG.trace(Bytes.toString(table2.getTableName()) + ": " + table2Amount); - - } - - private void checkTotalSum() throws IOException, - CommitUnsuccessfulException { - TransactionState transactionState = transactionManager.beginTransaction(); - int totalSum = 0; - int[] amounts = new int[tables.length]; - for (int i = 0; i < tables.length; i++) { - int amount = Bytes.toInt(tables[i].get(transactionState, row, COL) - .getValue()); - amounts[i] = amount; - totalSum += amount; - } - - transactionManager.tryCommit(transactionState); - - for (int i = 0; i < tables.length; i++) { - LOG.trace(Bytes.toString(tables[i].getTableName()) + ": " + amounts[i]); - } - - if (TOTAL_SUM != totalSum) { - super.consistencyFailure(); - } - } - - } - - public void testStressTransactions() throws IOException, InterruptedException { - writeInitalValues(); - - List transactionThreads = new LinkedList(); - - for (int i = 0; i < NUM_SINGLE_TABLE_THREADS; i++) { - TransactionThread transactionThread = new SingleTableTransactionThread(); - transactionThread.start(); - transactionThreads.add(transactionThread); - } - - for (int i = 0; i < NUM_MULTI_TABLE_THREADS; i++) { - TransactionThread transactionThread = new MultiTableTransactionThread(); - transactionThread.start(); - transactionThreads.add(transactionThread); - } - - for (TransactionThread transactionThread : transactionThreads) { - transactionThread.join(); - } - - for (TransactionThread transactionThread : transactionThreads) { - LOG.info(transactionThread.getName() + " done with " - + transactionThread.getNumAborts() + " aborts, and " - + transactionThread.getNumUnknowns() + " unknown transactions of " - + transactionThread.getNumRuns()); - } - - doFinalConsistencyChecks(); - } - - private void doFinalConsistencyChecks() throws IOException { - - int[] mtSums = new int[NUM_MT_ROWS]; - for (int i = 0; i < mtSums.length; i++) { - mtSums[i] = 0; - } - - for (TransactionalTable table : tables) { - int thisTableSum = 0; - for (int i = 0; i < NUM_ST_ROWS; i++) { - byte[] row = makeSTRow(i); - thisTableSum += Bytes.toInt(table.get(row, COL).getValue()); - } - Assert.assertEquals(SingleTableTransactionThread.TOTAL_SUM, thisTableSum); - - for (int i = 0; i < NUM_MT_ROWS; i++) { - byte[] row = makeMTRow(i); - mtSums[i] += Bytes.toInt(table.get(row, COL).getValue()); - } - } - - for (int mtSum : mtSums) { - Assert.assertEquals(MultiTableTransactionThread.TOTAL_SUM, mtSum); - } - } -} diff --git a/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterOnMultipleFamilies.java b/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterOnMultipleFamilies.java index b16e3bb0c85..90b8e8013f0 100644 --- a/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterOnMultipleFamilies.java +++ b/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterOnMultipleFamilies.java @@ -32,9 +32,13 @@ 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.KeyValue; 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.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; @@ -71,10 +75,14 @@ public class DisabledTestRowFilterOnMultipleFamilies extends HBaseClusterTestCas 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); + Put put = new Put(Bytes.toBytes("row_" + String.format("%1$05d", i))); + byte [][] famAndQf = KeyValue.parseColumn(TEXT_COLUMN1); + put.add(famAndQf[0], famAndQf[1], VALUE); + + famAndQf = KeyValue.parseColumn(TEXT_COLUMN2); + put.add(famAndQf[0], famAndQf[1], Bytes.toBytes(String.format("%1$05d", i))); + + table.put(put); } LOG.info("Print table contents using scanner before map/reduce for " + TABLE_NAME); @@ -85,7 +93,9 @@ public class DisabledTestRowFilterOnMultipleFamilies extends HBaseClusterTestCas 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); + Scan scan = new Scan(); + scan.addColumns(columns); + ResultScanner scanner = table.getScanner(scan); int numFound = doScan(scanner, printValues); Assert.assertEquals(NUM_ROWS, numFound); } @@ -96,21 +106,24 @@ public class DisabledTestRowFilterOnMultipleFamilies extends HBaseClusterTestCas 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); + Scan scan = new Scan(); + scan.addColumns(columns); +// scan.setFilter(filter); + ResultScanner scanner = table.getScanner(scan); int numFound = doScan(scanner, printValues); Assert.assertEquals(NUM_ROWS, numFound); } - private int doScan(final Scanner scanner, final boolean printValues) throws IOException { + private int doScan(final ResultScanner scanner, final boolean printValues) throws IOException { { int count = 0; try { - for (RowResult result : scanner) { + for (Result result : scanner) { if (printValues) { LOG.info("row: " + Bytes.toString(result.getRow())); - for (Map.Entry e : result.entrySet()) { + for (Map.Entry e : result.getRowResult().entrySet()) { LOG.info(" column: " + e.getKey() + " value: " + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); } diff --git a/src/test/org/apache/hadoop/hbase/filter/DisabledTestPageRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestPageFilter.java similarity index 67% rename from src/test/org/apache/hadoop/hbase/filter/DisabledTestPageRowFilter.java rename to src/test/org/apache/hadoop/hbase/filter/TestPageFilter.java index 3c0fdfb9207..5c9b2d865a5 100644 --- a/src/test/org/apache/hadoop/hbase/filter/DisabledTestPageRowFilter.java +++ b/src/test/org/apache/hadoop/hbase/filter/TestPageFilter.java @@ -24,31 +24,25 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.util.Bytes; import junit.framework.TestCase; /** - * Tests for the page row filter + * Tests for the page filter */ -public class DisabledTestPageRowFilter extends TestCase { - - RowFilterInterface mainFilter; +public class TestPageFilter extends TestCase { 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); + Filter f = new PageFilter(ROW_LIMIT); + pageSizeTests(f); } /** @@ -56,43 +50,49 @@ public class DisabledTestPageRowFilter extends TestCase { * @throws Exception */ public void testSerialization() throws Exception { + Filter f = new PageFilter(ROW_LIMIT); // Decompose mainFilter to bytes. ByteArrayOutputStream stream = new ByteArrayOutputStream(); DataOutputStream out = new DataOutputStream(stream); - mainFilter.write(out); + f.write(out); out.close(); byte[] buffer = stream.toByteArray(); - // Recompose mainFilter. DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer)); - RowFilterInterface newFilter = new PageRowFilter(); + Filter newFilter = new PageFilter(); 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); + private void pageSizeTests(Filter f) throws Exception { + testFiltersBeyondPageSize(f, ROW_LIMIT); // Test reset works by going in again. - filter.reset(); - testFiltersBeyondPageSize(filter, ROW_LIMIT); + f.reset(); + testFiltersBeyondPageSize(f, ROW_LIMIT); } - private void testFiltersBeyondPageSize(final RowFilterInterface filter, - final int pageSize) { + private void testFiltersBeyondPageSize(final Filter f, final int pageSize) { + int count = 0; for (int i = 0; i < (pageSize * 2); i++) { - byte [] row = Bytes.toBytes(Integer.toString(i)); - boolean filterOut = filter.filterRowKey(row); + byte [] bytes = Bytes.toBytes(Integer.toString(i) + ":tail"); + KeyValue kv = new KeyValue(bytes, bytes); + boolean filterOut = + f.filterRowKey(kv.getBuffer(), kv.getRowOffset(), kv.getRowLength()); if (!filterOut) { - assertFalse("Disagrees with 'filter'", filter.filterAllRemaining()); + assertFalse("Disagrees with 'filter'", f.filterAllRemaining()); } else { // Once we have all for a page, calls to filterAllRemaining should // stay true. - assertTrue("Disagrees with 'filter'", filter.filterAllRemaining()); + assertTrue("Disagrees with 'filter'", f.filterAllRemaining()); assertTrue(i >= pageSize); } - filter.rowProcessed(filterOut, row); + count++; + if (Filter.ReturnCode.NEXT_ROW == f.filterKeyValue(kv)) { + break; + } } + assertEquals(pageSize, count); } } diff --git a/src/test/org/apache/hadoop/hbase/filter/TestRowPrefixFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestRowPrefixFilter.java new file mode 100644 index 00000000000..6854a04d8f9 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestRowPrefixFilter.java @@ -0,0 +1,91 @@ +/* + * 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 junit.framework.TestCase; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.UnsupportedEncodingException; + +public class TestRowPrefixFilter extends TestCase { + Filter 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(); + } + } + + protected void setUp() throws Exception { + super.setUp(); + this.mainFilter = new RowPrefixFilter(Bytes.toBytes(HOST_PREFIX)); + } + + public void testPrefixOnRow() throws Exception { + prefixRowTests(mainFilter); + } + + 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)); + Filter newFilter = new RowPrefixFilter(); + newFilter.readFields(in); + + // Ensure the serialization preserved the filter by running all test. + prefixRowTests(newFilter); + } + + private void prefixRowTests(Filter filter) throws Exception { + for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) { + byte [] t = createRow(c); + assertFalse("Failed with characer " + c, filter.filterRowKey(t, 0, t.length)); + } + String yahooSite = "com.yahoo.www"; + byte [] yahooSiteBytes = Bytes.toBytes(yahooSite); + assertTrue("Failed with character " + + yahooSite, filter.filterRowKey(yahooSiteBytes, 0, yahooSiteBytes.length)); + } + + private byte [] createRow(final char c) { + return Bytes.toBytes(HOST_PREFIX + Character.toString(c)); + } + + + +} diff --git a/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java b/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java index 5a547a6940f..1fc89e9b570 100644 --- a/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java +++ b/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java @@ -95,4 +95,4 @@ public class TestHbaseObjectWritable extends TestCase { dis.close(); return product; } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java b/src/test/org/apache/hadoop/hbase/mapred/DisabledTestTableIndex.java similarity index 91% rename from src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java rename to src/test/org/apache/hadoop/hbase/mapred/DisabledTestTableIndex.java index b4842843548..8f2cd525fb7 100644 --- a/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java +++ b/src/test/org/apache/hadoop/hbase/mapred/DisabledTestTableIndex.java @@ -36,7 +36,9 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Scanner; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.HTableDescriptor; @@ -57,8 +59,8 @@ import org.apache.lucene.search.TermQuery; /** * Test Map/Reduce job to build index over HBase table */ -public class TestTableIndex extends MultiRegionTable { - private static final Log LOG = LogFactory.getLog(TestTableIndex.class); +public class DisabledTestTableIndex extends MultiRegionTable { + private static final Log LOG = LogFactory.getLog(DisabledTestTableIndex.class); static final String TABLE_NAME = "moretest"; static final String INPUT_COLUMN = "contents:"; @@ -75,7 +77,7 @@ public class TestTableIndex extends MultiRegionTable { private JobConf jobConf = null; /** default constructor */ - public TestTableIndex() { + public DisabledTestTableIndex() { super(INPUT_COLUMN); desc = new HTableDescriptor(TABLE_NAME); desc.addFamily(new HColumnDescriptor(INPUT_COLUMN)); @@ -107,7 +109,7 @@ public class TestTableIndex extends MultiRegionTable { conf.set("hbase.index.conf", createIndexConfContent()); try { - jobConf = new JobConf(conf, TestTableIndex.class); + jobConf = new JobConf(conf, DisabledTestTableIndex.class); jobConf.setJobName("index column contents"); jobConf.setNumMapTasks(2); // number of indexes to partition into @@ -173,14 +175,15 @@ public class TestTableIndex extends MultiRegionTable { private void scanTable(boolean printResults) throws IOException { HTable table = new HTable(conf, TABLE_NAME); - Scanner scanner = table.getScanner(columns, - HConstants.EMPTY_START_ROW); + Scan scan = new Scan(); + scan.addColumns(columns); + ResultScanner scanner = table.getScanner(scan); try { - for (RowResult r : scanner) { + for (Result r : scanner) { if (printResults) { LOG.info("row: " + r.getRow()); } - for (Map.Entry e : r.entrySet()) { + for (Map.Entry e : r.getRowResult().entrySet()) { if (printResults) { LOG.info(" column: " + e.getKey() + " value: " + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); @@ -207,7 +210,7 @@ public class TestTableIndex extends MultiRegionTable { FileSystem localfs = FileSystem.getLocal(conf); FileStatus [] indexDirs = localfs.listStatus(localDir); Searcher searcher = null; - Scanner scanner = null; + ResultScanner scanner = null; try { if (indexDirs.length == 1) { searcher = new IndexSearcher((new File(indexDirs[0].getPath(). @@ -224,7 +227,9 @@ public class TestTableIndex extends MultiRegionTable { } HTable table = new HTable(conf, TABLE_NAME); - scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); + Scan scan = new Scan(); + scan.addColumns(columns); + scanner = table.getScanner(scan); IndexConfiguration indexConf = new IndexConfiguration(); String content = conf.get("hbase.index.conf"); @@ -234,7 +239,7 @@ public class TestTableIndex extends MultiRegionTable { String rowkeyName = indexConf.getRowkeyName(); int count = 0; - for (RowResult r : scanner) { + for (Result r : scanner) { String value = Bytes.toString(r.getRow()); Term term = new Term(rowkeyName, value); int hitCount = searcher.search(new TermQuery(term)).length(); @@ -256,6 +261,6 @@ public class TestTableIndex extends MultiRegionTable { * @param args unused */ public static void main(String[] args) { - TestRunner.run(new TestSuite(TestTableIndex.class)); + TestRunner.run(new TestSuite(DisabledTestTableIndex.class)); } } \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java b/src/test/org/apache/hadoop/hbase/mapred/DisabledTestTableMapReduce.java similarity index 93% rename from src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java rename to src/test/org/apache/hadoop/hbase/mapred/DisabledTestTableMapReduce.java index 220c1c9880e..e4a9774a719 100644 --- a/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java +++ b/src/test/org/apache/hadoop/hbase/mapred/DisabledTestTableMapReduce.java @@ -31,7 +31,9 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MultiRegionTable; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Scanner; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -49,9 +51,9 @@ import org.apache.hadoop.mapred.Reporter; * on our tables is simple - take every row in the table, reverse the value of * a particular cell, and write it back to the table. */ -public class TestTableMapReduce extends MultiRegionTable { +public class DisabledTestTableMapReduce extends MultiRegionTable { private static final Log LOG = - LogFactory.getLog(TestTableMapReduce.class.getName()); + LogFactory.getLog(DisabledTestTableMapReduce.class.getName()); static final String MULTI_REGION_TABLE_NAME = "mrtest"; static final String INPUT_COLUMN = "contents:"; @@ -63,7 +65,7 @@ public class TestTableMapReduce extends MultiRegionTable { }; /** constructor */ - public TestTableMapReduce() { + public DisabledTestTableMapReduce() { super(INPUT_COLUMN); desc = new HTableDescriptor(MULTI_REGION_TABLE_NAME); desc.addFamily(new HColumnDescriptor(INPUT_COLUMN)); @@ -128,7 +130,7 @@ public class TestTableMapReduce extends MultiRegionTable { JobConf jobConf = null; try { LOG.info("Before map/reduce startup"); - jobConf = new JobConf(conf, TestTableMapReduce.class); + jobConf = new JobConf(conf, DisabledTestTableMapReduce.class); jobConf.setJobName("process column contents"); jobConf.setNumReduceTasks(1); TableMapReduceUtil.initTableMapJob(Bytes.toString(table.getTableName()), @@ -184,10 +186,11 @@ public class TestTableMapReduce extends MultiRegionTable { * @throws NullPointerException if we failed to find a cell value */ private void verifyAttempt(final HTable table) throws IOException, NullPointerException { - Scanner scanner = - table.getScanner(columns, HConstants.EMPTY_START_ROW); + Scan scan = new Scan(); + scan.addColumns(columns); + ResultScanner scanner = table.getScanner(scan); try { - for (RowResult r : scanner) { + for (Result r : scanner) { if (LOG.isDebugEnabled()) { if (r.size() > 2 ) { throw new IOException("Too many results, expected 2 got " + @@ -197,7 +200,7 @@ public class TestTableMapReduce extends MultiRegionTable { byte[] firstValue = null; byte[] secondValue = null; int count = 0; - for(Map.Entry e: r.entrySet()) { + for(Map.Entry e: r.getRowResult().entrySet()) { if (count == 0) { firstValue = e.getValue().getValue(); } diff --git a/src/test/org/apache/hadoop/hbase/regionserver/DisabledTestRegionServerExit.java b/src/test/org/apache/hadoop/hbase/regionserver/DisabledTestRegionServerExit.java index 99140cc615b..eddafe72afb 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/DisabledTestRegionServerExit.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/DisabledTestRegionServerExit.java @@ -32,7 +32,10 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.LocalHBaseCluster; 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.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.util.Bytes; @@ -102,15 +105,15 @@ public class DisabledTestRegionServerExit extends HBaseClusterTestCase { private byte [] createTableAndAddRow(final String tableName) throws IOException { HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY)); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); HBaseAdmin admin = new HBaseAdmin(conf); admin.createTable(desc); // put some values in the table this.table = new HTable(conf, tableName); byte [] row = Bytes.toBytes("row1"); - BatchUpdate b = new BatchUpdate(row); - b.put(HConstants.COLUMN_FAMILY, Bytes.toBytes(tableName)); - table.commit(b); + Put put = new Put(row); + put.add(HConstants.CATALOG_FAMILY, null, Bytes.toBytes(tableName)); + table.put(put); return row; } @@ -166,27 +169,29 @@ public class DisabledTestRegionServerExit extends HBaseClusterTestCase { // Now try to open a scanner on the meta table. Should stall until // meta server comes back up. HTable t = new HTable(conf, HConstants.META_TABLE_NAME); - Scanner s = - t.getScanner(HConstants.COLUMN_FAMILY_ARRAY, - HConstants.EMPTY_START_ROW); + Scan scan = new Scan(); + scan.addFamily(HConstants.CATALOG_FAMILY); + + ResultScanner s = t.getScanner(scan); s.close(); } catch (IOException e) { LOG.fatal("could not re-open meta table because", e); fail(); } - Scanner scanner = null; + ResultScanner scanner = null; try { // Verify that the client can find the data after the region has moved // to a different server - scanner = - table.getScanner(HConstants.COLUMN_FAMILY_ARRAY, - HConstants.EMPTY_START_ROW); + Scan scan = new Scan(); + scan.addFamily(HConstants.CATALOG_FAMILY); + + scanner = table.getScanner(scan); LOG.info("Obtained scanner " + scanner); - for (RowResult r : scanner) { + for (Result r : scanner) { assertTrue(Bytes.equals(r.getRow(), row)); assertEquals(1, r.size()); - byte[] bytes = r.get(HConstants.COLUMN_FAMILY).getValue(); + byte[] bytes = r.getRowResult().get(HConstants.CATALOG_FAMILY).getValue(); assertNotNull(bytes); assertTrue(tableName.equals(Bytes.toString(bytes))); } diff --git a/src/test/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java b/src/test/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java new file mode 100644 index 00000000000..1c44ad7e157 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java @@ -0,0 +1,91 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.hbase.regionserver.KeyValueScanner; +import org.apache.hadoop.hbase.KeyValue; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.Collections; + +/** + * A fixture that implements and presents a KeyValueScanner. + * It takes a list of key/values which is then sorted according + * to the provided comparator, and then the whole thing pretends + * to be a store file scanner. + */ +public class KeyValueScanFixture implements KeyValueScanner { + ArrayList data; + Iterator iter = null; + KeyValue current = null; + KeyValue.KVComparator comparator; + + public KeyValueScanFixture(KeyValue.KVComparator comparator, + KeyValue... incData) { + this.comparator = comparator; + + data = new ArrayList(incData.length); + for( int i = 0; i < incData.length ; ++i) { + data.add(incData[i]); + } + Collections.sort(data, this.comparator); + } + + @Override + public KeyValue peek() { + return this.current; + } + + @Override + public KeyValue next() { + KeyValue res = current; + + if (iter.hasNext()) + current = iter.next(); + else + current = null; + return res; + } + + @Override + public boolean seek(KeyValue key) { + // start at beginning. + iter = data.iterator(); + int cmp; + KeyValue kv = null; + do { + if (!iter.hasNext()) { + current = null; + return false; + } + kv = iter.next(); + cmp = comparator.compare(key, kv); + } while (cmp > 0); + current = kv; + return true; + } + + @Override + public void close() { + // noop. + } +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java b/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java index d130baf5889..e8044482e0b 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java @@ -23,7 +23,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HServerAddress; @@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.HServerAddress; * ${HBASE_HOME}/bin/hbase ./bin/hbase org.apache.hadoop.hbase.OOMERegionServer start. */ public class OOMERegionServer extends HRegionServer { - private List retainer = new ArrayList(); + private List retainer = new ArrayList(); public OOMERegionServer(HBaseConfiguration conf) throws IOException { super(conf); @@ -46,12 +46,12 @@ public class OOMERegionServer extends HRegionServer { super(address, conf); } - public void batchUpdate(byte [] regionName, BatchUpdate b) + public void put(byte [] regionName, Put put) throws IOException { - super.batchUpdate(regionName, b, -1L); + super.put(regionName, put); for (int i = 0; i < 30; i++) { // Add the batch update 30 times to bring on the OOME faster. - this.retainer.add(b); + this.retainer.add(put); } } diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestAtomicIncrement.java b/src/test/org/apache/hadoop/hbase/regionserver/TestAtomicIncrement.java deleted file mode 100644 index 651daf10e53..00000000000 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestAtomicIncrement.java +++ /dev/null @@ -1,121 +0,0 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HBaseClusterTestCase; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.util.Bytes; - -public class TestAtomicIncrement extends HBaseClusterTestCase { - static final Log LOG = LogFactory.getLog(TestAtomicIncrement.class); - - private static final byte [] CONTENTS = Bytes.toBytes("contents:"); - - public void testIncrement() throws IOException { - try { - HTable table = null; - - // Setup - - HTableDescriptor desc = new HTableDescriptor(getName()); - desc.addFamily( - new HColumnDescriptor(CONTENTS, // Column name - 1, // Max versions - HColumnDescriptor.DEFAULT_COMPRESSION, // no compression - HColumnDescriptor.DEFAULT_IN_MEMORY, // not in memory - HColumnDescriptor.DEFAULT_BLOCKCACHE, - HColumnDescriptor.DEFAULT_LENGTH, - HColumnDescriptor.DEFAULT_TTL, - false - ) - ); - - // Create the table - - HBaseAdmin admin = new HBaseAdmin(conf); - admin.createTable(desc); - - try { - // Give cache flusher and log roller a chance to run - // Otherwise we'll never hit the bloom filter, just the memcache - Thread.sleep(conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000) * 10); - - } catch (InterruptedException e) { - // ignore - } - // Open table - - table = new HTable(conf, desc.getName()); - - byte [] row = Bytes.toBytes("foo"); - byte [] column = "contents:1".getBytes(HConstants.UTF8_ENCODING); - // increment by 1: - assertEquals(1L, table.incrementColumnValue(row, column, 1)); - - // set a weird value, then increment: - row = Bytes.toBytes("foo2"); - byte [] value = {0,0,2}; - BatchUpdate bu = new BatchUpdate(row); - bu.put(column, value); - table.commit(bu); - - assertEquals(3L, table.incrementColumnValue(row, column, 1)); - - assertEquals(-2L, table.incrementColumnValue(row, column, -5)); - assertEquals(-502L, table.incrementColumnValue(row, column, -500)); - assertEquals(1500L, table.incrementColumnValue(row, column, 2002)); - assertEquals(1501L, table.incrementColumnValue(row, column, 1)); - - row = Bytes.toBytes("foo3"); - byte[] value2 = {1,2,3,4,5,6,7,8,9}; - bu = new BatchUpdate(row); - bu.put(column, value2); - table.commit(bu); - - try { - table.incrementColumnValue(row, column, 1); - fail(); - } catch (IOException e) { - System.out.println("Expected exception: " + e); - // expected exception. - } - - - } catch (Exception e) { - e.printStackTrace(); - if (e instanceof IOException) { - IOException i = (IOException) e; - throw i; - } - fail(); - } - - } - -} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestBloomFilters.java b/src/test/org/apache/hadoop/hbase/regionserver/TestBloomFilters.java deleted file mode 100644 index 91073e2afcd..00000000000 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestBloomFilters.java +++ /dev/null @@ -1,247 +0,0 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver; - -import java.io.IOException; -import java.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -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.HColumnDescriptor; -import org.apache.hadoop.hbase.HBaseClusterTestCase; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.HBaseAdmin; - -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.util.Bytes; - -/** Tests per-column bloom filters */ -public class TestBloomFilters extends HBaseClusterTestCase { - static final Log LOG = LogFactory.getLog(TestBloomFilters.class); - - private static final byte [] CONTENTS = Bytes.toBytes("contents:"); - - private static final byte [][] rows = { - Bytes.toBytes("wmjwjzyv"), - Bytes.toBytes("baietibz"), - Bytes.toBytes("guhsgxnv"), - Bytes.toBytes("mhnqycto"), - Bytes.toBytes("xcyqafgz"), - Bytes.toBytes("zidoamgb"), - Bytes.toBytes("tftfirzd"), - Bytes.toBytes("okapqlrg"), - Bytes.toBytes("yccwzwsq"), - Bytes.toBytes("qmonufqu"), - Bytes.toBytes("wlsctews"), - Bytes.toBytes("mksdhqri"), - Bytes.toBytes("wxxllokj"), - Bytes.toBytes("eviuqpls"), - Bytes.toBytes("bavotqmj"), - Bytes.toBytes("yibqzhdl"), - Bytes.toBytes("csfqmsyr"), - Bytes.toBytes("guxliyuh"), - Bytes.toBytes("pzicietj"), - Bytes.toBytes("qdwgrqwo"), - Bytes.toBytes("ujfzecmi"), - Bytes.toBytes("dzeqfvfi"), - Bytes.toBytes("phoegsij"), - Bytes.toBytes("bvudfcou"), - Bytes.toBytes("dowzmciz"), - Bytes.toBytes("etvhkizp"), - Bytes.toBytes("rzurqycg"), - Bytes.toBytes("krqfxuge"), - Bytes.toBytes("gflcohtd"), - Bytes.toBytes("fcrcxtps"), - Bytes.toBytes("qrtovxdq"), - Bytes.toBytes("aypxwrwi"), - Bytes.toBytes("dckpyznr"), - Bytes.toBytes("mdaawnpz"), - Bytes.toBytes("pakdfvca"), - Bytes.toBytes("xjglfbez"), - Bytes.toBytes("xdsecofi"), - Bytes.toBytes("sjlrfcab"), - Bytes.toBytes("ebcjawxv"), - Bytes.toBytes("hkafkjmy"), - Bytes.toBytes("oimmwaxo"), - Bytes.toBytes("qcuzrazo"), - Bytes.toBytes("nqydfkwk"), - Bytes.toBytes("frybvmlb"), - Bytes.toBytes("amxmaqws"), - Bytes.toBytes("gtkovkgx"), - Bytes.toBytes("vgwxrwss"), - Bytes.toBytes("xrhzmcep"), - Bytes.toBytes("tafwziil"), - Bytes.toBytes("erjmncnv"), - Bytes.toBytes("heyzqzrn"), - Bytes.toBytes("sowvyhtu"), - Bytes.toBytes("heeixgzy"), - Bytes.toBytes("ktcahcob"), - Bytes.toBytes("ljhbybgg"), - Bytes.toBytes("jiqfcksl"), - Bytes.toBytes("anjdkjhm"), - Bytes.toBytes("uzcgcuxp"), - Bytes.toBytes("vzdhjqla"), - Bytes.toBytes("svhgwwzq"), - Bytes.toBytes("zhswvhbp"), - Bytes.toBytes("ueceybwy"), - Bytes.toBytes("czkqykcw"), - Bytes.toBytes("ctisayir"), - Bytes.toBytes("hppbgciu"), - Bytes.toBytes("nhzgljfk"), - Bytes.toBytes("vaziqllf"), - Bytes.toBytes("narvrrij"), - Bytes.toBytes("kcevbbqi"), - Bytes.toBytes("qymuaqnp"), - Bytes.toBytes("pwqpfhsr"), - Bytes.toBytes("peyeicuk"), - Bytes.toBytes("kudlwihi"), - Bytes.toBytes("pkmqejlm"), - Bytes.toBytes("ylwzjftl"), - Bytes.toBytes("rhqrlqar"), - Bytes.toBytes("xmftvzsp"), - Bytes.toBytes("iaemtihk"), - Bytes.toBytes("ymsbrqcu"), - Bytes.toBytes("yfnlcxto"), - Bytes.toBytes("nluqopqh"), - Bytes.toBytes("wmrzhtox"), - Bytes.toBytes("qnffhqbl"), - Bytes.toBytes("zypqpnbw"), - Bytes.toBytes("oiokhatd"), - Bytes.toBytes("mdraddiu"), - Bytes.toBytes("zqoatltt"), - Bytes.toBytes("ewhulbtm"), - Bytes.toBytes("nmswpsdf"), - Bytes.toBytes("xsjeteqe"), - Bytes.toBytes("ufubcbma"), - Bytes.toBytes("phyxvrds"), - Bytes.toBytes("vhnfldap"), - Bytes.toBytes("zrrlycmg"), - Bytes.toBytes("becotcjx"), - Bytes.toBytes("wvbubokn"), - Bytes.toBytes("avkgiopr"), - Bytes.toBytes("mbqqxmrv"), - Bytes.toBytes("ibplgvuu"), - Bytes.toBytes("dghvpkgc") - }; - - private static final byte [][] testKeys = { - Bytes.toBytes("abcdefgh"), - Bytes.toBytes("ijklmnop"), - Bytes.toBytes("qrstuvwx"), - Bytes.toBytes("yzabcdef") - }; - - /** - * Test that uses automatic bloom filter - * @throws IOException - */ - @SuppressWarnings("null") - public void testComputedParameters() throws IOException { - try { - HTable table = null; - - // Setup - - HTableDescriptor desc = new HTableDescriptor(getName()); - desc.addFamily( - new HColumnDescriptor(CONTENTS, // Column name - 1, // Max versions - HColumnDescriptor.DEFAULT_COMPRESSION, // no compression - HColumnDescriptor.DEFAULT_IN_MEMORY, // not in memory - HColumnDescriptor.DEFAULT_BLOCKCACHE, - HColumnDescriptor.DEFAULT_LENGTH, - HColumnDescriptor.DEFAULT_TTL, - true - ) - ); - - // Create the table - - HBaseAdmin admin = new HBaseAdmin(conf); - admin.createTable(desc); - - // Open table - - table = new HTable(conf, desc.getName()); - - // Store some values - - for(int i = 0; i < 100; i++) { - byte [] row = rows[i]; - String value = Bytes.toString(row); - BatchUpdate b = new BatchUpdate(row); - b.put(CONTENTS, value.getBytes(HConstants.UTF8_ENCODING)); - table.commit(b); - } - - // Get HRegionInfo for our table - Map regions = table.getRegionsInfo(); - assertEquals(1, regions.size()); - HRegionInfo info = null; - for (HRegionInfo hri: regions.keySet()) { - info = hri; - break; - } - - // Request a cache flush - HRegionServer hrs = cluster.getRegionServer(0); - - hrs.getFlushRequester().request(hrs.getOnlineRegion(info.getRegionName())); - - try { - // Give cache flusher and log roller a chance to run - // Otherwise we'll never hit the bloom filter, just the memcache - Thread.sleep(conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000) * 10); - - } catch (InterruptedException e) { - // ignore - } - - for(int i = 0; i < testKeys.length; i++) { - Cell value = table.get(testKeys[i], CONTENTS); - if(value != null && value.getValue().length != 0) { - LOG.error("non existant key: " + Bytes.toString(testKeys[i]) + " returned value: " + - Bytes.toString(value.getValue())); - fail(); - } - } - - for (int i = 0; i < rows.length; i++) { - Cell value = table.get(rows[i], CONTENTS); - if (value == null || value.getValue().length == 0) { - LOG.error("No value returned for row " + Bytes.toString(rows[i])); - fail(); - } - } - } catch (Exception e) { - e.printStackTrace(); - if (e instanceof IOException) { - IOException i = (IOException) e; - throw i; - } - fail(); - } - } -} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java index 7a105446b4f..fc144cf9ca8 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -26,6 +26,10 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.util.Bytes; @@ -37,7 +41,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; public class TestCompaction extends HBaseTestCase { static final Log LOG = LogFactory.getLog(TestCompaction.class.getName()); private HRegion r = null; - private static final byte [] COLUMN_FAMILY = COLFAMILY_NAME1; + private static final byte [] COLUMN_FAMILY = fam1; private final byte [] STARTROW = Bytes.toBytes(START_KEY); private static final byte [] COLUMN_FAMILY_TEXT = COLUMN_FAMILY; private static final int COMPACTION_THRESHOLD = MAXVERSIONS; @@ -90,11 +94,16 @@ public class TestCompaction extends HBaseTestCase { // Default is that there only 3 (MAXVERSIONS) versions allowed per column. // Assert == 3 when we ask for versions. addContent(new HRegionIncommon(r), Bytes.toString(COLUMN_FAMILY)); + + // FIX!! - Cell[] cellValues = - Cell.createSingleCellArray(r.get(STARTROW, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/)); +// Cell[] cellValues = +// Cell.createSingleCellArray(r.get(STARTROW, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/)); + Result result = r.get(new Get(STARTROW).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null); + // Assert that I can get 3 versions since it is the max I should get - assertEquals(cellValues.length, 3); + assertEquals(3, result.size()); +// assertEquals(cellValues.length, 3); r.flushcache(); r.compactStores(); // Always 3 versions if that is what max versions is. @@ -102,32 +111,49 @@ public class TestCompaction extends HBaseTestCase { // Increment the least significant character so we get to next row. secondRowBytes[START_KEY_BYTES.length - 1]++; // FIX - cellValues = Cell.createSingleCellArray(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100/*Too many*/)); - LOG.info("Count of " + Bytes.toString(secondRowBytes) + ": " + - cellValues.length); - assertTrue(cellValues.length == 3); + result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null); + + // Assert that I can get 3 versions since it is the max I should get + assertEquals(3, result.size()); +// +// cellValues = Cell.createSingleCellArray(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100/*Too many*/)); +// LOG.info("Count of " + Bytes.toString(secondRowBytes) + ": " + +// cellValues.length); +// assertTrue(cellValues.length == 3); // Now add deletes to memcache and then flush it. That will put us over // the compaction threshold of 3 store files. Compacting these store files // should result in a compacted store file that has no references to the // deleted row. - r.deleteAll(secondRowBytes, COLUMN_FAMILY_TEXT, System.currentTimeMillis(), - null); + Delete delete = new Delete(secondRowBytes, System.currentTimeMillis(), null); + byte [][] famAndQf = {COLUMN_FAMILY, null}; + delete.deleteFamily(famAndQf[0]); + r.delete(delete, null, true); + // Assert deleted. - assertNull(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/)); + + result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null ); + assertTrue(result.isEmpty()); + + r.flushcache(); - assertNull(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/)); + result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null ); + assertTrue(result.isEmpty()); + // Add a bit of data and flush. Start adding at 'bbb'. createSmallerStoreFile(this.r); r.flushcache(); // Assert that the second row is still deleted. - cellValues = Cell.createSingleCellArray(r.get(secondRowBytes, - COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/)); - assertNull(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/)); + result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null ); + assertTrue(result.isEmpty()); + // Force major compaction. r.compactStores(true); assertEquals(r.getStore(COLUMN_FAMILY_TEXT).getStorefiles().size(), 1); - assertNull(r.get(secondRowBytes, COLUMN_FAMILY_TEXT, -1, 100 /*Too many*/)); + + result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null ); + assertTrue(result.isEmpty()); + // Make sure the store files do have some 'aaa' keys in them -- exactly 3. // Also, that compacted store files do not have any secondRowBytes because // they were deleted. diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java b/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java deleted file mode 100644 index 6a207cde6a7..00000000000 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java +++ /dev/null @@ -1,233 +0,0 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hbase.HBaseTestCase; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Test the functionality of deleteAll. - */ -public class TestDeleteAll extends HBaseTestCase { - static final Log LOG = LogFactory.getLog(TestDeleteAll.class); - - private final String COLUMN_REGEX = "[a-zA-Z0-9]*:[b|c]?"; - - private MiniDFSCluster miniHdfs; - - @Override - protected void setUp() throws Exception { - super.setUp(); - try { - 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()); - } catch (Exception e) { - LOG.fatal("error starting MiniDFSCluster", e); - throw e; - } - } - - /** - * Tests for HADOOP-1550. - * @throws Exception - */ - public void testDeleteAll() throws Exception { - HRegion region = null; - HRegionIncommon region_incommon = null; - try { - HTableDescriptor htd = createTableDescriptor(getName()); - region = createNewHRegion(htd, null, null); - region_incommon = new HRegionIncommon(region); - - // test memcache - makeSureItWorks(region, region_incommon, false); - // test hstore - makeSureItWorks(region, region_incommon, true); - - // regex test memcache - makeSureRegexWorks(region, region_incommon, false); - // regex test hstore - makeSureRegexWorks(region, region_incommon, true); - } finally { - if (region != null) { - try { - region.close(); - } catch (Exception e) { - e.printStackTrace(); - } - region.getLog().closeAndDelete(); - } - } - } - - private void makeSureItWorks(HRegion region, HRegionIncommon region_incommon, - boolean flush) - throws Exception{ - // insert a few versions worth of data for a row - byte [] row = Bytes.toBytes("test_row"); - long now = System.currentTimeMillis(); - long past = now - 100; - long future = now + 100; - Thread.sleep(100); - LOG.info("now=" + now + ", past=" + past + ", future=" + future); - - byte [] colA = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a"); - byte [] colB = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b"); - byte [] colC = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "c"); - byte [] colD = Bytes.toBytes(Bytes.toString(COLUMNS[0])); - - BatchUpdate batchUpdate = new BatchUpdate(row, now); - batchUpdate.put(colA, cellData(0, flush).getBytes()); - batchUpdate.put(colB, cellData(0, flush).getBytes()); - batchUpdate.put(colC, cellData(0, flush).getBytes()); - batchUpdate.put(colD, cellData(0, flush).getBytes()); - region_incommon.commit(batchUpdate); - - batchUpdate = new BatchUpdate(row, past); - batchUpdate.put(colA, cellData(1, flush).getBytes()); - batchUpdate.put(colB, cellData(1, flush).getBytes()); - batchUpdate.put(colC, cellData(1, flush).getBytes()); - batchUpdate.put(colD, cellData(1, flush).getBytes()); - region_incommon.commit(batchUpdate); - - batchUpdate = new BatchUpdate(row, future); - batchUpdate.put(colA, cellData(2, flush).getBytes()); - batchUpdate.put(colB, cellData(2, flush).getBytes()); - batchUpdate.put(colC, cellData(2, flush).getBytes()); - batchUpdate.put(colD, cellData(2, flush).getBytes()); - region_incommon.commit(batchUpdate); - - if (flush) {region_incommon.flushcache();} - - // call delete all at a timestamp, make sure only the most recent stuff is left behind - region.deleteAll(row, now, null); - if (flush) {region_incommon.flushcache();} - assertCellEquals(region, row, colA, future, cellData(2, flush)); - assertCellEquals(region, row, colA, past, null); - assertCellEquals(region, row, colA, now, null); - assertCellEquals(region, row, colD, future, cellData(2, flush)); - assertCellEquals(region, row, colD, past, null); - assertCellEquals(region, row, colD, now, null); - - // call delete all w/o a timestamp, make sure nothing is left. - region.deleteAll(row, HConstants.LATEST_TIMESTAMP, null); - if (flush) {region_incommon.flushcache();} - assertCellEquals(region, row, colA, now, null); - assertCellEquals(region, row, colA, past, null); - assertCellEquals(region, row, colA, future, null); - assertCellEquals(region, row, colD, now, null); - assertCellEquals(region, row, colD, past, null); - assertCellEquals(region, row, colD, future, null); - - } - - private void makeSureRegexWorks(HRegion region, HRegionIncommon region_incommon, - boolean flush) - throws Exception{ - // insert a few versions worth of data for a row - byte [] row = Bytes.toBytes("test_row"); - long t0 = System.currentTimeMillis(); - long t1 = t0 - 15000; - long t2 = t1 - 15000; - - byte [] colA = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a"); - byte [] colB = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b"); - byte [] colC = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "c"); - byte [] colD = Bytes.toBytes(Bytes.toString(COLUMNS[0])); - - BatchUpdate batchUpdate = new BatchUpdate(row, t0); - batchUpdate.put(colA, cellData(0, flush).getBytes()); - batchUpdate.put(colB, cellData(0, flush).getBytes()); - batchUpdate.put(colC, cellData(0, flush).getBytes()); - batchUpdate.put(colD, cellData(0, flush).getBytes()); - region_incommon.commit(batchUpdate); - - batchUpdate = new BatchUpdate(row, t1); - batchUpdate.put(colA, cellData(1, flush).getBytes()); - batchUpdate.put(colB, cellData(1, flush).getBytes()); - batchUpdate.put(colC, cellData(1, flush).getBytes()); - batchUpdate.put(colD, cellData(1, flush).getBytes()); - region_incommon.commit(batchUpdate); - - batchUpdate = new BatchUpdate(row, t2); - batchUpdate.put(colA, cellData(2, flush).getBytes()); - batchUpdate.put(colB, cellData(2, flush).getBytes()); - batchUpdate.put(colC, cellData(2, flush).getBytes()); - batchUpdate.put(colD, cellData(2, flush).getBytes()); - region_incommon.commit(batchUpdate); - - if (flush) {region_incommon.flushcache();} - - // call delete the matching columns at a timestamp, - // make sure only the most recent stuff is left behind - region.deleteAllByRegex(row, COLUMN_REGEX, t1, null); - if (flush) {region_incommon.flushcache();} - assertCellEquals(region, row, colA, t0, cellData(0, flush)); - assertCellEquals(region, row, colA, t1, cellData(1, flush)); - assertCellEquals(region, row, colA, t2, cellData(2, flush)); - assertCellEquals(region, row, colB, t0, cellData(0, flush)); - assertCellEquals(region, row, colB, t1, null); - assertCellEquals(region, row, colB, t2, null); - assertCellEquals(region, row, colC, t0, cellData(0, flush)); - assertCellEquals(region, row, colC, t1, null); - assertCellEquals(region, row, colC, t2, null); - assertCellEquals(region, row, colD, t0, cellData(0, flush)); - assertCellEquals(region, row, colD, t1, null); - assertCellEquals(region, row, colD, t2, null); - - // call delete all w/o a timestamp, make sure nothing is left. - region.deleteAllByRegex(row, COLUMN_REGEX, - HConstants.LATEST_TIMESTAMP, null); - if (flush) {region_incommon.flushcache();} - assertCellEquals(region, row, colA, t0, cellData(0, flush)); - assertCellEquals(region, row, colA, t1, cellData(1, flush)); - assertCellEquals(region, row, colA, t2, cellData(2, flush)); - assertCellEquals(region, row, colB, t0, null); - assertCellEquals(region, row, colB, t1, null); - assertCellEquals(region, row, colB, t2, null); - assertCellEquals(region, row, colC, t0, null); - assertCellEquals(region, row, colC, t1, null); - assertCellEquals(region, row, colC, t2, null); - assertCellEquals(region, row, colD, t0, null); - assertCellEquals(region, row, colD, t1, null); - assertCellEquals(region, row, colD, t2, null); - - } - - private String cellData(int tsNum, boolean flush){ - return "t" + tsNum + " data" + (flush ? " - with flush" : ""); - } - - @Override - protected void tearDown() throws Exception { - if (this.miniHdfs != null) { - shutdownDfs(this.miniHdfs); - } - super.tearDown(); - } -} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteCompare.java b/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteCompare.java new file mode 100644 index 00000000000..d08941214e1 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteCompare.java @@ -0,0 +1,191 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueTestUtil; +import org.apache.hadoop.hbase.regionserver.DeleteCompare.DeleteCode; +import org.apache.hadoop.hbase.util.Bytes; + +import junit.framework.TestCase; + +public class TestDeleteCompare extends TestCase { + + //Cases to compare: + //1. DeleteFamily and whatever of the same row + //2. DeleteColumn and whatever of the same row + qualifier + //3. Delete and the matching put + //4. Big test that include starting on the wrong row and qualifier + public void testDeleteCompare_DeleteFamily() { + //Creating memcache + Set memcache = new TreeSet(KeyValue.COMPARATOR); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c")); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c")); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c")); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col2", 1, "d-c")); + + memcache.add(KeyValueTestUtil.create("row11", "fam", "col3", 3, "d-c")); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col3", 2, "d-c")); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col3", 1, "d-c")); + + memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 1, "d-c")); + + //Creating expected result + List expected = new ArrayList(); + expected.add(DeleteCode.SKIP); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.SKIP); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.DONE); + + KeyValue delete = KeyValueTestUtil.create("row11", + "fam", "", 2, KeyValue.Type.DeleteFamily, "dont-care"); + byte [] deleteBuffer = delete.getBuffer(); + int deleteRowOffset = delete.getRowOffset(); + short deleteRowLen = delete.getRowLength(); + int deleteQualifierOffset = delete.getQualifierOffset(); + int deleteQualifierLen = delete.getQualifierLength(); + int deleteTimestampOffset = deleteQualifierOffset + deleteQualifierLen; + byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG]; + + List actual = new ArrayList(); + for(KeyValue mem : memcache){ + actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset, + deleteRowLen, deleteQualifierOffset, deleteQualifierLen, + deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR)); + + } + + assertEquals(expected.size(), actual.size()); + for(int i=0; i memcache = new TreeSet(KeyValue.COMPARATOR); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c")); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c")); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c")); + memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 1, "d-c")); + + + //Creating expected result + List expected = new ArrayList(); + expected.add(DeleteCode.SKIP); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.DONE); + + KeyValue delete = KeyValueTestUtil.create("row11", "fam", "col1", 2, + KeyValue.Type.DeleteColumn, "dont-care"); + byte [] deleteBuffer = delete.getBuffer(); + int deleteRowOffset = delete.getRowOffset(); + short deleteRowLen = delete.getRowLength(); + int deleteQualifierOffset = delete.getQualifierOffset(); + int deleteQualifierLen = delete.getQualifierLength(); + int deleteTimestampOffset = deleteQualifierOffset + deleteQualifierLen; + byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG]; + + List actual = new ArrayList(); + for(KeyValue mem : memcache){ + actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset, + deleteRowLen, deleteQualifierOffset, deleteQualifierLen, + deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR)); + + } + + assertEquals(expected.size(), actual.size()); + for(int i=0; i memcache = new TreeSet(KeyValue.COMPARATOR); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c")); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c")); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c")); + + //Creating expected result + List expected = new ArrayList(); + expected.add(DeleteCode.SKIP); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.DONE); + + KeyValue delete = KeyValueTestUtil.create("row11", "fam", "col1", 2, + KeyValue.Type.Delete, "dont-care"); + byte [] deleteBuffer = delete.getBuffer(); + int deleteRowOffset = delete.getRowOffset(); + short deleteRowLen = delete.getRowLength(); + int deleteQualifierOffset = delete.getQualifierOffset(); + int deleteQualifierLen = delete.getQualifierLength(); + int deleteTimestampOffset = deleteQualifierOffset + deleteQualifierLen; + byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG]; + + List actual = new ArrayList(); + for(KeyValue mem : memcache){ + actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset, + deleteRowLen, deleteQualifierOffset, deleteQualifierLen, + deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR)); + } + + assertEquals(expected.size(), actual.size()); + for(int i=0; i memcache = new TreeSet(KeyValue.COMPARATOR); + memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c")); + memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 4, "d-c")); + memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 3, "d-c")); + memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 2, "d-c")); + memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 1, + KeyValue.Type.Delete, "dont-care")); + memcache.add(KeyValueTestUtil.create("row31", "fam", "col1", 1, "dont-care")); + + //Creating expected result + List expected = new ArrayList(); + expected.add(DeleteCode.SKIP); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.DELETE); + expected.add(DeleteCode.DONE); + + KeyValue delete = KeyValueTestUtil.create("row21", "fam", "col1", 5, + KeyValue.Type.DeleteColumn, "dont-care"); + byte [] deleteBuffer = delete.getBuffer(); + int deleteRowOffset = delete.getRowOffset(); + short deleteRowLen = delete.getRowLength(); + int deleteQualifierOffset = delete.getQualifierOffset(); + int deleteQualifierLen = delete.getQualifierLength(); + int deleteTimestampOffset = deleteQualifierOffset + deleteQualifierLen; + byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG]; + + List actual = new ArrayList(); + for(KeyValue mem : memcache){ + actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset, + deleteRowLen, deleteQualifierOffset, deleteQualifierLen, + deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR)); + + } + + assertEquals(expected.size(), actual.size()); + for(int i=0; i columns = new TreeSet(Bytes.BYTES_COMPARATOR); + //Looking for every other + columns.add(col2); + columns.add(col4); + List expected = new ArrayList(); + expected.add(MatchCode.SKIP); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.SKIP); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.DONE); + int maxVersions = 1; + + ColumnTracker exp = new ExplicitColumnTracker(columns, maxVersions); + + //Create "Scanner" + List scanner = new ArrayList(); + scanner.add(col1); + scanner.add(col2); + scanner.add(col3); + scanner.add(col4); + scanner.add(col5); + + //Initialize result + List result = new ArrayList(); + + //"Match" + for(byte [] col : scanner){ + result.add(exp.checkColumn(col, 0, col.length)); + } + + assertEquals(expected.size(), result.size()); + for(int i=0; i< expected.size(); i++){ + assertEquals(expected.get(i), result.get(i)); + if(PRINT){ + System.out.println("Expected " +expected.get(i) + ", actual " + + result.get(i)); + } + } + } + + public void testGet_MultiVersion(){ + if(PRINT){ + System.out.println("\nMultiVersion"); + } + byte [] col1 = Bytes.toBytes("col1"); + byte [] col2 = Bytes.toBytes("col2"); + byte [] col3 = Bytes.toBytes("col3"); + byte [] col4 = Bytes.toBytes("col4"); + byte [] col5 = Bytes.toBytes("col5"); + + //Create tracker + TreeSet columns = new TreeSet(Bytes.BYTES_COMPARATOR); + //Looking for every other + columns.add(col2); + columns.add(col4); + + List expected = new ArrayList(); + expected.add(MatchCode.SKIP); + expected.add(MatchCode.SKIP); + expected.add(MatchCode.SKIP); + + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.SKIP); + + expected.add(MatchCode.SKIP); + expected.add(MatchCode.SKIP); + expected.add(MatchCode.SKIP); + + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.DONE); + + expected.add(MatchCode.DONE); + expected.add(MatchCode.DONE); + expected.add(MatchCode.DONE); + int maxVersions = 2; + + ColumnTracker exp = new ExplicitColumnTracker(columns, maxVersions); + + //Create "Scanner" + List scanner = new ArrayList(); + scanner.add(col1); + scanner.add(col1); + scanner.add(col1); + scanner.add(col2); + scanner.add(col2); + scanner.add(col2); + scanner.add(col3); + scanner.add(col3); + scanner.add(col3); + scanner.add(col4); + scanner.add(col4); + scanner.add(col4); + scanner.add(col5); + scanner.add(col5); + scanner.add(col5); + + //Initialize result + List result = new ArrayList(); + + //"Match" + for(byte [] col : scanner){ + result.add(exp.checkColumn(col, 0, col.length)); + } + + assertEquals(expected.size(), result.size()); + for(int i=0; i< expected.size(); i++){ + assertEquals(expected.get(i), result.get(i)); + if(PRINT){ + System.out.println("Expected " +expected.get(i) + ", actual " + + result.get(i)); + } + } + } + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java b/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java deleted file mode 100644 index aef0cc4e91e..00000000000 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java +++ /dev/null @@ -1,168 +0,0 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver; - -import java.io.IOException; -import java.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hbase.HBaseTestCase; - -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Writables; - -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.BatchUpdate; - -/** Test case for get */ -public class TestGet extends HBaseTestCase { - private static final Log LOG = LogFactory.getLog(TestGet.class.getName()); - - 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"; - - - - 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); - } - } - } - - /** - * the test - * @throws IOException - */ - public void testGet() throws IOException { - MiniDFSCluster cluster = null; - HRegion region = null; - - try { - - // Initialization - - cluster = new MiniDFSCluster(conf, 2, true, (String[])null); - // Set the hbase.rootdir to be the home directory in mini dfs. - this.conf.set(HConstants.HBASE_DIR, - cluster.getFileSystem().getHomeDirectory().toString()); - - 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. - - 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(); - } - if (cluster != null) { - shutdownDfs(cluster); - } - } - } -} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java b/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java deleted file mode 100644 index 44d7823a809..00000000000 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java +++ /dev/null @@ -1,721 +0,0 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.NavigableSet; -import java.util.TreeSet; - -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.KeyValue; -import org.apache.hadoop.hbase.filter.StopRowFilter; -import org.apache.hadoop.hbase.filter.WhileMatchRowFilter; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hdfs.MiniDFSCluster; - -/** - * {@link TestGet} is a medley of tests of get all done up as a single test. - * This class - */ -public class TestGet2 extends HBaseTestCase implements HConstants { - private MiniDFSCluster miniHdfs; - - private static final String T00 = "000"; - private static final String T10 = "010"; - private static final String T11 = "011"; - private static final String T12 = "012"; - private static final String T20 = "020"; - private static final String T30 = "030"; - private static final String T31 = "031"; - - @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()); - } - - - public void testGetFullMultiMapfile() throws IOException { - HRegion region = null; - BatchUpdate batchUpdate = null; - Map results = null; - - try { - HTableDescriptor htd = createTableDescriptor(getName()); - region = createNewHRegion(htd, null, null); - - // Test ordering issue - // - byte [] row = Bytes.toBytes("row1"); - - // write some data - batchUpdate = new BatchUpdate(row); - batchUpdate.put(COLUMNS[0], "olderValue".getBytes()); - region.batchUpdate(batchUpdate, null); - - // flush - region.flushcache(); - - // assert that getFull gives us the older value - results = region.getFull(row, (NavigableSet)null, - LATEST_TIMESTAMP, 1, null); - assertEquals("olderValue", - new String(results.get(COLUMNS[0]).getValue())); - - // write a new value for the cell - batchUpdate = new BatchUpdate(row); - batchUpdate.put(COLUMNS[0], "newerValue".getBytes()); - region.batchUpdate(batchUpdate, null); - - // flush - region.flushcache(); - - // assert that getFull gives us the later value - results = region.getFull(row, (NavigableSet)null, - LATEST_TIMESTAMP, 1, null); - assertEquals("newerValue", new String(results.get(COLUMNS[0]).getValue())); - - // - // Test the delete masking issue - // - byte [] row2 = Bytes.toBytes("row2"); - byte [] cell1 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a"); - byte [] cell2 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b"); - byte [] cell3 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "c"); - - long now = System.currentTimeMillis(); - - // write some data at two columns - batchUpdate = new BatchUpdate(row2, now); - batchUpdate.put(cell1, "column0 value".getBytes()); - batchUpdate.put(cell2, "column1 value".getBytes()); - region.batchUpdate(batchUpdate, null); - - // flush - region.flushcache(); - - // assert i get both columns - results = region.getFull(row2, - (NavigableSet)null, LATEST_TIMESTAMP, 1, null); - assertEquals("Should have two columns in the results map", 2, results.size()); - assertEquals("column0 value", new String(results.get(cell1).getValue())); - assertEquals("column1 value", new String(results.get(cell2).getValue())); - - // write a delete for the first column - batchUpdate = new BatchUpdate(row2, now); - batchUpdate.delete(cell1); - batchUpdate.put(cell2, "column1 new value".getBytes()); - region.batchUpdate(batchUpdate, null); - - // flush - region.flushcache(); - - // assert i get the second column only - results = region.getFull(row2, (NavigableSet)null, - LATEST_TIMESTAMP, 1, null); - System.out.println(Bytes.toString(results.keySet().iterator().next())); - assertEquals("Should have one column in the results map", 1, results.size()); - assertNull("column0 value", results.get(cell1)); - assertEquals("column1 new value", new String(results.get(cell2).getValue())); - - // - // Include a delete and value from the memcache in the mix - // - batchUpdate = new BatchUpdate(row2, now); - batchUpdate.delete(cell2); - batchUpdate.put(cell3, "column3 value!".getBytes()); - region.batchUpdate(batchUpdate, null); - - // assert i get the third column only - results = region.getFull(row2, (NavigableSet)null, LATEST_TIMESTAMP, 1, null); - assertEquals("Should have one column in the results map", 1, results.size()); - assertNull("column0 value", results.get(cell1)); - assertNull("column1 value", results.get(cell2)); - assertEquals("column3 value!", new String(results.get(cell3).getValue())); - - } finally { - if (region != null) { - try { - region.close(); - } catch (Exception e) { - e.printStackTrace(); - } - region.getLog().closeAndDelete(); - } - } - } - - /** - * Test for HBASE-808 and HBASE-809. - * @throws Exception - */ - public void testMaxVersionsAndDeleting() throws Exception { - HRegion region = null; - try { - HTableDescriptor htd = createTableDescriptor(getName()); - region = createNewHRegion(htd, null, null); - - byte [] column = COLUMNS[0]; - for (int i = 0; i < 100; i++) { - addToRow(region, T00, column, i, T00.getBytes()); - } - checkVersions(region, T00, column); - // Flush and retry. - region.flushcache(); - checkVersions(region, T00, column); - - // Now delete all then retry - region.deleteAll(Bytes.toBytes(T00), System.currentTimeMillis(), null); - Cell [] cells = Cell.createSingleCellArray(region.get(Bytes.toBytes(T00), column, -1, - HColumnDescriptor.DEFAULT_VERSIONS)); - assertTrue(cells == null); - region.flushcache(); - cells = Cell.createSingleCellArray(region.get(Bytes.toBytes(T00), column, -1, - HColumnDescriptor.DEFAULT_VERSIONS)); - assertTrue(cells == null); - - // Now add back the rows - for (int i = 0; i < 100; i++) { - addToRow(region, T00, column, i, T00.getBytes()); - } - // Run same verifications. - checkVersions(region, T00, column); - // Flush and retry. - region.flushcache(); - checkVersions(region, T00, column); - } finally { - if (region != null) { - try { - region.close(); - } catch (Exception e) { - e.printStackTrace(); - } - region.getLog().closeAndDelete(); - } - } - } - - /** For HBASE-694 - * @throws IOException - */ - public void testGetClosestRowBefore2() throws IOException { - - HRegion region = null; - BatchUpdate batchUpdate = null; - - try { - HTableDescriptor htd = createTableDescriptor(getName()); - region = createNewHRegion(htd, null, null); - - // set up some test data - String t10 = "010"; - String t20 = "020"; - String t30 = "030"; - String t40 = "040"; - - batchUpdate = new BatchUpdate(t10); - batchUpdate.put(COLUMNS[0], "t10 bytes".getBytes()); - region.batchUpdate(batchUpdate, null); - - batchUpdate = new BatchUpdate(t30); - batchUpdate.put(COLUMNS[0], "t30 bytes".getBytes()); - region.batchUpdate(batchUpdate, null); - - batchUpdate = new BatchUpdate(t40); - batchUpdate.put(COLUMNS[0], "t40 bytes".getBytes()); - region.batchUpdate(batchUpdate, null); - - // try finding "035" - String t35 = "035"; - Map results = - region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes"); - - region.flushcache(); - - // try finding "035" - results = region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes"); - - batchUpdate = new BatchUpdate(t20); - batchUpdate.put(COLUMNS[0], "t20 bytes".getBytes()); - region.batchUpdate(batchUpdate, null); - - // try finding "035" - results = region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes"); - - region.flushcache(); - - // try finding "035" - results = region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes"); - } finally { - if (region != null) { - try { - region.close(); - } catch (Exception e) { - e.printStackTrace(); - } - region.getLog().closeAndDelete(); - } - } - } - - private void addToRow(final HRegion r, final String row, final byte [] column, - final long ts, final byte [] bytes) - throws IOException { - BatchUpdate batchUpdate = new BatchUpdate(row, ts); - batchUpdate.put(column, bytes); - r.batchUpdate(batchUpdate, null); - } - - private void checkVersions(final HRegion region, final String row, - final byte [] column) - throws IOException { - byte [] r = Bytes.toBytes(row); - Cell [] cells = Cell.createSingleCellArray(region.get(r, column, -1, 100)); - assertTrue(cells.length == HColumnDescriptor.DEFAULT_VERSIONS); - cells = Cell.createSingleCellArray(region.get(r, column, -1, 1)); - assertTrue(cells.length == 1); - cells = Cell.createSingleCellArray(region.get(r, column, -1, 10000)); - assertTrue(cells.length == HColumnDescriptor.DEFAULT_VERSIONS); - } - - /** - * Test file of multiple deletes and with deletes as final key. - * @throws IOException - * @see HBASE-751 - */ - public void testGetClosestRowBefore3() throws IOException { - HRegion region = null; - BatchUpdate batchUpdate = null; - try { - HTableDescriptor htd = createTableDescriptor(getName()); - region = createNewHRegion(htd, null, null); - - batchUpdate = new BatchUpdate(T00); - batchUpdate.put(COLUMNS[0], T00.getBytes()); - region.batchUpdate(batchUpdate, null); - - batchUpdate = new BatchUpdate(T10); - batchUpdate.put(COLUMNS[0], T10.getBytes()); - region.batchUpdate(batchUpdate, null); - - batchUpdate = new BatchUpdate(T20); - batchUpdate.put(COLUMNS[0], T20.getBytes()); - region.batchUpdate(batchUpdate, null); - - Map results = - region.getClosestRowBefore(Bytes.toBytes(T20), COLUMNS[0]); - assertEquals(T20, new String(results.get(COLUMNS[0]).getValue())); - - batchUpdate = new BatchUpdate(T20); - batchUpdate.delete(COLUMNS[0]); - region.batchUpdate(batchUpdate, null); - - results = region.getClosestRowBefore(Bytes.toBytes(T20), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - - batchUpdate = new BatchUpdate(T30); - batchUpdate.put(COLUMNS[0], T30.getBytes()); - region.batchUpdate(batchUpdate, null); - - results = region.getClosestRowBefore(Bytes.toBytes(T30), COLUMNS[0]); - assertEquals(T30, new String(results.get(COLUMNS[0]).getValue())); - - batchUpdate = new BatchUpdate(T30); - batchUpdate.delete(COLUMNS[0]); - region.batchUpdate(batchUpdate, null); - - results = region.getClosestRowBefore(Bytes.toBytes(T30), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - results = region.getClosestRowBefore(Bytes.toBytes(T31), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - - region.flushcache(); - - // try finding "010" after flush - results = region.getClosestRowBefore(Bytes.toBytes(T30), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - results = region.getClosestRowBefore(Bytes.toBytes(T31), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - - // Put into a different column family. Should make it so I still get t10 - batchUpdate = new BatchUpdate(T20); - batchUpdate.put(COLUMNS[1], T20.getBytes()); - region.batchUpdate(batchUpdate, null); - - results = region.getClosestRowBefore(Bytes.toBytes(T30), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - results = region.getClosestRowBefore(Bytes.toBytes(T31), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - region.flushcache(); - results = region.getClosestRowBefore(Bytes.toBytes(T30), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - results = region.getClosestRowBefore(Bytes.toBytes(T31), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - - // Now try combo of memcache and mapfiles. Delete the t20 COLUMS[1] - // in memory; make sure we get back t10 again. - batchUpdate = new BatchUpdate(T20); - batchUpdate.delete(COLUMNS[1]); - region.batchUpdate(batchUpdate, null); - results = region.getClosestRowBefore(Bytes.toBytes(T30), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - - // Ask for a value off the end of the file. Should return t10. - results = region.getClosestRowBefore(Bytes.toBytes(T31), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - region.flushcache(); - results = region.getClosestRowBefore(Bytes.toBytes(T31), COLUMNS[0]); - assertEquals(T10, new String(results.get(COLUMNS[0]).getValue())); - - // Ok. Let the candidate come out of mapfiles but have delete of - // the candidate be in memory. - batchUpdate = new BatchUpdate(T11); - batchUpdate.put(COLUMNS[0], T11.getBytes()); - region.batchUpdate(batchUpdate, null); - batchUpdate = new BatchUpdate(T10); - batchUpdate.delete(COLUMNS[0]); - region.batchUpdate(batchUpdate, null); - results = region.getClosestRowBefore(Bytes.toBytes(T12), COLUMNS[0]); - assertEquals(T11, new String(results.get(COLUMNS[0]).getValue())); - } finally { - if (region != null) { - try { - region.close(); - } catch (Exception e) { - e.printStackTrace(); - } - region.getLog().closeAndDelete(); - } - } - } - - /** - * Tests for HADOOP-2161. - * @throws Exception - */ - public void testGetFull() throws Exception { - HRegion region = null; - InternalScanner scanner = null; - try { - HTableDescriptor htd = createTableDescriptor(getName()); - region = createNewHRegion(htd, null, null); - for (int i = 0; i < COLUMNS.length; i++) { - addContent(region, COLUMNS[i]); - } - // Find two rows to use doing getFull. - final byte [] arbitraryStartRow = Bytes.toBytes("b"); - byte [] actualStartRow = null; - final byte [] arbitraryStopRow = Bytes.toBytes("c"); - byte [] actualStopRow = null; - byte [][] columns = {COLFAMILY_NAME1}; - scanner = region.getScanner(columns, - arbitraryStartRow, HConstants.LATEST_TIMESTAMP, - new WhileMatchRowFilter(new StopRowFilter(arbitraryStopRow))); - List value = new ArrayList(); - while (scanner.next(value)) { - if (actualStartRow == null) { - actualStartRow = value.get(0).getRow(); - } else { - actualStopRow = value.get(0).getRow(); - } - } - // Assert I got all out. - assertColumnsPresent(region, actualStartRow); - assertColumnsPresent(region, actualStopRow); - // Force a flush so store files come into play. - region.flushcache(); - // Assert I got all out. - assertColumnsPresent(region, actualStartRow); - assertColumnsPresent(region, actualStopRow); - } finally { - if (scanner != null) { - scanner.close(); - } - if (region != null) { - try { - region.close(); - } catch (Exception e) { - e.printStackTrace(); - } - region.getLog().closeAndDelete(); - } - } - } - - /** - * @throws IOException - */ - public void testGetAtTimestamp() throws IOException{ - HRegion region = null; - HRegionIncommon region_incommon = null; - try { - HTableDescriptor htd = createTableDescriptor(getName()); - region = createNewHRegion(htd, null, null); - region_incommon = new HRegionIncommon(region); - - long right_now = System.currentTimeMillis(); - long one_second_ago = right_now - 1000; - - String t = "test_row"; - BatchUpdate batchUpdate = new BatchUpdate(t, one_second_ago); - batchUpdate.put(COLUMNS[0], "old text".getBytes()); - region_incommon.commit(batchUpdate); - - batchUpdate = new BatchUpdate(t, right_now); - batchUpdate.put(COLUMNS[0], "new text".getBytes()); - region_incommon.commit(batchUpdate); - - assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0], - right_now, "new text"); - assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0], - one_second_ago, "old text"); - - // Force a flush so store files come into play. - region_incommon.flushcache(); - - assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0], right_now, "new text"); - assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0], one_second_ago, "old text"); - - } finally { - if (region != null) { - try { - region.close(); - } catch (Exception e) { - e.printStackTrace(); - } - region.getLog().closeAndDelete(); - } - } - } - - /** - * For HADOOP-2443 - * @throws IOException - */ - public void testGetClosestRowBefore() throws IOException{ - - HRegion region = null; - BatchUpdate batchUpdate = null; - - try { - HTableDescriptor htd = createTableDescriptor(getName()); - region = createNewHRegion(htd, null, null); - - // set up some test data - String t10 = "010"; - String t20 = "020"; - String t30 = "030"; - String t35 = "035"; - String t40 = "040"; - - batchUpdate = new BatchUpdate(t10); - batchUpdate.put(COLUMNS[0], "t10 bytes".getBytes()); - region.batchUpdate(batchUpdate, null); - - batchUpdate = new BatchUpdate(t20); - batchUpdate.put(COLUMNS[0], "t20 bytes".getBytes()); - region.batchUpdate(batchUpdate, null); - - batchUpdate = new BatchUpdate(t30); - batchUpdate.put(COLUMNS[0], "t30 bytes".getBytes()); - region.batchUpdate(batchUpdate, null); - - batchUpdate = new BatchUpdate(t35); - batchUpdate.put(COLUMNS[0], "t35 bytes".getBytes()); - region.batchUpdate(batchUpdate, null); - - batchUpdate = new BatchUpdate(t35); - batchUpdate.delete(COLUMNS[0]); - region.batchUpdate(batchUpdate, null); - - batchUpdate = new BatchUpdate(t40); - batchUpdate.put(COLUMNS[0], "t40 bytes".getBytes()); - region.batchUpdate(batchUpdate, null); - - // try finding "015" - String t15 = "015"; - Map results = - region.getClosestRowBefore(Bytes.toBytes(t15), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t10 bytes"); - - // try "020", we should get that row exactly - results = region.getClosestRowBefore(Bytes.toBytes(t20), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t20 bytes"); - - // try "038", should skip deleted "035" and get "030" - String t38 = "038"; - results = region.getClosestRowBefore(Bytes.toBytes(t38), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes"); - - // try "050", should get stuff from "040" - String t50 = "050"; - results = region.getClosestRowBefore(Bytes.toBytes(t50), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t40 bytes"); - - // force a flush - region.flushcache(); - - // try finding "015" - results = region.getClosestRowBefore(Bytes.toBytes(t15), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t10 bytes"); - - // try "020", we should get that row exactly - results = region.getClosestRowBefore(Bytes.toBytes(t20), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t20 bytes"); - - // try "038", should skip deleted "035" and get "030" - results = region.getClosestRowBefore(Bytes.toBytes(t38), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes"); - - // try "050", should get stuff from "040" - results = region.getClosestRowBefore(Bytes.toBytes(t50), COLUMNS[0]); - assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t40 bytes"); - } finally { - if (region != null) { - try { - region.close(); - } catch (Exception e) { - e.printStackTrace(); - } - region.getLog().closeAndDelete(); - } - } - } - - /** - * For HBASE-40 - * @throws IOException - */ - public void testGetFullWithSpecifiedColumns() throws IOException { - HRegion region = null; - HRegionIncommon region_incommon = null; - try { - HTableDescriptor htd = createTableDescriptor(getName()); - region = createNewHRegion(htd, null, null); - region_incommon = new HRegionIncommon(region); - - // write a row with a bunch of columns - byte [] row = Bytes.toBytes("some_row"); - BatchUpdate bu = new BatchUpdate(row); - bu.put(COLUMNS[0], "column 0".getBytes()); - bu.put(COLUMNS[1], "column 1".getBytes()); - bu.put(COLUMNS[2], "column 2".getBytes()); - region.batchUpdate(bu, null); - - assertSpecifiedColumns(region, row); - // try it again with a cache flush to involve the store, not just the - // memcache. - region_incommon.flushcache(); - assertSpecifiedColumns(region, row); - - } finally { - if (region != null) { - try { - region.close(); - } catch (Exception e) { - e.printStackTrace(); - } - region.getLog().closeAndDelete(); - } - } - } - - private void assertSpecifiedColumns(final HRegion region, final byte [] row) - throws IOException { - TreeSet all = new TreeSet(Bytes.BYTES_COMPARATOR); - TreeSet one = new TreeSet(Bytes.BYTES_COMPARATOR); - TreeSet none = new TreeSet(Bytes.BYTES_COMPARATOR); - - all.add(COLUMNS[0]); - all.add(COLUMNS[1]); - all.add(COLUMNS[2]); - one.add(COLUMNS[0]); - - // make sure we get all of them with standard getFull - Map result = region.getFull(row, null, - HConstants.LATEST_TIMESTAMP, 1, null); - assertEquals(new String(result.get(COLUMNS[0]).getValue()), "column 0"); - assertEquals(new String(result.get(COLUMNS[1]).getValue()), "column 1"); - assertEquals(new String(result.get(COLUMNS[2]).getValue()), "column 2"); - - // try to get just one - result = region.getFull(row, one, HConstants.LATEST_TIMESTAMP, 1, null); - assertEquals(new String(result.get(COLUMNS[0]).getValue()), "column 0"); - assertNull(result.get(COLUMNS[1])); - assertNull(result.get(COLUMNS[2])); - - // try to get all of them (specified) - result = region.getFull(row, all, HConstants.LATEST_TIMESTAMP, 1, null); - assertEquals(new String(result.get(COLUMNS[0]).getValue()), "column 0"); - assertEquals(new String(result.get(COLUMNS[1]).getValue()), "column 1"); - assertEquals(new String(result.get(COLUMNS[2]).getValue()), "column 2"); - - // try to get none with empty column set - result = region.getFull(row, none, HConstants.LATEST_TIMESTAMP, 1, null); - assertNull(result.get(COLUMNS[0])); - assertNull(result.get(COLUMNS[1])); - assertNull(result.get(COLUMNS[2])); - } - - private void assertColumnsPresent(final HRegion r, final byte [] row) - throws IOException { - Map result = - r.getFull(row, null, HConstants.LATEST_TIMESTAMP, 1, null); - int columnCount = 0; - for (Map.Entry e: result.entrySet()) { - columnCount++; - byte [] column = e.getKey(); - boolean legitColumn = false; - for (int i = 0; i < COLUMNS.length; i++) { - // Assert value is same as row. This is 'nature' of the data added. - assertTrue(Bytes.equals(row, e.getValue().getValue())); - if (Bytes.equals(COLUMNS[i], column)) { - legitColumn = true; - break; - } - } - assertTrue("is legit column name", legitColumn); - } - assertEquals("count of columns", columnCount, COLUMNS.length); - } - - @Override - protected void tearDown() throws Exception { - if (this.miniHdfs != null) { - this.miniHdfs.shutdown(); - } - super.tearDown(); - } -} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestGetDeleteTracker.java b/src/test/org/apache/hadoop/hbase/regionserver/TestGetDeleteTracker.java new file mode 100644 index 00000000000..5314fa4b7b9 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestGetDeleteTracker.java @@ -0,0 +1,313 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.HBaseTestCase; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.regionserver.GetDeleteTracker.Delete; +import org.apache.hadoop.hbase.util.Bytes; + + +public class TestGetDeleteTracker extends HBaseTestCase implements HConstants { + + private final boolean PRINT = true; + + private byte [] col1 = null; + private byte [] col2 = null; + + private int col1Len = 0; + private int col2Len = 0; + + private byte [] empty = null; + + private long ts1 = 0L; + private long ts2 = 0L; + private long ts3 = 0L; + + + private Delete del10 = null; + private Delete del11 = null; + private Delete delQf10 = null; + private Delete delQf11 = null; + private Delete delFam10 = null; + + private Delete del20 = null; + private Delete del21 = null; + private Delete delQf20 = null; + private Delete delQf21 = null; + private Delete delFam20 = null; + + + private Delete del30 = null; + + GetDeleteTracker dt = null; + private byte del = KeyValue.Type.Delete.getCode(); + private byte delCol = KeyValue.Type.DeleteColumn.getCode(); + private byte delFam = KeyValue.Type.DeleteFamily.getCode(); + + protected void setUp() throws Exception { + dt = new GetDeleteTracker(KeyValue.KEY_COMPARATOR); + col1 = "col".getBytes(); + col2 = "col2".getBytes(); + col1Len = col1.length; + col2Len = col2.length; + + empty = new byte[0]; + + //ts1 + ts1 = System.nanoTime(); + del10 = dt.new Delete(col1, 0, col1Len, del, ts1); + del11 = dt.new Delete(col2, 0, col2Len, del, ts1); + delQf10 = dt.new Delete(col1, 0, col1Len, delCol, ts1); + delQf11 = dt.new Delete(col2, 0, col2Len, delCol, ts1); + delFam10 = dt.new Delete(empty, 0, 0, delFam, ts1); + + //ts2 + ts2 = System.nanoTime(); + del20 = dt.new Delete(col1, 0, col1Len, del, ts2); + del21 = dt.new Delete(col2, 0, col2Len, del, ts2); + delQf20 = dt.new Delete(col1, 0, col1Len, delCol, ts2); + delQf21 = dt.new Delete(col2, 0, col2Len, delCol, ts2); + delFam20 = dt.new Delete(empty, 0, 0, delFam, ts1); + + //ts3 + ts3 = System.nanoTime(); + del30 = dt.new Delete(col1, 0, col1Len, del, ts3); + } + + public void testUpdate_CompareDeletes() { + GetDeleteTracker.DeleteCompare res = null; + + + //Testing Delete and Delete + res = dt.compareDeletes(del10, del10); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_BOTH, res); + + //Testing Delete qf1 and Delete qf2 and <==> + res = dt.compareDeletes(del10, del11); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res); + res = dt.compareDeletes(del11, del10); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res); + + //Testing Delete ts1 and Delete ts2 and <==> + res = dt.compareDeletes(del10, del20); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res); + res = dt.compareDeletes(del20, del10); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res); + + + + //Testing DeleteColumn and DeleteColumn + res = dt.compareDeletes(delQf10, delQf10); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_BOTH, res); + + //Testing DeleteColumn qf1 and DeleteColumn qf2 and <==> + res = dt.compareDeletes(delQf10, delQf11); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res); + res = dt.compareDeletes(delQf11, delQf10); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res); + + //Testing DeleteColumn ts1 and DeleteColumn ts2 and <==> + res = dt.compareDeletes(delQf10, delQf20); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_BOTH, res); + res = dt.compareDeletes(delQf20, delQf10); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_BOTH, res); + + + + //Testing Delete and DeleteColumn and <==> + res = dt.compareDeletes(del10, delQf10); + assertEquals(DeleteTracker.DeleteCompare.NEXT_OLD, res); + res = dt.compareDeletes(delQf10, del10); + assertEquals(DeleteTracker.DeleteCompare.NEXT_NEW, res); + + //Testing Delete qf1 and DeleteColumn qf2 and <==> + res = dt.compareDeletes(del10, delQf11); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res); + res = dt.compareDeletes(delQf11, del10); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res); + + //Testing Delete qf2 and DeleteColumn qf1 and <==> + res = dt.compareDeletes(del11, delQf10); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res); + res = dt.compareDeletes(delQf10, del11); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res); + + //Testing Delete ts2 and DeleteColumn ts1 and <==> + res = dt.compareDeletes(del20, delQf10); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_OLD_NEXT_OLD, res); + res = dt.compareDeletes(delQf10, del20); + assertEquals(DeleteTracker.DeleteCompare.INCLUDE_NEW_NEXT_NEW, res); + + //Testing Delete ts1 and DeleteColumn ts2 and <==> + res = dt.compareDeletes(del10, delQf20); + assertEquals(DeleteTracker.DeleteCompare.NEXT_OLD, res); + res = dt.compareDeletes(delQf20, del10); + assertEquals(DeleteTracker.DeleteCompare.NEXT_NEW, res); + + } + + public void testUpdate(){ + //Building lists + List dels1 = new ArrayList(); + dels1.add(delQf10); + dels1.add(del21); + + List dels2 = new ArrayList(); + dels2.add(delFam10); + dels2.add(del30); + dels2.add(delQf20); + + List res = new ArrayList(); + res.add(del30); + res.add(delQf20); + res.add(del21); + + //Adding entries + for(Delete del : dels1){ + dt.add(del.buffer, del.qualifierOffset, del.qualifierLength, + del.timestamp, del.type); + } + + //update() + dt.update(); + + //Check deleteList + List delList = dt.deletes; + assertEquals(dels1.size(), delList.size()); + for(int i=0; i dels = new ArrayList(); + dels.add(delQf10); + dels.add(del21); + + //Adding entries + for(Delete del : dels){ + dt.add(del.buffer, del.qualifierOffset, del.qualifierLength, + del.timestamp, del.type); + } + + //update() + dt.update(); + + assertEquals(false, dt.isDeleted(col2, 0, col2Len, ts3)); + assertEquals(false, dt.isDeleted(col2, 0, col2Len, ts1)); + } + public void testIsDeleted_Delete(){ + //Building lists + List dels = new ArrayList(); + dels.add(del21); + + //Adding entries + for(Delete del : dels){ + dt.add(del.buffer, del.qualifierOffset, del.qualifierLength, + del.timestamp, del.type); + } + + //update() + dt.update(); + + assertEquals(true, dt.isDeleted(col2, 0, col2Len, ts2)); + } + + public void testIsDeleted_DeleteColumn(){ + //Building lists + List dels = new ArrayList(); + dels.add(delQf21); + + //Adding entries + for(Delete del : dels){ + dt.add(del.buffer, del.qualifierOffset, del.qualifierLength, + del.timestamp, del.type); + } + + //update() + dt.update(); + + assertEquals(true, dt.isDeleted(col2, 0, col2Len, ts1)); + } + + public void testIsDeleted_DeleteFamily(){ + //Building lists + List dels = new ArrayList(); + dels.add(delFam20); + + //Adding entries + for(Delete del : dels){ + dt.add(del.buffer, del.qualifierOffset, del.qualifierLength, + del.timestamp, del.type); + } + + //update() + dt.update(); + + assertEquals(true, dt.isDeleted(col2, 0, col2Len, ts1)); + } + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java b/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java index bd53e3c2996..cbbb574a35b 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java @@ -170,7 +170,7 @@ public class TestHLog extends HBaseTestCase implements HConstants { assertTrue(Bytes.equals(regionName, key.getRegionName())); assertTrue(Bytes.equals(tableName, key.getTablename())); assertTrue(Bytes.equals(HLog.METAROW, val.getRow())); - assertTrue(Bytes.equals(HLog.METACOLUMN, val.getColumn())); + assertTrue(Bytes.equals(HLog.METAFAMILY, val.getFamily())); assertEquals(0, Bytes.compareTo(HLog.COMPLETE_CACHE_FLUSH, val.getValue())); System.out.println(key + " " + val); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java b/src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java deleted file mode 100644 index 71a4206ba7d..00000000000 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java +++ /dev/null @@ -1,458 +0,0 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver; - -import java.io.IOException; -import java.rmi.UnexpectedException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.NavigableSet; -import java.util.Set; -import java.util.TreeMap; -import java.util.TreeSet; - -import junit.framework.TestCase; - -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.regionserver.HRegion.Counter; -import org.apache.hadoop.hbase.util.Bytes; - -/** memcache test case */ -public class TestHMemcache extends TestCase { - private Memcache hmemcache; - - private static final int ROW_COUNT = 10; - - private static final int COLUMNS_COUNT = 10; - - private static final String COLUMN_FAMILY = "column"; - - private static final int FIRST_ROW = 1; - private static final int NUM_VALS = 1000; - private static final byte [] CONTENTS_BASIC = Bytes.toBytes("contents:basic"); - private static final String CONTENTSTR = "contentstr"; - private static final String ANCHORNUM = "anchor:anchornum-"; - private static final String ANCHORSTR = "anchorstr"; - - @Override - public void setUp() throws Exception { - super.setUp(); - this.hmemcache = new Memcache(); - } - - public void testGetWithDeletes() throws IOException { - Memcache mc = new Memcache(HConstants.FOREVER, KeyValue.ROOT_COMPARATOR); - final int start = 0; - final int end = 5; - long now = System.currentTimeMillis(); - for (int k = start; k <= end; k++) { - byte [] row = Bytes.toBytes(k); - KeyValue key = new KeyValue(row, CONTENTS_BASIC, now, - (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING)); - mc.add(key); - System.out.println(key); - key = new KeyValue(row, Bytes.toBytes(ANCHORNUM + k), now, - (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING)); - mc.add(key); - System.out.println(key); - } - KeyValue key = new KeyValue(Bytes.toBytes(start), CONTENTS_BASIC, now); - List keys = mc.get(key, 1); - assertTrue(keys.size() == 1); - KeyValue delete = key.cloneDelete(); - mc.add(delete); - keys = mc.get(delete, 1); - assertTrue(keys.size() == 0); - } - - public void testBinary() throws IOException { - Memcache mc = new Memcache(HConstants.FOREVER, KeyValue.ROOT_COMPARATOR); - final int start = 43; - final int end = 46; - for (int k = start; k <= end; k++) { - byte [] kk = Bytes.toBytes(k); - byte [] row = - Bytes.toBytes(".META.,table," + Bytes.toString(kk) + ",1," + k); - KeyValue key = new KeyValue(row, CONTENTS_BASIC, - System.currentTimeMillis(), - (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING)); - mc.add(key); - System.out.println(key); -// key = new KeyValue(row, Bytes.toBytes(ANCHORNUM + k), -// System.currentTimeMillis(), -// (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING)); -// mc.add(key); -// System.out.println(key); - } - int index = start; - for (KeyValue kv: mc.memcache) { - System.out.println(kv); - byte [] b = kv.getRow(); - // Hardcoded offsets into String - String str = Bytes.toString(b, 13, 4); - byte [] bb = Bytes.toBytes(index); - String bbStr = Bytes.toString(bb); - assertEquals(str, bbStr); - index++; - } - } - - /** - * @throws IOException - */ - public void testMemcache() throws IOException { - for (int k = FIRST_ROW; k <= NUM_VALS; k++) { - byte [] row = Bytes.toBytes("row_" + k); - KeyValue key = new KeyValue(row, CONTENTS_BASIC, - System.currentTimeMillis(), - (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING)); - hmemcache.add(key); - key = new KeyValue(row, Bytes.toBytes(ANCHORNUM + k), - System.currentTimeMillis(), - (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING)); - hmemcache.add(key); - } - // this.hmemcache.dump(); - - // Read them back - - for (int k = FIRST_ROW; k <= NUM_VALS; k++) { - List results; - byte [] row = Bytes.toBytes("row_" + k); - KeyValue key = new KeyValue(row, CONTENTS_BASIC, Long.MAX_VALUE); - results = hmemcache.get(key, 1); - assertNotNull("no data for " + key.toString(), results); - assertEquals(1, results.size()); - KeyValue kv = results.get(0); - String bodystr = Bytes.toString(kv.getBuffer(), kv.getValueOffset(), - kv.getValueLength()); - String teststr = CONTENTSTR + k; - assertTrue("Incorrect value for key: (" + key.toString() + - "), expected: '" + teststr + "' got: '" + - bodystr + "'", teststr.compareTo(bodystr) == 0); - - key = new KeyValue(row, Bytes.toBytes(ANCHORNUM + k), Long.MAX_VALUE); - results = hmemcache.get(key, 1); - assertNotNull("no data for " + key.toString(), results); - assertEquals(1, results.size()); - kv = results.get(0); - bodystr = Bytes.toString(kv.getBuffer(), kv.getValueOffset(), - kv.getValueLength()); - teststr = ANCHORSTR + k; - assertTrue("Incorrect value for key: (" + key.toString() + - "), expected: '" + teststr + "' got: '" + bodystr + "'", - teststr.compareTo(bodystr) == 0); - } - } - - private byte [] getRowName(final int index) { - return Bytes.toBytes("row" + Integer.toString(index)); - } - - private byte [] getColumnName(final int rowIndex, final int colIndex) { - return Bytes.toBytes(COLUMN_FAMILY + ":" + Integer.toString(rowIndex) + ";" + - Integer.toString(colIndex)); - } - - /** - * Adds {@link #ROW_COUNT} rows and {@link #COLUMNS_COUNT} - * @param hmc Instance to add rows to. - * @throws IOException - */ - private void addRows(final Memcache hmc) { - for (int i = 0; i < ROW_COUNT; i++) { - long timestamp = System.currentTimeMillis(); - for (int ii = 0; ii < COLUMNS_COUNT; ii++) { - byte [] k = getColumnName(i, ii); - hmc.add(new KeyValue(getRowName(i), k, timestamp, k)); - } - } - } - - private void runSnapshot(final Memcache hmc) throws UnexpectedException { - // Save off old state. - int oldHistorySize = hmc.getSnapshot().size(); - hmc.snapshot(); - Set ss = hmc.getSnapshot(); - // Make some assertions about what just happened. - assertTrue("History size has not increased", oldHistorySize < ss.size()); - hmc.clearSnapshot(ss); - } - - /** - * Test memcache snapshots - * @throws IOException - */ - public void testSnapshotting() throws IOException { - final int snapshotCount = 5; - // Add some rows, run a snapshot. Do it a few times. - for (int i = 0; i < snapshotCount; i++) { - addRows(this.hmemcache); - runSnapshot(this.hmemcache); - Set ss = this.hmemcache.getSnapshot(); - assertEquals("History not being cleared", 0, ss.size()); - } - } - - private void isExpectedRowWithoutTimestamps(final int rowIndex, - List kvs) { - int i = 0; - for (KeyValue kv: kvs) { - String expectedColname = Bytes.toString(getColumnName(rowIndex, i++)); - String colnameStr = kv.getColumnString(); - assertEquals("Column name", colnameStr, expectedColname); - // Value is column name as bytes. Usually result is - // 100 bytes in size at least. This is the default size - // for BytesWriteable. For comparison, convert bytes to - // String and trim to remove trailing null bytes. - String colvalueStr = Bytes.toString(kv.getBuffer(), kv.getValueOffset(), - kv.getValueLength()); - assertEquals("Content", colnameStr, colvalueStr); - } - } - - /** Test getFull from memcache - * @throws InterruptedException - */ - public void testGetFull() throws InterruptedException { - addRows(this.hmemcache); - Thread.sleep(1); - addRows(this.hmemcache); - Thread.sleep(1); - addRows(this.hmemcache); - Thread.sleep(1); - addRows(this.hmemcache); - long now = System.currentTimeMillis(); - Map versionCounter = - new TreeMap(this.hmemcache.comparatorIgnoreTimestamp); - for (int i = 0; i < ROW_COUNT; i++) { - KeyValue kv = new KeyValue(getRowName(i), now); - List all = new ArrayList(); - NavigableSet deletes = - new TreeSet(KeyValue.COMPARATOR); - this.hmemcache.getFull(kv, null, null, 1, versionCounter, deletes, all, - System.currentTimeMillis()); - isExpectedRowWithoutTimestamps(i, all); - } - // Test getting two versions. - versionCounter = - new TreeMap(this.hmemcache.comparatorIgnoreTimestamp); - for (int i = 0; i < ROW_COUNT; i++) { - KeyValue kv = new KeyValue(getRowName(i), now); - List all = new ArrayList(); - NavigableSet deletes = - new TreeSet(KeyValue.COMPARATOR); - this.hmemcache.getFull(kv, null, null, 2, versionCounter, deletes, all, - System.currentTimeMillis()); - byte [] previousRow = null; - int count = 0; - for (KeyValue k: all) { - if (previousRow != null) { - assertTrue(this.hmemcache.comparator.compareRows(k, previousRow) == 0); - } - previousRow = k.getRow(); - count++; - } - assertEquals(ROW_COUNT * 2, count); - } - } - - /** Test getNextRow from memcache - * @throws InterruptedException - */ - public void testGetNextRow() throws InterruptedException { - addRows(this.hmemcache); - // Add more versions to make it a little more interesting. - Thread.sleep(1); - addRows(this.hmemcache); - KeyValue closestToEmpty = this.hmemcache.getNextRow(KeyValue.LOWESTKEY); - assertTrue(KeyValue.COMPARATOR.compareRows(closestToEmpty, - new KeyValue(getRowName(0), System.currentTimeMillis())) == 0); - for (int i = 0; i < ROW_COUNT; i++) { - KeyValue nr = this.hmemcache.getNextRow(new KeyValue(getRowName(i), - System.currentTimeMillis())); - if (i + 1 == ROW_COUNT) { - assertEquals(nr, null); - } else { - assertTrue(KeyValue.COMPARATOR.compareRows(nr, - new KeyValue(getRowName(i + 1), System.currentTimeMillis())) == 0); - } - } - } - - /** Test getClosest from memcache - * @throws InterruptedException - */ - public void testGetClosest() throws InterruptedException { - addRows(this.hmemcache); - // Add more versions to make it a little more interesting. - Thread.sleep(1); - addRows(this.hmemcache); - KeyValue kv = this.hmemcache.getNextRow(KeyValue.LOWESTKEY); - assertTrue(KeyValue.COMPARATOR.compareRows(new KeyValue(getRowName(0), - System.currentTimeMillis()), kv) == 0); - for (int i = 0; i < ROW_COUNT; i++) { - KeyValue nr = this.hmemcache.getNextRow(new KeyValue(getRowName(i), - System.currentTimeMillis())); - if (i + 1 == ROW_COUNT) { - assertEquals(nr, null); - } else { - assertTrue(KeyValue.COMPARATOR.compareRows(nr, - new KeyValue(getRowName(i + 1), System.currentTimeMillis())) == 0); - } - } - } - - /** - * Test memcache scanner - * @throws IOException - * @throws InterruptedException - */ - public void testScanner() throws IOException, InterruptedException { - addRows(this.hmemcache); - Thread.sleep(1); - addRows(this.hmemcache); - Thread.sleep(1); - addRows(this.hmemcache); - long timestamp = System.currentTimeMillis(); - NavigableSet columns = new TreeSet(Bytes.BYTES_COMPARATOR); - for (int i = 0; i < ROW_COUNT; i++) { - for (int ii = 0; ii < COLUMNS_COUNT; ii++) { - columns.add(getColumnName(i, ii)); - } - } - InternalScanner scanner = - this.hmemcache.getScanner(timestamp, columns, HConstants.EMPTY_START_ROW); - List results = new ArrayList(); - for (int i = 0; scanner.next(results); i++) { - KeyValue.COMPARATOR.compareRows(results.get(0), getRowName(i)); - assertEquals("Count of columns", COLUMNS_COUNT, results.size()); - isExpectedRowWithoutTimestamps(i, results); - // Clear out set. Otherwise row results accumulate. - results.clear(); - } - } - - /** For HBASE-528 */ - public void testGetRowKeyAtOrBefore() { - // set up some test data - byte [] t10 = Bytes.toBytes("010"); - byte [] t20 = Bytes.toBytes("020"); - byte [] t30 = Bytes.toBytes("030"); - byte [] t35 = Bytes.toBytes("035"); - byte [] t40 = Bytes.toBytes("040"); - - hmemcache.add(getKV(t10, "t10 bytes".getBytes())); - hmemcache.add(getKV(t20, "t20 bytes".getBytes())); - hmemcache.add(getKV(t30, "t30 bytes".getBytes())); - hmemcache.add(getKV(t35, "t35 bytes".getBytes())); - // write a delete in there to see if things still work ok - hmemcache.add(getDeleteKV(t35)); - hmemcache.add(getKV(t40, "t40 bytes".getBytes())); - - NavigableSet results = null; - - // try finding "015" - results = - new TreeSet(this.hmemcache.comparator.getComparatorIgnoringType()); - KeyValue t15 = new KeyValue(Bytes.toBytes("015"), - System.currentTimeMillis()); - hmemcache.getRowKeyAtOrBefore(t15, results); - KeyValue kv = results.last(); - assertTrue(KeyValue.COMPARATOR.compareRows(kv, t10) == 0); - - // try "020", we should get that row exactly - results = - new TreeSet(this.hmemcache.comparator.getComparatorIgnoringType()); - hmemcache.getRowKeyAtOrBefore(new KeyValue(t20, System.currentTimeMillis()), - results); - assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t20) == 0); - - // try "030", we should get that row exactly - results = - new TreeSet(this.hmemcache.comparator.getComparatorIgnoringType()); - hmemcache.getRowKeyAtOrBefore(new KeyValue(t30, System.currentTimeMillis()), - results); - assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t30) == 0); - - // try "038", should skip the deleted "035" and give "030" - results = - new TreeSet(this.hmemcache.comparator.getComparatorIgnoringType()); - byte [] t38 = Bytes.toBytes("038"); - hmemcache.getRowKeyAtOrBefore(new KeyValue(t38, System.currentTimeMillis()), - results); - assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t30) == 0); - - // try "050", should get stuff from "040" - results = - new TreeSet(this.hmemcache.comparator.getComparatorIgnoringType()); - byte [] t50 = Bytes.toBytes("050"); - hmemcache.getRowKeyAtOrBefore(new KeyValue(t50, System.currentTimeMillis()), - results); - assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t40) == 0); - } - - private KeyValue getDeleteKV(byte [] row) { - return new KeyValue(row, Bytes.toBytes("test_col:"), - HConstants.LATEST_TIMESTAMP, KeyValue.Type.Delete, null); - } - - private KeyValue getKV(byte [] row, byte [] value) { - return new KeyValue(row, Bytes.toBytes("test_col:"), - HConstants.LATEST_TIMESTAMP, value); - } - - /** - * Test memcache scanner scanning cached rows, HBASE-686 - * @throws IOException - */ - public void testScanner_686() throws IOException { - addRows(this.hmemcache); - long timestamp = System.currentTimeMillis(); - NavigableSet cols = new TreeSet(Bytes.BYTES_COMPARATOR); - for (int i = 0; i < ROW_COUNT; i++) { - for (int ii = 0; ii < COLUMNS_COUNT; ii++) { - cols.add(getColumnName(i, ii)); - } - } - //starting from each row, validate results should contain the starting row - for (int startRowId = 0; startRowId < ROW_COUNT; startRowId++) { - InternalScanner scanner = this.hmemcache.getScanner(timestamp, - cols, getRowName(startRowId)); - List results = new ArrayList(); - for (int i = 0; scanner.next(results); i++) { - int rowId = startRowId + i; - assertTrue("Row name", - KeyValue.COMPARATOR.compareRows(results.get(0), - getRowName(rowId)) == 0); - assertEquals("Count of columns", COLUMNS_COUNT, results.size()); - List row = new ArrayList(); - for (KeyValue kv : results) { - row.add(kv); - } - isExpectedRowWithoutTimestamps(rowId, row); - // Clear out set. Otherwise row results accumulate. - results.clear(); - } - } - } -} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 945124079cd..e9087c18caa 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -21,21 +21,26 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; +import java.util.TreeMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; 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.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.UnknownScannerException; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.regionserver.HRegion.RegionScanner; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hdfs.MiniDFSCluster; /** * Basic stand-alone testing of HRegion. @@ -46,156 +51,390 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; public class TestHRegion extends HBaseTestCase { static final Log LOG = LogFactory.getLog(TestHRegion.class); - private static final int FIRST_ROW = 1; - private static final int NUM_VALS = 1000; - private static final String CONTENTS_BASIC_STR = "contents:basic"; - private static final byte [] CONTENTS_BASIC = Bytes.toBytes(CONTENTS_BASIC_STR); - private static final String CONTENTSTR = "contentstr"; - private static final String ANCHORNUM = "anchor:anchornum-"; - private static final String ANCHORSTR = "anchorstr"; - private static final byte [] CONTENTS_FIRSTCOL = Bytes.toBytes("contents:firstcol"); - private static final byte [] ANCHOR_SECONDCOL = Bytes.toBytes("anchor:secondcol"); + HRegion region = null; + private final String DIR = "test/build/data/TestHRegion/"; - private MiniDFSCluster cluster = null; - private HTableDescriptor desc = null; - HRegion r = null; - HRegionIncommon region = null; + private final int MAX_VERSIONS = 2; - private static int numInserted = 0; - /** * @see org.apache.hadoop.hbase.HBaseTestCase#setUp() */ @Override protected void setUp() throws Exception { - this.conf.set("hbase.hstore.compactionThreshold", "2"); - - conf.setLong("hbase.hregion.max.filesize", 65536); - - cluster = new MiniDFSCluster(conf, 2, true, (String[])null); - fs = cluster.getFileSystem(); - - // Set the hbase.rootdir to be the home directory in mini dfs. - this.conf.set(HConstants.HBASE_DIR, - this.cluster.getFileSystem().getHomeDirectory().toString()); - super.setUp(); } - /** - * Since all the "tests" depend on the results of the previous test, they are - * not Junit tests that can stand alone. Consequently we have a single Junit - * test that runs the "sub-tests" as private methods. - * @throws IOException - */ - public void testHRegion() throws IOException { - try { - init(); - locks(); - badPuts(); - basic(); - scan(); - splitAndMerge(); - read(); - } finally { - shutdownDfs(cluster); - } - } - - // Create directories, start mini cluster, etc. - private void init() throws IOException { - desc = new HTableDescriptor("test"); - desc.addFamily(new HColumnDescriptor("contents:")); - desc.addFamily(new HColumnDescriptor("anchor:")); - r = createNewHRegion(desc, null, null); - region = new HRegionIncommon(r); - LOG.info("setup completed."); + ////////////////////////////////////////////////////////////////////////////// + // New tests that doesn't spin up a mini cluster but rather just test the + // individual code pieces in the HRegion. Putting files locally in + // /tmp/testtable + ////////////////////////////////////////////////////////////////////////////// + + + ////////////////////////////////////////////////////////////////////////////// + // checkAndPut tests + ////////////////////////////////////////////////////////////////////////////// + public void testCheckAndPut_WithEmptyRowValue() throws IOException { + byte [] tableName = Bytes.toBytes("testtable"); + byte [] row1 = Bytes.toBytes("row1"); + byte [] fam1 = Bytes.toBytes("fam1"); + byte [] qf1 = Bytes.toBytes("qualifier"); + byte [] emptyVal = new byte[] {}; + byte [] val1 = Bytes.toBytes("value1"); + byte [] val2 = Bytes.toBytes("value2"); + Integer lockId = null; + + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, fam1); + //Putting data in key + Put put = new Put(row1); + put.add(fam1, qf1, val1); + + //checkAndPut with correct value + boolean res = region.checkAndPut(row1, fam1, qf1, emptyVal, put, lockId, + true); + assertTrue(res); + + // not empty anymore + res = region.checkAndPut(row1, fam1, qf1, emptyVal, put, lockId, true); + assertFalse(res); + + put = new Put(row1); + put.add(fam1, qf1, val2); + //checkAndPut with correct value + res = region.checkAndPut(row1, fam1, qf1, val1, put, lockId, true); + assertTrue(res); + } + + public void testCheckAndPut_WithWrongValue() throws IOException{ + byte [] tableName = Bytes.toBytes("testtable"); + byte [] row1 = Bytes.toBytes("row1"); + byte [] fam1 = Bytes.toBytes("fam1"); + byte [] qf1 = Bytes.toBytes("qualifier"); + byte [] val1 = Bytes.toBytes("value1"); + byte [] val2 = Bytes.toBytes("value2"); + Integer lockId = null; + + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, fam1); + + //Putting data in key + Put put = new Put(row1); + put.add(fam1, qf1, val1); + region.put(put); + + //checkAndPut with wrong value + boolean res = region.checkAndPut(row1, fam1, qf1, val2, put, lockId, true); + assertEquals(false, res); } - // Test basic functionality. Writes to contents:basic and anchor:anchornum-* + public void testCheckAndPut_WithCorrectValue() throws IOException{ + byte [] tableName = Bytes.toBytes("testtable"); + byte [] row1 = Bytes.toBytes("row1"); + byte [] fam1 = Bytes.toBytes("fam1"); + byte [] qf1 = Bytes.toBytes("qualifier"); + byte [] val1 = Bytes.toBytes("value1"); + Integer lockId = null; - private void basic() throws IOException { - long startTime = System.currentTimeMillis(); + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, fam1); - // Write out a bunch of values - for (int k = FIRST_ROW; k <= NUM_VALS; k++) { - BatchUpdate batchUpdate = - new BatchUpdate(Bytes.toBytes("row_" + k), System.currentTimeMillis()); - batchUpdate.put(CONTENTS_BASIC, - (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING)); - batchUpdate.put(Bytes.toBytes(ANCHORNUM + k), - (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING)); - region.commit(batchUpdate); + //Putting data in key + Put put = new Put(row1); + put.add(fam1, qf1, val1); + region.put(put); + + //checkAndPut with correct value + boolean res = region.checkAndPut(row1, fam1, qf1, val1, put, lockId, true); + assertEquals(true, res); + } + + public void testCheckAndPut_ThatPutWasWritten() throws IOException{ + byte [] tableName = Bytes.toBytes("testtable"); + byte [] row1 = Bytes.toBytes("row1"); + byte [] fam1 = Bytes.toBytes("fam1"); + byte [] fam2 = Bytes.toBytes("fam2"); + byte [] qf1 = Bytes.toBytes("qualifier"); + byte [] val1 = Bytes.toBytes("value1"); + byte [] val2 = Bytes.toBytes("value2"); + Integer lockId = null; + + byte [][] families = {fam1, fam2}; + + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, families); + + //Putting data in the key to check + Put put = new Put(row1); + put.add(fam1, qf1, val1); + region.put(put); + + //Creating put to add + long ts = System.currentTimeMillis(); + KeyValue kv = new KeyValue(row1, fam2, qf1, ts, KeyValue.Type.Put, val2); + put = new Put(row1); + put.add(kv); + + //checkAndPut with wrong value + Store store = region.getStore(fam1); + int size = store.memcache.memcache.size(); + + boolean res = region.checkAndPut(row1, fam1, qf1, val1, put, lockId, true); + assertEquals(true, res); + size = store.memcache.memcache.size(); + + Get get = new Get(row1); + get.addColumn(fam2, qf1); + KeyValue [] actual = region.get(get, null).raw(); + + KeyValue [] expected = {kv}; + + assertEquals(expected.length, actual.length); + for(int i=0; i kvs = new ArrayList(); + kvs.add(new KeyValue(row1, fam4, null, null)); + + //testing existing family + byte [] family = fam2; + try { + region.delete(family, kvs, true); + } catch (Exception e) { + assertTrue("Family " +new String(family)+ " does not exist", false); + } + + //testing non existing family + boolean ok = false; + family = fam4; + try { + region.delete(family, kvs, true); + } catch (Exception e) { + ok = true; + } + assertEquals("Family " +new String(family)+ " does exist", true, ok); + } + + //Visual test, since the method doesn't return anything + public void testDelete_CheckTimestampUpdated() + throws IOException { + byte [] tableName = Bytes.toBytes("testtable"); + byte [] row1 = Bytes.toBytes("row1"); + byte [] fam1 = Bytes.toBytes("fam1"); + byte [] col1 = Bytes.toBytes("col1"); + byte [] col2 = Bytes.toBytes("col2"); + byte [] col3 = Bytes.toBytes("col3"); + + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, fam1); + + //Building checkerList + List kvs = new ArrayList(); + kvs.add(new KeyValue(row1, fam1, col1, null)); + kvs.add(new KeyValue(row1, fam1, col2, null)); + kvs.add(new KeyValue(row1, fam1, col3, null)); + + region.delete(fam1, kvs, true); + } + + ////////////////////////////////////////////////////////////////////////////// + // Get tests + ////////////////////////////////////////////////////////////////////////////// + public void testGet_FamilyChecker() throws IOException { + byte [] tableName = Bytes.toBytes("testtable"); + byte [] row1 = Bytes.toBytes("row1"); + byte [] fam1 = Bytes.toBytes("fam1"); + byte [] fam2 = Bytes.toBytes("False"); + byte [] col1 = Bytes.toBytes("col1"); + + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, fam1); + + Get get = new Get(row1); + get.addColumn(fam2, col1); + + //Test + try { + region.get(get, null); + } catch (NoSuchColumnFamilyException e){ + assertFalse(false); + return; + } + assertFalse(true); + } + + public void testGet_Basic() throws IOException { + byte [] tableName = Bytes.toBytes("testtable"); + byte [] row1 = Bytes.toBytes("row1"); + byte [] fam1 = Bytes.toBytes("fam1"); + byte [] col1 = Bytes.toBytes("col1"); + byte [] col2 = Bytes.toBytes("col2"); + byte [] col3 = Bytes.toBytes("col3"); + byte [] col4 = Bytes.toBytes("col4"); + byte [] col5 = Bytes.toBytes("col5"); + + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, fam1); + + //Add to memcache + Put put = new Put(row1); + put.add(fam1, col1, null); + put.add(fam1, col2, null); + put.add(fam1, col3, null); + put.add(fam1, col4, null); + put.add(fam1, col5, null); + region.put(put); + + Get get = new Get(row1); + get.addColumn(fam1, col2); + get.addColumn(fam1, col4); + //Expected result + KeyValue kv1 = new KeyValue(row1, fam1, col2); + KeyValue kv2 = new KeyValue(row1, fam1, col4); + KeyValue [] expected = {kv1, kv2}; + + //Test + Result res = region.get(get, null); + + assertEquals(expected.length, res.size()); + for(int i=0; i result = new ArrayList(); + s.next(result); + + assertEquals(expected.length, result.size()); + for(int i=0; i= 0; i--) { - r.releaseRowLock(lockids[i]); + region.releaseRowLock(lockids[i]); LOG.debug(getName() + " unlocked " + i); } LOG.debug(getName() + " released " + @@ -247,409 +486,938 @@ public class TestHRegion extends HBaseTestCase { } LOG.info("locks completed."); } - - // Test scanners. Writes contents:firstcol and anchor:secondcol - private void scan() throws IOException { - byte [] cols [] = { - CONTENTS_FIRSTCOL, - ANCHOR_SECONDCOL - }; + ////////////////////////////////////////////////////////////////////////////// + // Merge test + ////////////////////////////////////////////////////////////////////////////// + public void testMerge() throws IOException { + byte [] tableName = Bytes.toBytes("testtable"); + byte [][] families = {fam1, fam2, fam3}; + + HBaseConfiguration hc = initSplit(); + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, hc, families); + + try { + LOG.info("" + addContent(region, fam3)); + region.flushcache(); + byte [] splitRow = region.compactStores(); + assertNotNull(splitRow); + LOG.info("SplitRow: " + Bytes.toString(splitRow)); + HRegion [] regions = split(region, splitRow); + try { + // Need to open the regions. + // TODO: Add an 'open' to HRegion... don't do open by constructing + // instance. + for (int i = 0; i < regions.length; i++) { + regions[i] = openClosedRegion(regions[i]); + } + Path oldRegionPath = region.getRegionDir(); + long startTime = System.currentTimeMillis(); + HRegion subregions [] = region.splitRegion(splitRow); + if (subregions != null) { + LOG.info("Split region elapsed time: " + + ((System.currentTimeMillis() - startTime) / 1000.0)); + assertEquals("Number of subregions", subregions.length, 2); + for (int i = 0; i < subregions.length; i++) { + subregions[i] = openClosedRegion(subregions[i]); + subregions[i].compactStores(); + } - // Test the Scanner!!! - String[] vals1 = new String[1000]; - for(int k = 0; k < vals1.length; k++) { - vals1[k] = Integer.toString(k); + // Now merge it back together + Path oldRegion1 = subregions[0].getRegionDir(); + Path oldRegion2 = subregions[1].getRegionDir(); + startTime = System.currentTimeMillis(); + region = HRegion.mergeAdjacent(subregions[0], subregions[1]); + LOG.info("Merge regions elapsed time: " + + ((System.currentTimeMillis() - startTime) / 1000.0)); + fs.delete(oldRegion1, true); + fs.delete(oldRegion2, true); + fs.delete(oldRegionPath, true); + } + LOG.info("splitAndMerge completed."); + } finally { + for (int i = 0; i < regions.length; i++) { + try { + regions[i].close(); + } catch (IOException e) { + // Ignore. + } + } + } + } finally { + if (region != null) { + region.close(); + region.getLog().closeAndDelete(); + } } - - // 1. Insert a bunch of values - long startTime = System.currentTimeMillis(); - for(int k = 0; k < vals1.length / 2; k++) { - String kLabel = String.format("%1$03d", k); - - BatchUpdate batchUpdate = - new BatchUpdate(Bytes.toBytes("row_vals1_" + kLabel), - System.currentTimeMillis()); - batchUpdate.put(cols[0], vals1[k].getBytes(HConstants.UTF8_ENCODING)); - batchUpdate.put(cols[1], vals1[k].getBytes(HConstants.UTF8_ENCODING)); - region.commit(batchUpdate); - numInserted += 2; + } + + ////////////////////////////////////////////////////////////////////////////// + // Scanner tests + ////////////////////////////////////////////////////////////////////////////// + public void testGetScanner_WithOkFamilies() throws IOException { + byte [] tableName = Bytes.toBytes("testtable"); + byte [] fam1 = Bytes.toBytes("fam1"); + byte [] fam2 = Bytes.toBytes("fam2"); + + byte [][] families = {fam1, fam2}; + + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, families); + + Scan scan = new Scan(); + scan.addFamily(fam1); + scan.addFamily(fam2); + try { + InternalScanner is = region.getScanner(scan); + } catch (Exception e) { + assertTrue("Families could not be found in Region", false); } - LOG.info("Write " + (vals1.length / 2) + " elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); + } + + public void testGetScanner_WithNotOkFamilies() throws IOException { + byte [] tableName = Bytes.toBytes("testtable"); + byte [] fam1 = Bytes.toBytes("fam1"); + byte [] fam2 = Bytes.toBytes("fam2"); - // 2. Scan from cache - startTime = System.currentTimeMillis(); - ScannerIncommon s = this.region.getScanner(cols, HConstants.EMPTY_START_ROW, - System.currentTimeMillis()); - int numFetched = 0; + byte [][] families = {fam1}; + + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, families); + + Scan scan = new Scan(); + scan.addFamily(fam2); + boolean ok = false; + try { + InternalScanner is = region.getScanner(scan); + } catch (Exception e) { + ok = true; + } + assertTrue("Families could not be found in Region", ok); + } + + public void testGetScanner_WithNoFamilies() throws IOException { + byte [] tableName = Bytes.toBytes("testtable"); + byte [] row1 = Bytes.toBytes("row1"); + byte [] fam1 = Bytes.toBytes("fam1"); + byte [] fam2 = Bytes.toBytes("fam2"); + byte [] fam3 = Bytes.toBytes("fam3"); + byte [] fam4 = Bytes.toBytes("fam4"); + + byte [][] families = {fam1, fam2, fam3, fam4}; + + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, families); + + //Putting data in Region + Put put = new Put(row1); + put.add(fam1, null, null); + put.add(fam2, null, null); + put.add(fam3, null, null); + put.add(fam4, null, null); + region.put(put); + + Scan scan = null; + InternalScanner is = null; + + //Testing to see how many scanners that is produced by getScanner, starting + //with known number, 2 - current = 1 + scan = new Scan(); + scan.addFamily(fam2); + scan.addFamily(fam4); + is = region.getScanner(scan); + assertEquals(1, ((RegionScanner)is).getStoreHeap().getHeap().size()); + + scan = new Scan(); + is = region.getScanner(scan); + assertEquals(families.length -1, + ((RegionScanner)is).getStoreHeap().getHeap().size()); + } + + public void testRegionScanner_Next() throws IOException { + byte [] tableName = Bytes.toBytes("testtable"); + byte [] row1 = Bytes.toBytes("row1"); + byte [] row2 = Bytes.toBytes("row2"); + byte [] fam1 = Bytes.toBytes("fam1"); + byte [] fam2 = Bytes.toBytes("fam2"); + byte [] fam3 = Bytes.toBytes("fam3"); + byte [] fam4 = Bytes.toBytes("fam4"); + + byte [][] families = {fam1, fam2, fam3, fam4}; + long ts = System.currentTimeMillis(); + + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, families); + + //Putting data in Region + Put put = null; + put = new Put(row1); + put.add(fam1, null, ts, null); + put.add(fam2, null, ts, null); + put.add(fam3, null, ts, null); + put.add(fam4, null, ts, null); + region.put(put); + + put = new Put(row2); + put.add(fam1, null, ts, null); + put.add(fam2, null, ts, null); + put.add(fam3, null, ts, null); + put.add(fam4, null, ts, null); + region.put(put); + + Scan scan = new Scan(); + scan.addFamily(fam2); + scan.addFamily(fam4); + InternalScanner is = region.getScanner(scan); + + List res = null; + + //Result 1 + List expected1 = new ArrayList(); + expected1.add(new KeyValue(row1, fam2, null, ts, KeyValue.Type.Put, null)); + expected1.add(new KeyValue(row1, fam4, null, ts, KeyValue.Type.Put, null)); + + res = new ArrayList(); + is.next(res); + for(int i=0; i expected2 = new ArrayList(); + expected2.add(new KeyValue(row2, fam2, null, ts, KeyValue.Type.Put, null)); + expected2.add(new KeyValue(row2, fam4, null, ts, KeyValue.Type.Put, null)); + + res = new ArrayList(); + is.next(res); + for(int i=0; i expected = new ArrayList(); + expected.add(kv13); + expected.add(kv12); + + Scan scan = new Scan(row1); + scan.addColumn(fam1, qf1); + scan.setMaxVersions(MAX_VERSIONS); + List actual = new ArrayList(); + InternalScanner scanner = region.getScanner(scan); + + boolean hasNext = scanner.next(actual); + assertEquals(false, hasNext); + + //Verify result + for(int i=0; i expected = new ArrayList(); + expected.add(kv13); + expected.add(kv12); + expected.add(kv23); + expected.add(kv22); + + Scan scan = new Scan(row1); + scan.addColumn(fam1, qf1); + scan.addColumn(fam1, qf2); + scan.setMaxVersions(MAX_VERSIONS); + List actual = new ArrayList(); + InternalScanner scanner = region.getScanner(scan); + + boolean hasNext = scanner.next(actual); + assertEquals(false, hasNext); + + //Verify result + for(int i=0; i expected = new ArrayList(); + expected.add(kv14); + expected.add(kv13); + expected.add(kv12); + expected.add(kv24); + expected.add(kv23); + expected.add(kv22); + + Scan scan = new Scan(row1); + scan.addColumn(fam1, qf1); + scan.addColumn(fam1, qf2); + int versions = 3; + scan.setMaxVersions(versions); + List actual = new ArrayList(); + InternalScanner scanner = region.getScanner(scan); + + boolean hasNext = scanner.next(actual); + assertEquals(false, hasNext); + + //Verify result + for(int i=0; i expected = new ArrayList(); + expected.add(kv13); + expected.add(kv12); + expected.add(kv23); + expected.add(kv22); + + Scan scan = new Scan(row1); + scan.addFamily(fam1); + scan.setMaxVersions(MAX_VERSIONS); + List actual = new ArrayList(); + InternalScanner scanner = region.getScanner(scan); + + boolean hasNext = scanner.next(actual); + assertEquals(false, hasNext); + + //Verify result + for(int i=0; i expected = new ArrayList(); + expected.add(kv13); + expected.add(kv12); + expected.add(kv23); + expected.add(kv22); + + Scan scan = new Scan(row1); + scan.addFamily(fam1); + scan.setMaxVersions(MAX_VERSIONS); + List actual = new ArrayList(); + InternalScanner scanner = region.getScanner(scan); + + boolean hasNext = scanner.next(actual); + assertEquals(false, hasNext); + + //Verify result + for(int i=0; i expected = new ArrayList(); + expected.add(kv14); + expected.add(kv13); + expected.add(kv12); + expected.add(kv24); + expected.add(kv23); + expected.add(kv22); + + Scan scan = new Scan(row1); + int versions = 3; + scan.setMaxVersions(versions); + List actual = new ArrayList(); + InternalScanner scanner = region.getScanner(scan); + + boolean hasNext = scanner.next(actual); + assertEquals(false, hasNext); + + //Verify result + for(int i=0; i + // than one of our unsplitable references. it will. + for (int j = 0; j < 2; j++) { + addContent(regions[i], fam3); + } + addContent(regions[i], fam2); + addContent(regions[i], fam1); + regions[i].flushcache(); + } + + byte [][] midkeys = new byte [regions.length][]; + // To make regions splitable force compaction. + for (int i = 0; i < regions.length; i++) { + midkeys[i] = regions[i].compactStores(); + } + + TreeMap sortedMap = new TreeMap(); + // Split these two daughter regions so then I'll have 4 regions. Will + // split because added data above. + for (int i = 0; i < regions.length; i++) { + HRegion[] rs = null; + if (midkeys[i] != null) { + rs = split(regions[i], midkeys[i]); + for (int j = 0; j < rs.length; j++) { + sortedMap.put(Bytes.toString(rs[j].getRegionName()), + openClosedRegion(rs[j])); + } + } + } + LOG.info("Made 4 regions"); + // The splits should have been even. Test I can get some arbitrary row + // out of each. + int interval = (LAST_CHAR - FIRST_CHAR) / 3; + byte[] b = Bytes.toBytes(START_KEY); + for (HRegion r : sortedMap.values()) { + assertGet(r, fam3, b); + b[0] += interval; + } + } finally { + for (int i = 0; i < regions.length; i++) { + try { + regions[i].close(); + } catch (IOException e) { + // Ignore. + } + } + } + } finally { + if (region != null) { + region.close(); + region.getLog().closeAndDelete(); + } + } + } + + public void testSplitRegion() throws IOException { + byte [] tableName = Bytes.toBytes("testtable"); + byte [] qualifier = Bytes.toBytes("qualifier"); + HBaseConfiguration hc = initSplit(); + int numRows = 10; + byte [][] families = {fam1, fam3}; + + //Setting up region + String method = this.getName(); + initHRegion(tableName, method, hc, families); + + //Put data in region + int startRow = 100; + putData(startRow, numRows, qualifier, families); + int splitRow = startRow + numRows; + putData(splitRow, numRows, qualifier, families); + region.flushcache(); + + HRegion [] regions = null; + try { + regions = region.splitRegion(Bytes.toBytes("" + splitRow)); + //Opening the regions returned. + for (int i = 0; i < regions.length; i++) { + regions[i] = openClosedRegion(regions[i]); + } + //Verifying that the region has been split + assertEquals(2, regions.length); + + //Verifying that all data is still there and that data is in the right + //place + verifyData(regions[0], startRow, numRows, qualifier, families); + verifyData(regions[1], splitRow, numRows, qualifier, families); + + } finally { + if (region != null) { + region.close(); + region.getLog().closeAndDelete(); + } + } + } + + private void putData(int startRow, int numRows, byte [] qf, + byte [] ...families) + throws IOException { + for(int i=startRow; i()); + s.close(); + } catch(UnknownScannerException ex) { + ex.printStackTrace(); + fail("Got the " + ex); + } + } + + + private void assertGet(final HRegion r, final byte [] family, final byte [] k) + throws IOException { + // Now I have k, get values out and assert they are as expected. + Get get = new Get(k).addFamily(family).setMaxVersions(); + KeyValue [] results = r.get(get, null).raw(); + for (int j = 0; j < results.length; j++) { + byte [] tmp = results[j].getValue(); + // Row should be equal to value every time. + assertTrue(Bytes.equals(k, tmp)); + } + } + + /* + * Assert first value in the passed region is firstValue. + * @param r + * @param column + * @param firstValue + * @throws IOException + */ + private void assertScan(final HRegion r, final byte [] column, + final byte [] firstValue) + throws IOException { + byte [][] cols = {column}; + Scan scan = new Scan(); + scan.addColumns(cols); + InternalScanner s = r.getScanner(scan); try { List curVals = new ArrayList(); - int k = 0; - while(s.next(curVals)) { + boolean first = true; + OUTER_LOOP: while(s.next(curVals)) { for (KeyValue kv: curVals) { byte [] val = kv.getValue(); - int curval = - Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); - for(int j = 0; j < cols.length; j++) { - if (!kv.matchingColumn(cols[j])) { - assertEquals("Error at: " + kv + " " + Bytes.toString(cols[j]), - k, curval); - numFetched++; - break; - } - } - } - curVals.clear(); - k++; - } - } finally { - s.close(); - } - assertEquals(numInserted, numFetched); - - LOG.info("Scanned " + (vals1.length / 2) - + " rows from cache. Elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - - // 3. Flush to disk - startTime = System.currentTimeMillis(); - region.flushcache(); - LOG.info("Cache flush elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - - // 4. Scan from disk - startTime = System.currentTimeMillis(); - s = this.region.getScanner(cols, HConstants.EMPTY_START_ROW, - System.currentTimeMillis()); - numFetched = 0; - try { - List curVals = new ArrayList(); - int k = 0; - while(s.next(curVals)) { - for(Iterator it = curVals.iterator(); it.hasNext(); ) { - KeyValue kv = it.next(); - byte [] col = kv.getColumn(); - byte [] val = kv.getValue(); - int curval = - Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); - for(int j = 0; j < cols.length; j++) { - if (Bytes.compareTo(col, cols[j]) == 0) { - assertEquals("Error at:" + kv.getRow() + "/" - + kv.getTimestamp() - + ", Value for " + col + " should be: " + k - + ", but was fetched as: " + curval, k, curval); - numFetched++; - } - } - } - curVals.clear(); - k++; - } - } finally { - s.close(); - } - assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched); - - LOG.info("Scanned " + (vals1.length / 2) - + " rows from disk. Elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - - // 5. Insert more values - startTime = System.currentTimeMillis(); - for(int k = vals1.length/2; k < vals1.length; k++) { - String kLabel = String.format("%1$03d", k); - BatchUpdate batchUpdate = - new BatchUpdate(Bytes.toBytes("row_vals1_" + kLabel), - System.currentTimeMillis()); - batchUpdate.put(cols[0], vals1[k].getBytes(HConstants.UTF8_ENCODING)); - batchUpdate.put(cols[1], vals1[k].getBytes(HConstants.UTF8_ENCODING)); - region.commit(batchUpdate); - numInserted += 2; - } - - LOG.info("Write " + (vals1.length / 2) + " rows (second half). Elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - - // 6. Scan from cache and disk - startTime = System.currentTimeMillis(); - s = this.region.getScanner(cols, HConstants.EMPTY_START_ROW, - System.currentTimeMillis()); - numFetched = 0; - try { - List curVals = new ArrayList(); - int k = 0; - while(s.next(curVals)) { - for(Iterator it = curVals.iterator(); it.hasNext(); ) { - KeyValue kv = it.next(); - byte [] col = kv.getColumn(); - byte [] val = kv.getValue(); - int curval = - Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); - for(int j = 0; j < cols.length; j++) { - if(Bytes.compareTo(col, cols[j]) == 0) { - assertEquals("Error at:" + kv.getRow() + "/" - + kv.getTimestamp() - + ", Value for " + col + " should be: " + k - + ", but was fetched as: " + curval, k, curval); - numFetched++; - } - } - } - curVals.clear(); - k++; - } - } finally { - s.close(); - } - assertEquals("Inserted " + numInserted + " values, but fetched " + - numFetched, numInserted, numFetched); - - LOG.info("Scanned " + vals1.length - + " rows from cache and disk. Elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - - // 7. Flush to disk - startTime = System.currentTimeMillis(); - region.flushcache(); - LOG.info("Cache flush elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - - // 8. Scan from disk - startTime = System.currentTimeMillis(); - s = this.region.getScanner(cols, HConstants.EMPTY_START_ROW, - System.currentTimeMillis()); - numFetched = 0; - try { - List curVals = new ArrayList(); - int k = 0; - while(s.next(curVals)) { - for(Iterator it = curVals.iterator(); it.hasNext(); ) { - KeyValue kv = it.next(); - byte [] col = kv.getColumn(); - byte [] val = kv.getValue(); - int curval = - Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); - for (int j = 0; j < cols.length; j++) { - if (Bytes.compareTo(col, cols[j]) == 0) { - assertEquals("Value for " + Bytes.toString(col) + " should be: " + k - + ", but was fetched as: " + curval, curval, k); - numFetched++; - } - } - } - curVals.clear(); - k++; - } - } finally { - s.close(); - } - assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, - numInserted, numFetched); - LOG.info("Scanned " + vals1.length - + " rows from disk. Elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - - // 9. Scan with a starting point - startTime = System.currentTimeMillis(); - s = this.region.getScanner(cols, Bytes.toBytes("row_vals1_500"), - System.currentTimeMillis()); - numFetched = 0; - try { - List curVals = new ArrayList(); - int k = 500; - while(s.next(curVals)) { - for(Iterator it = curVals.iterator(); it.hasNext(); ) { - KeyValue kv = it.next(); - byte [] col = kv.getColumn(); - byte [] val = kv.getValue(); - int curval = - Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); - for (int j = 0; j < cols.length; j++) { - if (Bytes.compareTo(col, cols[j]) == 0) { - assertEquals("Value for " + Bytes.toString(col) + " should be: " + k - + ", but was fetched as: " + curval, curval, k); - numFetched++; - } - } - } - curVals.clear(); - k++; - } - } finally { - s.close(); - } - assertEquals("Should have fetched " + (numInserted / 2) + - " values, but fetched " + numFetched, (numInserted / 2), numFetched); - - LOG.info("Scanned " + (numFetched / 2) - + " rows from disk with specified start point. Elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - - LOG.info("scan completed."); - } - - // NOTE: This test depends on testBatchWrite succeeding - private void splitAndMerge() throws IOException { - Path oldRegionPath = r.getRegionDir(); - byte [] splitRow = r.compactStores(); - assertNotNull(splitRow); - long startTime = System.currentTimeMillis(); - HRegion subregions [] = r.splitRegion(splitRow); - if (subregions != null) { - LOG.info("Split region elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - assertEquals("Number of subregions", subregions.length, 2); - for (int i = 0; i < subregions.length; i++) { - subregions[i] = openClosedRegion(subregions[i]); - subregions[i].compactStores(); - } - - // Now merge it back together - Path oldRegion1 = subregions[0].getRegionDir(); - Path oldRegion2 = subregions[1].getRegionDir(); - startTime = System.currentTimeMillis(); - r = HRegion.mergeAdjacent(subregions[0], subregions[1]); - region = new HRegionIncommon(r); - LOG.info("Merge regions elapsed time: " + - ((System.currentTimeMillis() - startTime) / 1000.0)); - fs.delete(oldRegion1, true); - fs.delete(oldRegion2, true); - fs.delete(oldRegionPath, true); - } - LOG.info("splitAndMerge completed."); - } - - // This test verifies that everything is still there after splitting and merging - - private void read() throws IOException { - // First verify the data written by testBasic() - byte [][] cols = { - Bytes.toBytes(ANCHORNUM + "[0-9]+"), - CONTENTS_BASIC - }; - long startTime = System.currentTimeMillis(); - InternalScanner s = - r.getScanner(cols, HConstants.EMPTY_START_ROW, - System.currentTimeMillis(), null); - try { - int contentsFetched = 0; - int anchorFetched = 0; - List curVals = new ArrayList(); - int k = 0; - while(s.next(curVals)) { - for(Iterator it = curVals.iterator(); it.hasNext(); ) { - KeyValue kv = it.next(); - byte [] col = kv.getColumn(); - byte [] val = kv.getValue(); - String curval = Bytes.toString(val); - if (Bytes.compareTo(col, CONTENTS_BASIC) == 0) { - assertTrue("Error at:" + kv - + ", Value for " + col + " should start with: " + CONTENTSTR - + ", but was fetched as: " + curval, - curval.startsWith(CONTENTSTR)); - contentsFetched++; - - } else if (Bytes.toString(col).startsWith(ANCHORNUM)) { - assertTrue("Error at:" + kv - + ", Value for " + Bytes.toString(col) + - " should start with: " + ANCHORSTR - + ", but was fetched as: " + curval, - curval.startsWith(ANCHORSTR)); - anchorFetched++; - + byte [] curval = val; + if (first) { + first = false; + assertTrue(Bytes.compareTo(curval, firstValue) == 0); } else { - LOG.info("UNEXPECTED COLUMN " + Bytes.toString(col)); + // Not asserting anything. Might as well break. + break OUTER_LOOP; } } - curVals.clear(); - k++; } - assertEquals("Expected " + NUM_VALS + " " + Bytes.toString(CONTENTS_BASIC) + - " values, but fetched " + contentsFetched, NUM_VALS, contentsFetched); - assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + - " values, but fetched " + anchorFetched, NUM_VALS, anchorFetched); - - LOG.info("Scanned " + NUM_VALS - + " rows from disk. Elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - } finally { s.close(); } - - // Verify testScan data - - cols = new byte [][] {CONTENTS_FIRSTCOL, ANCHOR_SECONDCOL}; - - startTime = System.currentTimeMillis(); - - s = r.getScanner(cols, HConstants.EMPTY_START_ROW, - System.currentTimeMillis(), null); - try { - int numFetched = 0; - List curVals = new ArrayList(); - int k = 0; - while(s.next(curVals)) { - for(Iterator it = curVals.iterator(); it.hasNext(); ) { - KeyValue kv = it.next(); - byte [] col = kv.getColumn(); - byte [] val = kv.getValue(); - int curval = - Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); - - for (int j = 0; j < cols.length; j++) { - if (Bytes.compareTo(col, cols[j]) == 0) { - assertEquals("Value for " + Bytes.toString(col) + " should be: " + k - + ", but was fetched as: " + curval, curval, k); - numFetched++; - } - } - } - curVals.clear(); - k++; - } - assertEquals("Inserted " + numInserted + " values, but fetched " + - numFetched, numInserted, numFetched); - - LOG.info("Scanned " + (numFetched / 2) - + " rows from disk. Elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - - } finally { - s.close(); - } - - // Test a scanner which only specifies the column family name - - cols = new byte [][] { - Bytes.toBytes("anchor:") - }; - - startTime = System.currentTimeMillis(); - - s = r.getScanner(cols, HConstants.EMPTY_START_ROW, System.currentTimeMillis(), null); - - try { - int fetched = 0; - List curVals = new ArrayList(); - while(s.next(curVals)) { - for(Iterator it = curVals.iterator(); it.hasNext(); ) { - it.next(); - fetched++; - } - curVals.clear(); - } - assertEquals("Inserted " + (NUM_VALS + numInserted/2) + - " values, but fetched " + fetched, (NUM_VALS + numInserted/2), fetched); - LOG.info("Scanned " + fetched - + " rows from disk. Elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - - } finally { - s.close(); - } - LOG.info("read completed."); } + protected HRegion [] split(final HRegion r, final byte [] splitRow) + throws IOException { + // Assert can get mid key from passed region. + assertGet(r, fam3, splitRow); + HRegion [] regions = r.splitRegion(splitRow); + assertEquals(regions.length, 2); + return regions; + } + + private HBaseConfiguration initSplit() { + HBaseConfiguration conf = new HBaseConfiguration(); + // Always compact if there is more than one store file. + conf.setInt("hbase.hstore.compactionThreshold", 2); + + // Make lease timeout longer, lease checks less frequent + conf.setInt("hbase.master.lease.period", 10 * 1000); + conf.setInt("hbase.master.lease.thread.wakefrequency", 5 * 1000); + + conf.setInt("hbase.regionserver.lease.period", 10 * 1000); + + // Increase the amount of time between client retries + conf.setLong("hbase.client.pause", 15 * 1000); + + // This size should make it so we always split using the addContent + // below. After adding all data, the first region is 1.3M + conf.setLong("hbase.hregion.max.filesize", 1024 * 128); + return conf; + } + + ////////////////////////////////////////////////////////////////////////////// + // Helpers + ////////////////////////////////////////////////////////////////////////////// + private HBaseConfiguration initHRegion() { + HBaseConfiguration conf = new HBaseConfiguration(); + + conf.set("hbase.hstore.compactionThreshold", "2"); + conf.setLong("hbase.hregion.max.filesize", 65536); + + return conf; + } + + private void initHRegion (byte [] tableName, String callingMethod, + byte[] ... families) throws IOException{ + initHRegion(tableName, callingMethod, new HBaseConfiguration(), families); + } + + private void initHRegion (byte [] tableName, String callingMethod, + HBaseConfiguration conf, byte [] ... families) throws IOException{ + HTableDescriptor htd = new HTableDescriptor(tableName); + for(byte [] family : families) { + HColumnDescriptor hcd = new HColumnDescriptor(family); + htd.addFamily(new HColumnDescriptor(family)); + } + + HRegionInfo info = new HRegionInfo(htd, null, null, false); + Path path = new Path(DIR + callingMethod); + region = HRegion.createHRegion(info, path, conf); + } } diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java b/src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java new file mode 100644 index 00000000000..3caa34c642a --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java @@ -0,0 +1,207 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +//import java.util.Random; +import java.util.Set; +//import java.util.SortedSet; +import java.util.TreeSet; + +import org.apache.hadoop.hbase.HBaseTestCase; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + + +public class TestKeyValueHeap extends HBaseTestCase +implements HConstants { + private final boolean PRINT = false; + + List scanners = new ArrayList(); + + private byte [] row1; + private byte [] fam1; + private byte [] col1; + private byte [] data; + + private byte [] row2; + private byte [] fam2; + private byte [] col2; + + private byte [] col3; + private byte [] col4; + private byte [] col5; + + public void setUp(){ + data = Bytes.toBytes("data"); + + row1 = Bytes.toBytes("row1"); + fam1 = Bytes.toBytes("fam1"); + col1 = Bytes.toBytes("col1"); + + row2 = Bytes.toBytes("row2"); + fam2 = Bytes.toBytes("fam2"); + col2 = Bytes.toBytes("col2"); + + col3 = Bytes.toBytes("col3"); + col4 = Bytes.toBytes("col4"); + col5 = Bytes.toBytes("col5"); + } + + public void testSorted(){ + //Cases that need to be checked are: + //1. The "smallest" KeyValue is in the same scanners as current + //2. Current scanner gets empty + + List l1 = new ArrayList(); + l1.add(new KeyValue(row1, fam1, col5, data)); + l1.add(new KeyValue(row2, fam1, col1, data)); + l1.add(new KeyValue(row2, fam1, col2, data)); + scanners.add(new Scanner(l1)); + + List l2 = new ArrayList(); + l2.add(new KeyValue(row1, fam1, col1, data)); + l2.add(new KeyValue(row1, fam1, col2, data)); + scanners.add(new Scanner(l2)); + + List l3 = new ArrayList(); + l3.add(new KeyValue(row1, fam1, col3, data)); + l3.add(new KeyValue(row1, fam1, col4, data)); + l3.add(new KeyValue(row1, fam2, col1, data)); + l3.add(new KeyValue(row1, fam2, col2, data)); + l3.add(new KeyValue(row2, fam1, col3, data)); + scanners.add(new Scanner(l3)); + + List expected = new ArrayList(); + expected.add(new KeyValue(row1, fam1, col1, data)); + expected.add(new KeyValue(row1, fam1, col2, data)); + expected.add(new KeyValue(row1, fam1, col3, data)); + expected.add(new KeyValue(row1, fam1, col4, data)); + expected.add(new KeyValue(row1, fam1, col5, data)); + expected.add(new KeyValue(row1, fam2, col1, data)); + expected.add(new KeyValue(row1, fam2, col2, data)); + expected.add(new KeyValue(row2, fam1, col1, data)); + expected.add(new KeyValue(row2, fam1, col2, data)); + expected.add(new KeyValue(row2, fam1, col3, data)); + + //Creating KeyValueHeap + KeyValueHeap kvh = + new KeyValueHeap(scanners.toArray(new Scanner[0]), KeyValue.COMPARATOR); + + List actual = new ArrayList(); + while(kvh.peek() != null){ + actual.add(kvh.next()); + } + + assertEquals(expected.size(), actual.size()); + for(int i=0; i l1 = new ArrayList(); + l1.add(new KeyValue(row1, fam1, col5, data)); + l1.add(new KeyValue(row2, fam1, col1, data)); + l1.add(new KeyValue(row2, fam1, col2, data)); + scanners.add(new Scanner(l1)); + + List l2 = new ArrayList(); + l2.add(new KeyValue(row1, fam1, col1, data)); + l2.add(new KeyValue(row1, fam1, col2, data)); + scanners.add(new Scanner(l2)); + + List l3 = new ArrayList(); + l3.add(new KeyValue(row1, fam1, col3, data)); + l3.add(new KeyValue(row1, fam1, col4, data)); + l3.add(new KeyValue(row1, fam2, col1, data)); + l3.add(new KeyValue(row1, fam2, col2, data)); + l3.add(new KeyValue(row2, fam1, col3, data)); + scanners.add(new Scanner(l3)); + + List expected = new ArrayList(); + expected.add(new KeyValue(row2, fam1, col1, data)); + + //Creating KeyValueHeap + KeyValueHeap kvh = + new KeyValueHeap(scanners.toArray(new Scanner[0]), KeyValue.COMPARATOR); + + KeyValue seekKv = new KeyValue(row2, fam1, null, null); + kvh.seek(seekKv); + + List actual = new ArrayList(); + actual.add(kvh.peek()); + + assertEquals(expected.size(), actual.size()); + for(int i=0; i scan = + new TreeSet((Comparator)KeyValue.COMPARATOR); + private Iterator iter; + private KeyValue current; + + public Scanner(List list) { + Collections.sort(list, (Comparator)KeyValue.COMPARATOR); + iter = list.iterator(); + if(iter.hasNext()){ + current = iter.next(); + } + } + + public KeyValue peek() { + return current; + } + + public KeyValue next() { + KeyValue oldCurrent = current; + if(iter.hasNext()){ + current = iter.next(); + } else { + current = null; + } + return oldCurrent; + } + + public void close(){} + + public boolean seek(KeyValue seekKv) { + while(iter.hasNext()){ + KeyValue next = iter.next(); + int ret = KeyValue.COMPARATOR.compare(next, seekKv); + if(ret >= 0){ + current = next; + return true; + } + } + return false; + } + } + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java b/src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java new file mode 100644 index 00000000000..e1ffcc3cbda --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java @@ -0,0 +1,68 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import junit.framework.TestCase; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueTestUtil; +import org.apache.hadoop.hbase.util.Bytes; + +public class TestKeyValueScanFixture extends TestCase { + + + public void testKeyValueScanFixture() { + KeyValue kvs[] = new KeyValue[]{ + KeyValueTestUtil.create("RowA", "family", "qf1", + 1, KeyValue.Type.Put, "value-1"), + KeyValueTestUtil.create("RowA", "family", "qf2", + 1, KeyValue.Type.Put, "value-2"), + KeyValueTestUtil.create("RowB", "family", "qf1", + 10, KeyValue.Type.Put, "value-10") + }; + KeyValueScanner scan = new KeyValueScanFixture( + KeyValue.COMPARATOR, kvs); + + // test simple things. + assertNull(scan.peek()); + KeyValue kv = KeyValue.createFirstOnRow(Bytes.toBytes("RowA")); + // should seek to this: + assertTrue(scan.seek(kv)); + KeyValue res = scan.peek(); + assertEquals(kvs[0], res); + + kv = KeyValue.createFirstOnRow(Bytes.toBytes("RowB")); + assertTrue(scan.seek(kv)); + res = scan.peek(); + assertEquals(kvs[2], res); + + // ensure we pull things out properly: + kv = KeyValue.createFirstOnRow(Bytes.toBytes("RowA")); + assertTrue(scan.seek(kv)); + assertEquals(kvs[0], scan.peek()); + assertEquals(kvs[0], scan.next()); + assertEquals(kvs[1], scan.peek()); + assertEquals(kvs[1], scan.next()); + assertEquals(kvs[2], scan.peek()); + assertEquals(kvs[2], scan.next()); + assertEquals(null, scan.peek()); + assertEquals(null, scan.next()); + } +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java b/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java index 5ef110d808e..34fc8d13416 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java @@ -26,11 +26,13 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.HBaseClusterTestCase; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.util.Bytes; @@ -106,16 +108,15 @@ public class TestLogRolling extends HBaseClusterTestCase { // Create the test table and open it HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY)); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); HBaseAdmin admin = new HBaseAdmin(conf); admin.createTable(desc); HTable table = new HTable(conf, tableName); for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls - BatchUpdate b = - new BatchUpdate("row" + String.format("%1$04d", i)); - b.put(HConstants.COLUMN_FAMILY, value); - table.commit(b); + Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i))); + put.add(HConstants.CATALOG_FAMILY, null, value); + table.put(put); if (i % 32 == 0) { // After every 32 writes sleep to let the log roller run diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestMemcache.java b/src/test/org/apache/hadoop/hbase/regionserver/TestMemcache.java new file mode 100644 index 00000000000..ed018da9395 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestMemcache.java @@ -0,0 +1,602 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.rmi.UnexpectedException; +import java.util.ArrayList; +import java.util.List; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.KeyValueTestUtil; +import org.apache.hadoop.hbase.util.Bytes; + +import junit.framework.TestCase; + +/** memcache test case */ +public class TestMemcache extends TestCase { + + private Memcache memcache; + + private static final int ROW_COUNT = 10; + + private static final int QUALIFIER_COUNT = 10; + + private static final byte [] FAMILY = Bytes.toBytes("column"); + + private static final int FIRST_ROW = 1; + private static final int NUM_VALS = 1000; + private static final byte [] CONTENTS_BASIC = Bytes.toBytes("contents:basic"); + private static final String CONTENTSTR = "contentstr"; + private static final String ANCHORNUM = "anchor:anchornum-"; + private static final String ANCHORSTR = "anchorstr"; + + @Override + public void setUp() throws Exception { + super.setUp(); + this.memcache = new Memcache(); + } + + public void testMultipleVersionsSimple() throws Exception { + Memcache m = new Memcache(HConstants.FOREVER, KeyValue.COMPARATOR); + byte [] row = Bytes.toBytes("testRow"); + byte [] family = Bytes.toBytes("testFamily"); + byte [] qf = Bytes.toBytes("testQualifier"); + long [] stamps = {1,2,3}; + byte [][] values = {Bytes.toBytes("value0"), Bytes.toBytes("value1"), + Bytes.toBytes("value2")}; + KeyValue key0 = new KeyValue(row, family, qf, stamps[0], values[0]); + KeyValue key1 = new KeyValue(row, family, qf, stamps[1], values[1]); + KeyValue key2 = new KeyValue(row, family, qf, stamps[2], values[2]); + + m.add(key0); + m.add(key1); + m.add(key2); + + assertTrue("Expected memcache to hold 3 values, actually has " + + m.memcache.size(), m.memcache.size() == 3); + } + + public void testBinary() throws IOException { + Memcache mc = new Memcache(HConstants.FOREVER, KeyValue.ROOT_COMPARATOR); + final int start = 43; + final int end = 46; + for (int k = start; k <= end; k++) { + byte [] kk = Bytes.toBytes(k); + byte [] row = + Bytes.toBytes(".META.,table," + Bytes.toString(kk) + ",1," + k); + KeyValue key = new KeyValue(row, CONTENTS_BASIC, + System.currentTimeMillis(), + (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING)); + mc.add(key); + System.out.println(key); +// key = new KeyValue(row, Bytes.toBytes(ANCHORNUM + k), +// System.currentTimeMillis(), +// (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING)); +// mc.add(key); +// System.out.println(key); + } + int index = start; + for (KeyValue kv: mc.memcache) { + System.out.println(kv); + byte [] b = kv.getRow(); + // Hardcoded offsets into String + String str = Bytes.toString(b, 13, 4); + byte [] bb = Bytes.toBytes(index); + String bbStr = Bytes.toString(bb); + assertEquals(str, bbStr); + index++; + } + } + + + /** + * Test memcache snapshots + * @throws IOException + */ + public void testSnapshotting() throws IOException { + final int snapshotCount = 5; + // Add some rows, run a snapshot. Do it a few times. + for (int i = 0; i < snapshotCount; i++) { + addRows(this.memcache); + runSnapshot(this.memcache); + Set ss = this.memcache.getSnapshot(); + assertEquals("History not being cleared", 0, ss.size()); + } + } + + + ////////////////////////////////////////////////////////////////////////////// + // Get tests + ////////////////////////////////////////////////////////////////////////////// + /** For HBASE-528 */ + public void testGetRowKeyAtOrBefore() { + // set up some test data + byte [] t10 = Bytes.toBytes("010"); + byte [] t20 = Bytes.toBytes("020"); + byte [] t30 = Bytes.toBytes("030"); + byte [] t35 = Bytes.toBytes("035"); + byte [] t40 = Bytes.toBytes("040"); + + memcache.add(getKV(t10, "t10 bytes".getBytes())); + memcache.add(getKV(t20, "t20 bytes".getBytes())); + memcache.add(getKV(t30, "t30 bytes".getBytes())); + memcache.add(getKV(t35, "t35 bytes".getBytes())); + // write a delete in there to see if things still work ok + memcache.add(getDeleteKV(t35)); + memcache.add(getKV(t40, "t40 bytes".getBytes())); + + NavigableSet results = null; + + // try finding "015" + results = + new TreeSet(this.memcache.comparator.getComparatorIgnoringType()); + KeyValue t15 = new KeyValue(Bytes.toBytes("015"), + System.currentTimeMillis()); + memcache.getRowKeyAtOrBefore(t15, results); + KeyValue kv = results.last(); + assertTrue(KeyValue.COMPARATOR.compareRows(kv, t10) == 0); + + // try "020", we should get that row exactly + results = + new TreeSet(this.memcache.comparator.getComparatorIgnoringType()); + memcache.getRowKeyAtOrBefore(new KeyValue(t20, System.currentTimeMillis()), + results); + assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t20) == 0); + + // try "030", we should get that row exactly + results = + new TreeSet(this.memcache.comparator.getComparatorIgnoringType()); + memcache.getRowKeyAtOrBefore(new KeyValue(t30, System.currentTimeMillis()), + results); + assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t30) == 0); + + // try "038", should skip the deleted "035" and give "030" + results = + new TreeSet(this.memcache.comparator.getComparatorIgnoringType()); + byte [] t38 = Bytes.toBytes("038"); + memcache.getRowKeyAtOrBefore(new KeyValue(t38, System.currentTimeMillis()), + results); + assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t30) == 0); + + // try "050", should get stuff from "040" + results = + new TreeSet(this.memcache.comparator.getComparatorIgnoringType()); + byte [] t50 = Bytes.toBytes("050"); + memcache.getRowKeyAtOrBefore(new KeyValue(t50, System.currentTimeMillis()), + results); + assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t40) == 0); + } + + + /** Test getNextRow from memcache + * @throws InterruptedException + */ + public void testGetNextRow() throws Exception { + addRows(this.memcache); + // Add more versions to make it a little more interesting. + Thread.sleep(1); + addRows(this.memcache); + KeyValue closestToEmpty = this.memcache.getNextRow(KeyValue.LOWESTKEY); + assertTrue(KeyValue.COMPARATOR.compareRows(closestToEmpty, + new KeyValue(Bytes.toBytes(0), System.currentTimeMillis())) == 0); + for (int i = 0; i < ROW_COUNT; i++) { + KeyValue nr = this.memcache.getNextRow(new KeyValue(Bytes.toBytes(i), + System.currentTimeMillis())); + if (i + 1 == ROW_COUNT) { + assertEquals(nr, null); + } else { + assertTrue(KeyValue.COMPARATOR.compareRows(nr, + new KeyValue(Bytes.toBytes(i + 1), System.currentTimeMillis())) == 0); + } + } + //starting from each row, validate results should contain the starting row + for (int startRowId = 0; startRowId < ROW_COUNT; startRowId++) { + InternalScanner scanner = + new StoreScanner(new Scan(Bytes.toBytes(startRowId)), FAMILY, + Integer.MAX_VALUE, this.memcache.comparator, null, + new KeyValueScanner[]{memcache.getScanner()}); + List results = new ArrayList(); + for (int i = 0; scanner.next(results); i++) { + int rowId = startRowId + i; + assertTrue("Row name", + KeyValue.COMPARATOR.compareRows(results.get(0), + Bytes.toBytes(rowId)) == 0); + assertEquals("Count of columns", QUALIFIER_COUNT, results.size()); + List row = new ArrayList(); + for (KeyValue kv : results) { + row.add(kv); + } + isExpectedRowWithoutTimestamps(rowId, row); + // Clear out set. Otherwise row results accumulate. + results.clear(); + } + } + } + + public void testGet_Basic_Found() throws IOException { + byte [] row = Bytes.toBytes("testrow"); + byte [] fam = Bytes.toBytes("testfamily"); + byte [] qf1 = Bytes.toBytes("testqualifier1"); + byte [] qf2 = Bytes.toBytes("testqualifier2"); + byte [] qf3 = Bytes.toBytes("testqualifier3"); + byte [] val = Bytes.toBytes("testval"); + + //Setting up memcache + KeyValue add1 = new KeyValue(row, fam ,qf1, val); + KeyValue add2 = new KeyValue(row, fam ,qf2, val); + KeyValue add3 = new KeyValue(row, fam ,qf3, val); + memcache.add(add1); + memcache.add(add2); + memcache.add(add3); + + //test + Get get = new Get(row); + NavigableSet columns = new TreeSet(Bytes.BYTES_COMPARATOR); + columns.add(qf2); + long ttl = Long.MAX_VALUE; + + QueryMatcher matcher = + new QueryMatcher(get, row, fam, columns, ttl, KeyValue.KEY_COMPARATOR, 1); + + List result = new ArrayList(); + boolean res = memcache.get(matcher, result); + assertEquals(true, res); + } + + public void testGet_Basic_NotFound() throws IOException { + byte [] row = Bytes.toBytes("testrow"); + byte [] fam = Bytes.toBytes("testfamily"); + byte [] qf1 = Bytes.toBytes("testqualifier1"); + byte [] qf2 = Bytes.toBytes("testqualifier2"); + byte [] qf3 = Bytes.toBytes("testqualifier3"); + byte [] val = Bytes.toBytes("testval"); + + //Setting up memcache + KeyValue add1 = new KeyValue(row, fam ,qf1, val); + KeyValue add3 = new KeyValue(row, fam ,qf3, val); + memcache.add(add1); + memcache.add(add3); + + //test + Get get = new Get(row); + NavigableSet columns = new TreeSet(Bytes.BYTES_COMPARATOR); + columns.add(qf2); + long ttl = Long.MAX_VALUE; + + QueryMatcher matcher = + new QueryMatcher(get, row, fam, columns, ttl, KeyValue.KEY_COMPARATOR, 1); + + List result = new ArrayList(); + boolean res = memcache.get(matcher, result); + assertEquals(false, res); + } + + public void testGet_MemcacheAndSnapShot() throws IOException { + byte [] row = Bytes.toBytes("testrow"); + byte [] fam = Bytes.toBytes("testfamily"); + byte [] qf1 = Bytes.toBytes("testqualifier1"); + byte [] qf2 = Bytes.toBytes("testqualifier2"); + byte [] qf3 = Bytes.toBytes("testqualifier3"); + byte [] qf4 = Bytes.toBytes("testqualifier4"); + byte [] qf5 = Bytes.toBytes("testqualifier5"); + byte [] val = Bytes.toBytes("testval"); + + //Creating get + Get get = new Get(row); + NavigableSet columns = new TreeSet(Bytes.BYTES_COMPARATOR); + columns.add(qf2); + columns.add(qf4); + long ttl = Long.MAX_VALUE; + + QueryMatcher matcher = + new QueryMatcher(get, row, fam, columns, ttl, KeyValue.KEY_COMPARATOR, 1); + + //Setting up memcache + memcache.add(new KeyValue(row, fam ,qf1, val)); + memcache.add(new KeyValue(row, fam ,qf2, val)); + memcache.add(new KeyValue(row, fam ,qf3, val)); + //Creating a snapshot + memcache.snapshot(); + assertEquals(3, memcache.snapshot.size()); + //Adding value to "new" memcache + assertEquals(0, memcache.memcache.size()); + memcache.add(new KeyValue(row, fam ,qf4, val)); + memcache.add(new KeyValue(row, fam ,qf5, val)); + assertEquals(2, memcache.memcache.size()); + + List result = new ArrayList(); + boolean res = memcache.get(matcher, result); + assertEquals(true, res); + } + + public void testGet_SpecificTimeStamp() throws IOException { + byte [] row = Bytes.toBytes("testrow"); + byte [] fam = Bytes.toBytes("testfamily"); + byte [] qf1 = Bytes.toBytes("testqualifier1"); + byte [] qf2 = Bytes.toBytes("testqualifier2"); + byte [] qf3 = Bytes.toBytes("testqualifier3"); + byte [] val = Bytes.toBytes("testval"); + + long ts1 = System.currentTimeMillis(); + long ts2 = ts1++; + long ts3 = ts2++; + + //Creating get + Get get = new Get(row); + get.setTimeStamp(ts2); + NavigableSet columns = new TreeSet(Bytes.BYTES_COMPARATOR); + columns.add(qf1); + columns.add(qf2); + columns.add(qf3); + long ttl = Long.MAX_VALUE; + + QueryMatcher matcher = + new QueryMatcher(get, row, fam, columns, ttl, KeyValue.KEY_COMPARATOR, 1); + + //Setting up expected + List expected = new ArrayList(); + KeyValue kv1 = new KeyValue(row, fam ,qf1, ts2, val); + KeyValue kv2 = new KeyValue(row, fam ,qf2, ts2, val); + KeyValue kv3 = new KeyValue(row, fam ,qf3, ts2, val); + expected.add(kv1); + expected.add(kv2); + expected.add(kv3); + + //Setting up memcache + memcache.add(new KeyValue(row, fam ,qf1, ts1, val)); + memcache.add(new KeyValue(row, fam ,qf2, ts1, val)); + memcache.add(new KeyValue(row, fam ,qf3, ts1, val)); + memcache.add(kv1); + memcache.add(kv2); + memcache.add(kv3); + memcache.add(new KeyValue(row, fam ,qf1, ts3, val)); + memcache.add(new KeyValue(row, fam ,qf2, ts3, val)); + memcache.add(new KeyValue(row, fam ,qf3, ts3, val)); + + //Get + List result = new ArrayList(); + memcache.get(matcher, result); + + assertEquals(expected.size(), result.size()); + for(int i=0; i expected = new ArrayList(); + expected.add(put3); + expected.add(del2); + expected.add(put1); + + assertEquals(3, memcache.memcache.size()); + int i=0; + for(KeyValue actual : memcache.memcache) { + assertEquals(expected.get(i++), actual); + } + } + + public void testGetWithDeleteColumn() throws IOException { + byte [] row = Bytes.toBytes("testrow"); + byte [] fam = Bytes.toBytes("testfamily"); + byte [] qf1 = Bytes.toBytes("testqualifier"); + byte [] val = Bytes.toBytes("testval"); + + long ts1 = System.nanoTime(); + KeyValue put1 = new KeyValue(row, fam, qf1, ts1, val); + long ts2 = ts1 + 1; + KeyValue put2 = new KeyValue(row, fam, qf1, ts2, val); + long ts3 = ts2 +1; + KeyValue put3 = new KeyValue(row, fam, qf1, ts3, val); + memcache.add(put1); + memcache.add(put2); + memcache.add(put3); + + assertEquals(3, memcache.memcache.size()); + + KeyValue del2 = + new KeyValue(row, fam, qf1, ts2, KeyValue.Type.DeleteColumn, val); + memcache.delete(del2); + + List expected = new ArrayList(); + expected.add(put3); + expected.add(del2); + + assertEquals(2, memcache.memcache.size()); + int i=0; + for(KeyValue actual : memcache.memcache) { + assertEquals(expected.get(i++), actual); + } + } + + + public void testGetWithDeleteFamily() throws IOException { + byte [] row = Bytes.toBytes("testrow"); + byte [] fam = Bytes.toBytes("testfamily"); + byte [] qf1 = Bytes.toBytes("testqualifier1"); + byte [] qf2 = Bytes.toBytes("testqualifier2"); + byte [] qf3 = Bytes.toBytes("testqualifier3"); + byte [] val = Bytes.toBytes("testval"); + long ts = System.nanoTime(); + + KeyValue put1 = new KeyValue(row, fam, qf1, ts, val); + KeyValue put2 = new KeyValue(row, fam, qf2, ts, val); + KeyValue put3 = new KeyValue(row, fam, qf3, ts, val); + memcache.add(put1); + memcache.add(put2); + memcache.add(put3); + + KeyValue del = + new KeyValue(row, fam, null, ts, KeyValue.Type.DeleteFamily, val); + memcache.delete(del); + + List expected = new ArrayList(); + expected.add(del); + + assertEquals(1, memcache.memcache.size()); + int i=0; + for(KeyValue actual : memcache.memcache) { + assertEquals(expected.get(i++), actual); + } + } + + public void testKeepDeleteInMemcache() { + byte [] row = Bytes.toBytes("testrow"); + byte [] fam = Bytes.toBytes("testfamily"); + byte [] qf = Bytes.toBytes("testqualifier"); + byte [] val = Bytes.toBytes("testval"); + long ts = System.nanoTime(); + memcache.add(new KeyValue(row, fam, qf, ts, val)); + KeyValue delete = new KeyValue(row, fam, qf, ts, KeyValue.Type.Delete, val); + memcache.delete(delete); + assertEquals(1, memcache.memcache.size()); + assertEquals(delete, memcache.memcache.first()); + } + + public void testRetainsDeleteVersion() throws IOException { + // add a put to memcache + memcache.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care")); + + // now process a specific delete: + KeyValue delete = KeyValueTestUtil.create( + "row1", "fam", "a", 100, KeyValue.Type.Delete, "dont-care"); + memcache.delete(delete); + + assertEquals(1, memcache.memcache.size()); + assertEquals(delete, memcache.memcache.first()); + } + public void testRetainsDeleteColumn() throws IOException { + // add a put to memcache + memcache.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care")); + + // now process a specific delete: + KeyValue delete = KeyValueTestUtil.create("row1", "fam", "a", 100, + KeyValue.Type.DeleteColumn, "dont-care"); + memcache.delete(delete); + + assertEquals(1, memcache.memcache.size()); + assertEquals(delete, memcache.memcache.first()); + } + public void testRetainsDeleteFamily() throws IOException { + // add a put to memcache + memcache.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care")); + + // now process a specific delete: + KeyValue delete = KeyValueTestUtil.create("row1", "fam", "a", 100, + KeyValue.Type.DeleteFamily, "dont-care"); + memcache.delete(delete); + + assertEquals(1, memcache.memcache.size()); + assertEquals(delete, memcache.memcache.first()); + } + + + ////////////////////////////////////////////////////////////////////////////// + // Helpers + ////////////////////////////////////////////////////////////////////////////// + private byte [] makeQualifier(final int i1, final int i2){ + return Bytes.toBytes(Integer.toString(i1) + ";" + + Integer.toString(i2)); + } + + /** + * Adds {@link #ROW_COUNT} rows and {@link #COLUMNS_COUNT} + * @param hmc Instance to add rows to. + * @throws IOException + */ + private void addRows(final Memcache hmc) { + for (int i = 0; i < ROW_COUNT; i++) { + long timestamp = System.currentTimeMillis(); + for (int ii = 0; ii < QUALIFIER_COUNT; ii++) { + byte [] row = Bytes.toBytes(i); + byte [] qf = makeQualifier(i, ii); + hmc.add(new KeyValue(row, FAMILY, qf, timestamp, qf)); + } + } + } + + private void runSnapshot(final Memcache hmc) throws UnexpectedException { + // Save off old state. + int oldHistorySize = hmc.getSnapshot().size(); + hmc.snapshot(); + Set ss = hmc.getSnapshot(); + // Make some assertions about what just happened. + assertTrue("History size has not increased", oldHistorySize < ss.size()); + hmc.clearSnapshot(ss); + } + + private void isExpectedRowWithoutTimestamps(final int rowIndex, + List kvs) { + int i = 0; + for (KeyValue kv: kvs) { + String expectedColname = Bytes.toString(makeQualifier(rowIndex, i++)); + String colnameStr = Bytes.toString(kv.getQualifier()); + assertEquals("Column name", colnameStr, expectedColname); + // Value is column name as bytes. Usually result is + // 100 bytes in size at least. This is the default size + // for BytesWriteable. For comparison, convert bytes to + // String and trim to remove trailing null bytes. + String colvalueStr = Bytes.toString(kv.getBuffer(), kv.getValueOffset(), + kv.getValueLength()); + assertEquals("Content", colnameStr, colvalueStr); + } + } + + private KeyValue getDeleteKV(byte [] row) { + return new KeyValue(row, Bytes.toBytes("test_col:"), + HConstants.LATEST_TIMESTAMP, KeyValue.Type.Delete, null); + } + + private KeyValue getKV(byte [] row, byte [] value) { + return new KeyValue(row, Bytes.toBytes("test_col:"), + HConstants.LATEST_TIMESTAMP, value); + } +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestMinorCompactingStoreScanner.java b/src/test/org/apache/hadoop/hbase/regionserver/TestMinorCompactingStoreScanner.java new file mode 100644 index 00000000000..c3d8c8ea9bf --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestMinorCompactingStoreScanner.java @@ -0,0 +1,83 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import junit.framework.TestCase; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueTestUtil; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class TestMinorCompactingStoreScanner extends TestCase { + + public void testDeleteFamiliy() throws IOException { + KeyValue[] kvs = new KeyValue[] { + KeyValueTestUtil.create("R1", "cf", "a", 100, KeyValue.Type.DeleteFamily, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "b", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "d", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "e", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "e", 11, KeyValue.Type.DeleteColumn, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "f", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "g", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "g", 11, KeyValue.Type.Delete, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "h", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "i", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R2", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) + }; + InternalScanner scan = + new MinorCompactingStoreScanner("cf", KeyValue.COMPARATOR, scanners); + List results = new ArrayList(); + assertTrue(scan.next(results)); + assertEquals(3, results.size()); + assertEquals(kvs[0], results.get(0)); + assertEquals(kvs[5], results.get(1)); + assertEquals(kvs[8], results.get(2)); + + results.clear(); + assertFalse(scan.next(results)); + assertEquals(1, results.size()); + assertEquals(kvs[kvs.length-1], results.get(0)); + } + + public void testDeleteVersion() throws IOException { + KeyValue[] kvs = new KeyValue[] { + KeyValueTestUtil.create("R1", "cf", "a", 15, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", 10, KeyValue.Type.Delete, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", 10, KeyValue.Type.Put, "dont-care") + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) + }; + InternalScanner scan = + new MinorCompactingStoreScanner("cf", KeyValue.COMPARATOR, scanners); + List results = new ArrayList(); + assertFalse(scan.next(results)); + assertEquals(2, results.size()); + assertEquals(kvs[0], results.get(0)); + assertEquals(kvs[1], results.get(1)); + } +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java b/src/test/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java new file mode 100644 index 00000000000..928d01213c2 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java @@ -0,0 +1,170 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NavigableSet; +import java.util.TreeSet; + +import org.apache.hadoop.hbase.HBaseTestCase; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValue.KeyComparator; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.io.TimeRange; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode; + + +public class TestQueryMatcher extends HBaseTestCase +implements HConstants { + private final boolean PRINT = false; + + private byte [] row1; + private byte [] row2; + private byte [] fam1; + private byte [] fam2; + private byte [] col1; + private byte [] col2; + private byte [] col3; + private byte [] col4; + private byte [] col5; + private byte [] col6; + + private byte [] data; + + private Get get; + + long ttl = Long.MAX_VALUE; + KeyComparator rowComparator; + + public void setUp(){ + row1 = Bytes.toBytes("row1"); + row2 = Bytes.toBytes("row2"); + fam1 = Bytes.toBytes("fam1"); + fam2 = Bytes.toBytes("fam2"); + col1 = Bytes.toBytes("col1"); + col2 = Bytes.toBytes("col2"); + col3 = Bytes.toBytes("col3"); + col4 = Bytes.toBytes("col4"); + col5 = Bytes.toBytes("col5"); + col6 = Bytes.toBytes("col6"); + + data = Bytes.toBytes("data"); + + //Create Get + get = new Get(row1); + get.addFamily(fam1); + get.addColumn(fam2, col2); + get.addColumn(fam2, col4); + get.addColumn(fam2, col5); + + rowComparator = KeyValue.KEY_COMPARATOR; + + } + + public void testMatch_ExplicitColumns() + throws IOException { + //Moving up from the Tracker by using Gets and List instead + //of just byte [] + + //Expected result + List expected = new ArrayList(); + expected.add(MatchCode.SKIP); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.SKIP); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.DONE); + + QueryMatcher qm = new QueryMatcher(get, get.getRow(), fam2, + get.getFamilyMap().get(fam2), ttl, rowComparator, 1); + + List memCache = new ArrayList(); + memCache.add(new KeyValue(row1, fam2, col1, data)); + memCache.add(new KeyValue(row1, fam2, col2, data)); + memCache.add(new KeyValue(row1, fam2, col3, data)); + memCache.add(new KeyValue(row1, fam2, col4, data)); + memCache.add(new KeyValue(row1, fam2, col5, data)); + + memCache.add(new KeyValue(row2, fam1, col1, data)); + + List actual = new ArrayList(); + + for(KeyValue kv : memCache){ + actual.add(qm.match(kv)); + } + + assertEquals(expected.size(), actual.size()); + for(int i=0; i< expected.size(); i++){ + assertEquals(expected.get(i), actual.get(i)); + if(PRINT){ + System.out.println("expected "+expected.get(i)+ + ", actual " +actual.get(i)); + } + } + } + + + public void testMatch_Wildcard() + throws IOException { + //Moving up from the Tracker by using Gets and List instead + //of just byte [] + + //Expected result + List expected = new ArrayList(); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.NEXT); + + QueryMatcher qm = new QueryMatcher(get, get.getRow(), fam2, null, + ttl, rowComparator, 1); + + List memCache = new ArrayList(); + memCache.add(new KeyValue(row1, fam2, col1, data)); + memCache.add(new KeyValue(row1, fam2, col2, data)); + memCache.add(new KeyValue(row1, fam2, col3, data)); + memCache.add(new KeyValue(row1, fam2, col4, data)); + memCache.add(new KeyValue(row1, fam2, col5, data)); + memCache.add(new KeyValue(row2, fam1, col1, data)); + + List actual = new ArrayList(); + + for(KeyValue kv : memCache){ + actual.add(qm.match(kv)); + } + + assertEquals(expected.size(), actual.size()); + for(int i=0; i< expected.size(); i++){ + assertEquals(expected.get(i), actual.get(i)); + if(PRINT){ + System.out.println("expected "+expected.get(i)+ + ", actual " +actual.get(i)); + } + } + } + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestScanDeleteTracker.java b/src/test/org/apache/hadoop/hbase/regionserver/TestScanDeleteTracker.java new file mode 100644 index 00000000000..27019a34bca --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestScanDeleteTracker.java @@ -0,0 +1,112 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.hbase.HBaseTestCase; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.Bytes; + + +public class TestScanDeleteTracker extends HBaseTestCase implements HConstants { + + private ScanDeleteTracker sdt; + private long timestamp = 10L; + private byte deleteType = 0; + + public void setUp(){ + sdt = new ScanDeleteTracker(KeyValue.KEY_COMPARATOR); + } + + public void testDeletedBy_Delete() { + byte [] qualifier = Bytes.toBytes("qualifier"); + deleteType = KeyValue.Type.Delete.getCode(); + + sdt.add(qualifier, 0, qualifier.length, timestamp, deleteType); + boolean ret = sdt.isDeleted(qualifier, 0, qualifier.length, timestamp); + assertEquals(true, ret); + } + + public void testDeletedBy_DeleteColumn() { + byte [] qualifier = Bytes.toBytes("qualifier"); + deleteType = KeyValue.Type.DeleteColumn.getCode(); + + sdt.add(qualifier, 0, qualifier.length, timestamp, deleteType); + timestamp -= 5; + boolean ret = sdt.isDeleted(qualifier, 0, qualifier.length, timestamp); + assertEquals(true, ret); + } + + public void testDeletedBy_DeleteFamily() { + byte [] qualifier = Bytes.toBytes("qualifier"); + deleteType = KeyValue.Type.DeleteFamily.getCode(); + + sdt.add(qualifier, 0, qualifier.length, timestamp, deleteType); + + timestamp -= 5; + boolean ret = sdt.isDeleted(qualifier, 0, qualifier.length, timestamp); + assertEquals(true, ret); + } + + public void testDelete_DeleteColumn() { + byte [] qualifier = Bytes.toBytes("qualifier"); + deleteType = KeyValue.Type.Delete.getCode(); + + sdt.add(qualifier, 0, qualifier.length, timestamp, deleteType); + + timestamp -= 5; + deleteType = KeyValue.Type.DeleteColumn.getCode(); + sdt.add(qualifier, 0, qualifier.length, timestamp, deleteType); + + timestamp -= 5; + boolean ret = sdt.isDeleted(qualifier, 0, qualifier.length, timestamp); + assertEquals(true, ret); + } + + + public void testDeleteColumn_Delete() { + byte [] qualifier = Bytes.toBytes("qualifier"); + deleteType = KeyValue.Type.DeleteColumn.getCode(); + + sdt.add(qualifier, 0, qualifier.length, timestamp, deleteType); + + qualifier = Bytes.toBytes("qualifier1"); + deleteType = KeyValue.Type.Delete.getCode(); + sdt.add(qualifier, 0, qualifier.length, timestamp, deleteType); + + boolean ret = sdt.isDeleted(qualifier, 0, qualifier.length, timestamp); + assertEquals(true, ret); + } + + //Testing new way where we save the Delete in case of a Delete for specific + //ts, could have just added the last line to the first test, but rather keep + //them separated + public void testDelete_KeepDelete(){ + byte [] qualifier = Bytes.toBytes("qualifier"); + deleteType = KeyValue.Type.Delete.getCode(); + + sdt.add(qualifier, 0, qualifier.length, timestamp, deleteType); + sdt.isDeleted(qualifier, 0, qualifier.length, timestamp); + assertEquals(false ,sdt.isEmpty()); + } + + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestScanWildcardColumnTracker.java b/src/test/org/apache/hadoop/hbase/regionserver/TestScanWildcardColumnTracker.java new file mode 100644 index 00000000000..673d1720fb2 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestScanWildcardColumnTracker.java @@ -0,0 +1,101 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.HBaseTestCase; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode; + +public class TestScanWildcardColumnTracker extends HBaseTestCase { + + final int VERSIONS = 2; + + public void testCheckColumn_Ok() { + //Create a WildcardColumnTracker + ScanWildcardColumnTracker tracker = + new ScanWildcardColumnTracker(VERSIONS); + + //Create list of qualifiers + List qualifiers = new ArrayList(); + qualifiers.add(Bytes.toBytes("qualifer1")); + qualifiers.add(Bytes.toBytes("qualifer2")); + qualifiers.add(Bytes.toBytes("qualifer3")); + qualifiers.add(Bytes.toBytes("qualifer4")); + + //Setting up expected result + List expected = new ArrayList(); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + + List actual = new ArrayList(); + + for(byte [] qualifier : qualifiers) { + MatchCode mc = tracker.checkColumn(qualifier, 0, qualifier.length); + actual.add(mc); + } + + //Compare actual with expected + for(int i=0; i qualifiers = new ArrayList(); + qualifiers.add(Bytes.toBytes("qualifer1")); + qualifiers.add(Bytes.toBytes("qualifer1")); + qualifiers.add(Bytes.toBytes("qualifer1")); + qualifiers.add(Bytes.toBytes("qualifer2")); + + //Setting up expected result + List expected = new ArrayList(); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.SKIP); + expected.add(MatchCode.INCLUDE); + + List actual = new ArrayList(); + + for(byte [] qualifier : qualifiers) { + MatchCode mc = tracker.checkColumn(qualifier, 0, qualifier.length); + actual.add(mc); + } + + //Compare actual with expected + for(int i=0; i qualifiers = new ArrayList(); + qualifiers.add(Bytes.toBytes("qualifer2")); + qualifiers.add(Bytes.toBytes("qualifer1")); + + boolean ok = false; + + try { + for(byte [] qualifier : qualifiers) { + MatchCode mc = tracker.checkColumn(qualifier, 0, qualifier.length); + } + } catch (Exception e) { + ok = true; + } + + assertEquals(true, ok); + } + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java b/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java index 4c7b8ae214a..baf1772b31c 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java @@ -34,6 +34,10 @@ 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.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.StopRowFilter; import org.apache.hadoop.hbase.filter.WhileMatchRowFilter; import org.apache.hadoop.hbase.io.BatchUpdate; @@ -48,24 +52,21 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; public class TestScanner extends HBaseTestCase { private final Log LOG = LogFactory.getLog(this.getClass()); - private static final byte [] FIRST_ROW = - HConstants.EMPTY_START_ROW; - private static final byte [][] COLS = { - HConstants.COLUMN_FAMILY - }; + private static final byte [] FIRST_ROW = HConstants.EMPTY_START_ROW; + private static final byte [][] COLS = { HConstants.CATALOG_FAMILY }; private static final byte [][] EXPLICIT_COLS = { - HConstants.COL_REGIONINFO, - HConstants.COL_SERVER, - HConstants.COL_STARTCODE + HConstants.REGIONINFO_QUALIFIER, HConstants.SERVER_QUALIFIER, + // TODO ryan + //HConstants.STARTCODE_QUALIFIER }; static final HTableDescriptor TESTTABLEDESC = new HTableDescriptor("testscanner"); static { - TESTTABLEDESC.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY, + TESTTABLEDESC.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY, 10, // Ten is arbitrary number. Keep versions to help debuggging. Compression.Algorithm.NONE.getName(), false, true, 8 * 1024, - Integer.MAX_VALUE, HConstants.FOREVER, false)); + HConstants.FOREVER, false)); } /** HRegionInfo for root region */ public static final HRegionInfo REGION_INFO = @@ -99,12 +100,13 @@ public class TestScanner extends HBaseTestCase { byte [] stoprow = Bytes.toBytes("ccc"); try { this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null); - addContent(this.r, HConstants.COLUMN_FAMILY); + addContent(this.r, HConstants.CATALOG_FAMILY); List results = new ArrayList(); // Do simple test of getting one row only first. - InternalScanner s = r.getScanner(HConstants.COLUMN_FAMILY_ARRAY, - Bytes.toBytes("abc"), HConstants.LATEST_TIMESTAMP, - new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("abd")))); + Scan scan = new Scan(Bytes.toBytes("abc"), Bytes.toBytes("abd")); + scan.addFamily(HConstants.CATALOG_FAMILY); + + InternalScanner s = r.getScanner(scan); int count = 0; while (s.next(results)) { count++; @@ -112,9 +114,10 @@ public class TestScanner extends HBaseTestCase { s.close(); assertEquals(1, count); // Now do something a bit more imvolved. - s = r.getScanner(HConstants.COLUMN_FAMILY_ARRAY, - startrow, HConstants.LATEST_TIMESTAMP, - new WhileMatchRowFilter(new StopRowFilter(stoprow))); + scan = new Scan(startrow, stoprow); + scan.addFamily(HConstants.CATALOG_FAMILY); + + s = r.getScanner(scan); count = 0; KeyValue kv = null; results = new ArrayList(); @@ -147,14 +150,15 @@ public class TestScanner extends HBaseTestCase { // Write information to the meta table - BatchUpdate batchUpdate = - new BatchUpdate(ROW_KEY, System.currentTimeMillis()); + Put put = new Put(ROW_KEY); + put.setTimeStamp(System.currentTimeMillis()); ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); DataOutputStream s = new DataOutputStream(byteStream); REGION_INFO.write(s); - batchUpdate.put(HConstants.COL_REGIONINFO, byteStream.toByteArray()); - region.commit(batchUpdate); + put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, + byteStream.toByteArray()); + region.put(put); // What we just committed is in the memcache. Verify that we can get // it back both with scanning and get @@ -177,13 +181,14 @@ public class TestScanner extends HBaseTestCase { HServerAddress address = new HServerAddress("foo.bar.com:1234"); - batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis()); + put = new Put(ROW_KEY); + put.setTimeStamp(System.currentTimeMillis()); + put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, + Bytes.toBytes(address.toString())); - batchUpdate.put(HConstants.COL_SERVER, Bytes.toBytes(address.toString())); +// put.add(HConstants.COL_STARTCODE, Bytes.toBytes(START_CODE)); - batchUpdate.put(HConstants.COL_STARTCODE, Bytes.toBytes(START_CODE)); - - region.commit(batchUpdate); + region.put(put); // Validate that we can still get the HRegionInfo, even though it is in // an older row on disk and there is a newer row in the memcache @@ -215,12 +220,12 @@ public class TestScanner extends HBaseTestCase { address = new HServerAddress("bar.foo.com:4321"); - batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis()); + put = new Put(ROW_KEY); + put.setTimeStamp(System.currentTimeMillis()); - batchUpdate.put(HConstants.COL_SERVER, - Bytes.toBytes(address.toString())); - - region.commit(batchUpdate); + put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, + Bytes.toBytes(address.toString())); + region.put(put); // Validate again @@ -273,6 +278,7 @@ public class TestScanner extends HBaseTestCase { private void scan(boolean validateStartcode, String serverName) throws IOException { InternalScanner scanner = null; + Scan scan = null; List results = new ArrayList(); byte [][][] scanColumns = { COLS, @@ -281,15 +287,20 @@ public class TestScanner extends HBaseTestCase { for(int i = 0; i < scanColumns.length; i++) { try { - scanner = r.getScanner(scanColumns[i], FIRST_ROW, - System.currentTimeMillis(), null); + scan = new Scan(FIRST_ROW); + scan.addColumns(scanColumns[i]); + scanner = r.getScanner(scan); while (scanner.next(results)) { - assertTrue(hasColumn(results, HConstants.COL_REGIONINFO)); - byte [] val = getColumn(results, HConstants.COL_REGIONINFO).getValue(); + assertTrue(hasColumn(results, HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER)); + byte [] val = getColumn(results, HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER).getValue(); validateRegionInfo(val); if(validateStartcode) { - assertTrue(hasColumn(results, HConstants.COL_STARTCODE)); - val = getColumn(results, HConstants.COL_STARTCODE).getValue(); +// assertTrue(hasColumn(results, HConstants.CATALOG_FAMILY, +// HConstants.STARTCODE_QUALIFIER)); +// val = getColumn(results, HConstants.CATALOG_FAMILY, +// HConstants.STARTCODE_QUALIFIER).getValue(); assertNotNull(val); assertFalse(val.length == 0); long startCode = Bytes.toLong(val); @@ -297,8 +308,10 @@ public class TestScanner extends HBaseTestCase { } if(serverName != null) { - assertTrue(hasColumn(results, HConstants.COL_SERVER)); - val = getColumn(results, HConstants.COL_SERVER).getValue(); + assertTrue(hasColumn(results, HConstants.CATALOG_FAMILY, + HConstants.SERVER_QUALIFIER)); + val = getColumn(results, HConstants.CATALOG_FAMILY, + HConstants.SERVER_QUALIFIER).getValue(); assertNotNull(val); assertFalse(val.length == 0); String server = Bytes.toString(val); @@ -317,27 +330,33 @@ public class TestScanner extends HBaseTestCase { } } - private boolean hasColumn(final List kvs, final byte [] column) { + private boolean hasColumn(final List kvs, final byte [] family, + final byte [] qualifier) { for (KeyValue kv: kvs) { - if (kv.matchingColumn(column)) { + if (kv.matchingFamily(family) && kv.matchingQualifier(qualifier)) { return true; } } - return false; + return false; } - - private KeyValue getColumn(final List kvs, final byte [] column) { + + private KeyValue getColumn(final List kvs, final byte [] family, + final byte [] qualifier) { for (KeyValue kv: kvs) { - if (kv.matchingColumn(column)) { + if (kv.matchingFamily(family) && kv.matchingQualifier(qualifier)) { return kv; } } return null; } - + + /** Use get to retrieve the HRegionInfo and validate it */ private void getRegionInfo() throws IOException { - byte [] bytes = region.get(ROW_KEY, HConstants.COL_REGIONINFO).getValue(); + Get get = new Get(ROW_KEY); + get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + Result result = region.get(get, null); + byte [] bytes = result.value(); validateRegionInfo(bytes); } @@ -349,8 +368,9 @@ public class TestScanner extends HBaseTestCase { this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null); HRegionIncommon hri = new HRegionIncommon(r); try { - LOG.info("Added: " + - addContent(hri, Bytes.toString(HConstants.COL_REGIONINFO))); + String columnString = Bytes.toString(HConstants.CATALOG_FAMILY) + ':' + + Bytes.toString(HConstants.REGIONINFO_QUALIFIER); + LOG.info("Added: " + addContent(hri, columnString)); int count = count(hri, -1); assertEquals(count, count(hri, 100)); assertEquals(count, count(hri, 0)); @@ -374,7 +394,7 @@ public class TestScanner extends HBaseTestCase { private int count(final HRegionIncommon hri, final int flushIndex) throws IOException { LOG.info("Taking out counting scan"); - ScannerIncommon s = hri.getScanner(EXPLICIT_COLS, + ScannerIncommon s = hri.getScanner(HConstants.CATALOG_FAMILY, EXPLICIT_COLS, HConstants.EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP); List values = new ArrayList(); int count = 0; diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java b/src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java deleted file mode 100644 index 99c24686199..00000000000 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java +++ /dev/null @@ -1,264 +0,0 @@ -/** - * Copyright 2007 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver; - -import java.io.IOException; -import java.util.TreeMap; -import java.util.List; -import java.util.ArrayList; - -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.KeyValue; -import org.apache.hadoop.hbase.UnknownScannerException; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * {@Link TestHRegion} does a split but this TestCase adds testing of fast - * split and manufactures odd-ball split scenarios. - */ -public class TestSplit extends HBaseClusterTestCase { - static final Log LOG = LogFactory.getLog(TestSplit.class.getName()); - - /** constructor */ - public TestSplit() { - super(); - - // Always compact if there is more than one store file. - conf.setInt("hbase.hstore.compactionThreshold", 2); - - // Make lease timeout longer, lease checks less frequent - conf.setInt("hbase.master.lease.period", 10 * 1000); - conf.setInt("hbase.master.lease.thread.wakefrequency", 5 * 1000); - - conf.setInt("hbase.regionserver.lease.period", 10 * 1000); - - // Increase the amount of time between client retries - conf.setLong("hbase.client.pause", 15 * 1000); - - // This size should make it so we always split using the addContent - // below. After adding all data, the first region is 1.3M - conf.setLong("hbase.hregion.max.filesize", 1024 * 128); - } - - /** - * Splits twice and verifies getting from each of the split regions. - * @throws Exception - */ - public void testBasicSplit() throws Exception { - HRegion region = null; - try { - HTableDescriptor htd = createTableDescriptor(getName()); - htd.addFamily(new HColumnDescriptor(COLFAMILY_NAME3)); - region = createNewHRegion(htd, null, null); - basicSplit(region); - } finally { - if (region != null) { - region.close(); - region.getLog().closeAndDelete(); - } - } - } - - /** - * Test for HBASE-810 - * @throws Exception - */ - public void testScanSplitOnRegion() throws Exception { - HRegion region = null; - try { - HTableDescriptor htd = createTableDescriptor(getName()); - htd.addFamily(new HColumnDescriptor(COLFAMILY_NAME3)); - region = createNewHRegion(htd, null, null); - addContent(region, COLFAMILY_NAME3); - region.flushcache(); - final byte [] midkey = region.compactStores(); - assertNotNull(midkey); - byte [][] cols = {COLFAMILY_NAME3}; - final InternalScanner s = region.getScanner(cols, - HConstants.EMPTY_START_ROW, System.currentTimeMillis(), null); - final HRegion regionForThread = region; - - Thread splitThread = new Thread() { - @Override - public void run() { - try { - split(regionForThread, midkey); - } catch (IOException e) { - fail("Unexpected exception " + e); - } - } - }; - splitThread.start(); - HRegionServer server = cluster.getRegionThreads().get(0).getRegionServer(); - long id = server.addScanner(s); - for(int i = 0; i < 6; i++) { - try { - BatchUpdate update = - new BatchUpdate(region.getRegionInfo().getStartKey()); - update.put(COLFAMILY_NAME3, Bytes.toBytes("val")); - region.batchUpdate(update); - Thread.sleep(1000); - } - catch (InterruptedException e) { - fail("Unexpected exception " + e); - } - } - server.next(id); - server.close(id); - } catch(UnknownScannerException ex) { - ex.printStackTrace(); - fail("Got the " + ex); - } - } - - private void basicSplit(final HRegion region) throws Exception { - LOG.info("" + addContent(region, COLFAMILY_NAME3)); - region.flushcache(); - byte [] splitRow = region.compactStores(); - assertNotNull(splitRow); - LOG.info("SplitRow: " + Bytes.toString(splitRow)); - HRegion [] regions = split(region, splitRow); - try { - // Need to open the regions. - // TODO: Add an 'open' to HRegion... don't do open by constructing - // instance. - for (int i = 0; i < regions.length; i++) { - regions[i] = openClosedRegion(regions[i]); - } - // Assert can get rows out of new regions. Should be able to get first - // row from first region and the midkey from second region. - assertGet(regions[0], COLFAMILY_NAME3, Bytes.toBytes(START_KEY)); - assertGet(regions[1], COLFAMILY_NAME3, splitRow); - // Test I can get scanner and that it starts at right place. - assertScan(regions[0], COLFAMILY_NAME3, - Bytes.toBytes(START_KEY)); - assertScan(regions[1], COLFAMILY_NAME3, splitRow); - // Now prove can't split regions that have references. - for (int i = 0; i < regions.length; i++) { - // Add so much data to this region, we create a store file that is > - // than one of our unsplitable references. it will. - for (int j = 0; j < 2; j++) { - addContent(regions[i], COLFAMILY_NAME3); - } - addContent(regions[i], COLFAMILY_NAME2); - addContent(regions[i], COLFAMILY_NAME1); - regions[i].flushcache(); - } - - byte [][] midkeys = new byte [regions.length][]; - // To make regions splitable force compaction. - for (int i = 0; i < regions.length; i++) { - midkeys[i] = regions[i].compactStores(); - } - - TreeMap sortedMap = new TreeMap(); - // Split these two daughter regions so then I'll have 4 regions. Will - // split because added data above. - for (int i = 0; i < regions.length; i++) { - HRegion[] rs = null; - if (midkeys[i] != null) { - rs = split(regions[i], midkeys[i]); - for (int j = 0; j < rs.length; j++) { - sortedMap.put(Bytes.toString(rs[j].getRegionName()), - openClosedRegion(rs[j])); - } - } - } - LOG.info("Made 4 regions"); - // The splits should have been even. Test I can get some arbitrary row out - // of each. - int interval = (LAST_CHAR - FIRST_CHAR) / 3; - byte[] b = Bytes.toBytes(START_KEY); - for (HRegion r : sortedMap.values()) { - assertGet(r, COLFAMILY_NAME3, b); - b[0] += interval; - } - } finally { - for (int i = 0; i < regions.length; i++) { - try { - regions[i].close(); - } catch (IOException e) { - // Ignore. - } - } - } - } - - private void assertGet(final HRegion r, final byte [] family, final byte [] k) - throws IOException { - // Now I have k, get values out and assert they are as expected. - Cell[] results = Cell.createSingleCellArray(r.get(k, family, -1, Integer.MAX_VALUE)); - for (int j = 0; j < results.length; j++) { - byte [] tmp = results[j].getValue(); - // Row should be equal to value every time. - assertTrue(Bytes.equals(k, tmp)); - } - } - - /* - * Assert first value in the passed region is firstValue. - * @param r - * @param column - * @param firstValue - * @throws IOException - */ - private void assertScan(final HRegion r, final byte [] column, - final byte [] firstValue) - throws IOException { - byte [][] cols = {column}; - InternalScanner s = r.getScanner(cols, - HConstants.EMPTY_START_ROW, System.currentTimeMillis(), null); - try { - List curVals = new ArrayList(); - boolean first = true; - OUTER_LOOP: while(s.next(curVals)) { - for (KeyValue kv: curVals) { - byte [] val = kv.getValue(); - byte [] curval = val; - if (first) { - first = false; - assertTrue(Bytes.compareTo(curval, firstValue) == 0); - } else { - // Not asserting anything. Might as well break. - break OUTER_LOOP; - } - } - } - } finally { - s.close(); - } - } - - protected HRegion [] split(final HRegion r, final byte [] splitRow) - throws IOException { - // Assert can get mid key from passed region. - assertGet(r, COLFAMILY_NAME3, splitRow); - HRegion [] regions = r.splitRegion(splitRow); - assertEquals(regions.length, 2); - return regions; - } -} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestStore.java b/src/test/org/apache/hadoop/hbase/regionserver/TestStore.java new file mode 100644 index 00000000000..b8618d57dc1 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestStore.java @@ -0,0 +1,325 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NavigableSet; +import java.util.concurrent.ConcurrentSkipListSet; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.util.Progressable; + +import junit.framework.TestCase; + +/** + * Test class fosr the Store + */ +public class TestStore extends TestCase { + Store store; + byte [] table = Bytes.toBytes("table"); + byte [] family = Bytes.toBytes("family"); + + byte [] row = Bytes.toBytes("row"); + byte [] qf1 = Bytes.toBytes("qf1"); + byte [] qf2 = Bytes.toBytes("qf2"); + byte [] qf3 = Bytes.toBytes("qf3"); + byte [] qf4 = Bytes.toBytes("qf4"); + byte [] qf5 = Bytes.toBytes("qf5"); + byte [] qf6 = Bytes.toBytes("qf6"); + + NavigableSet qualifiers = + new ConcurrentSkipListSet(Bytes.BYTES_COMPARATOR); + + List expected = new ArrayList(); + List result = new ArrayList(); + + long id = System.currentTimeMillis(); + Get get = new Get(row); + + private final String DIR = "test/build/data/TestStore/"; + + /** + * Setup + * @throws IOException + */ + @Override + public void setUp() throws IOException { + qualifiers.add(qf1); + qualifiers.add(qf3); + qualifiers.add(qf5); + + Iterator iter = qualifiers.iterator(); + while(iter.hasNext()){ + byte [] next = iter.next(); + expected.add(new KeyValue(row, family, next, null)); + get.addColumn(family, next); + } + } + + private void init(String methodName) throws IOException { + //Setting up a Store + Path basedir = new Path(DIR+methodName); + HColumnDescriptor hcd = new HColumnDescriptor(family); + HBaseConfiguration conf = new HBaseConfiguration(); + FileSystem fs = FileSystem.get(conf); + Path reconstructionLog = null; + Progressable reporter = null; + + HTableDescriptor htd = new HTableDescriptor(table); + htd.addFamily(hcd); + HRegionInfo info = new HRegionInfo(htd, null, null, false); + + store = new Store(basedir, info, hcd, fs, reconstructionLog, conf, + reporter); + } + + + ////////////////////////////////////////////////////////////////////////////// + // Get tests + ////////////////////////////////////////////////////////////////////////////// + /** + * Getting data from memcache only + * @throws IOException + */ + public void testGet_FromMemCacheOnly() throws IOException { + init(this.getName()); + + //Put data in memcache + this.store.add(new KeyValue(row, family, qf1, null)); + this.store.add(new KeyValue(row, family, qf2, null)); + this.store.add(new KeyValue(row, family, qf3, null)); + this.store.add(new KeyValue(row, family, qf4, null)); + this.store.add(new KeyValue(row, family, qf5, null)); + this.store.add(new KeyValue(row, family, qf6, null)); + + //Get + this.store.get(get, qualifiers, result); + + //Compare + assertCheck(); + } + + /** + * Getting data from files only + * @throws IOException + */ + public void testGet_FromFilesOnly() throws IOException { + init(this.getName()); + + //Put data in memcache + this.store.add(new KeyValue(row, family, qf1, null)); + this.store.add(new KeyValue(row, family, qf2, null)); + //flush + flush(1); + + //Add more data + this.store.add(new KeyValue(row, family, qf3, null)); + this.store.add(new KeyValue(row, family, qf4, null)); + //flush + flush(2); + + //Add more data + this.store.add(new KeyValue(row, family, qf5, null)); + this.store.add(new KeyValue(row, family, qf6, null)); + //flush + flush(3); + + //Get + this.store.get(get, qualifiers, result); + + //Need to sort the result since multiple files + Collections.sort(result, KeyValue.COMPARATOR); + + //Compare + assertCheck(); + } + + /** + * Getting data from memcache and files + * @throws IOException + */ + public void testGet_FromMemCacheAndFiles() throws IOException { + init(this.getName()); + + //Put data in memcache + this.store.add(new KeyValue(row, family, qf1, null)); + this.store.add(new KeyValue(row, family, qf2, null)); + //flush + flush(1); + + //Add more data + this.store.add(new KeyValue(row, family, qf3, null)); + this.store.add(new KeyValue(row, family, qf4, null)); + //flush + flush(2); + + //Add more data + this.store.add(new KeyValue(row, family, qf5, null)); + this.store.add(new KeyValue(row, family, qf6, null)); + + //Get + this.store.get(get, qualifiers, result); + + //Need to sort the result since multiple files + Collections.sort(result, KeyValue.COMPARATOR); + + //Compare + assertCheck(); + } + + private void flush(int storeFilessize) throws IOException{ + this.store.snapshot(); + this.store.flushCache(id++); + assertEquals(storeFilessize, this.store.getStorefiles().size()); + assertEquals(0, this.store.memcache.memcache.size()); + } + + private void assertCheck() { + assertEquals(expected.size(), result.size()); + for(int i=0; i qualifiers = + new ConcurrentSkipListSet(Bytes.BYTES_COMPARATOR); + qualifiers.add(qf1); + List result = new ArrayList(); + this.store.get(get, qualifiers, result); + assertEquals(value + amount, Bytes.toLong(result.get(0).getValue())); + } + + /** + * Same as above but for a negative number + * @throws IOException + */ + public void testIncrementColumnValue_UpdatingInPlace_Negative() + throws IOException { + init(this.getName()); + + //Put data in memcache + long value = 3L; + long amount = -1L; + this.store.add(new KeyValue(row, family, qf1, Bytes.toBytes(value))); + + this.store.incrementColumnValue(row, family, qf1, amount); + Get get = new Get(row); + get.addColumn(family, qf1); + NavigableSet qualifiers = + new ConcurrentSkipListSet(Bytes.BYTES_COMPARATOR); + qualifiers.add(qf1); + List result = new ArrayList(); + this.store.get(get, qualifiers, result); + assertEquals(value + amount, Bytes.toLong(result.get(0).getValue())); + } + + /** + * When there is no mathing key already, adding a new. + * @throws IOException + */ + public void testIncrementColumnValue_AddingNew() throws IOException { + init(this.getName()); + + //Put data in memcache + long value = 1L; + long amount = 3L; + this.store.add(new KeyValue(row, family, qf1, Bytes.toBytes(value))); + this.store.add(new KeyValue(row, family, qf2, Bytes.toBytes(value))); + + this.store.incrementColumnValue(row, family, qf3, amount); + Get get = new Get(row); + get.addColumn(family, qf3); + NavigableSet qualifiers = + new ConcurrentSkipListSet(Bytes.BYTES_COMPARATOR); + qualifiers.add(qf3); + List result = new ArrayList(); + this.store.get(get, qualifiers, result); + assertEquals(amount, Bytes.toLong(result.get(0).getValue())); + } + + /** + * When we have the key in a file add a new key + value to memcache with the + * updates value. + * @throws IOException + */ + public void testIncrementColumnValue_UpdatingFromSF() throws IOException { + init(this.getName()); + + //Put data in memcache + long value = 1L; + long amount = 3L; + this.store.add(new KeyValue(row, family, qf1, Bytes.toBytes(value))); + this.store.add(new KeyValue(row, family, qf2, Bytes.toBytes(value))); + + flush(1); + + this.store.incrementColumnValue(row, family, qf1, amount); + Get get = new Get(row); + get.addColumn(family, qf1); + NavigableSet qualifiers = + new ConcurrentSkipListSet(Bytes.BYTES_COMPARATOR); + qualifiers.add(qf1); + List result = new ArrayList(); + this.store.get(get, qualifiers, result); + assertEquals(value + amount, Bytes.toLong(result.get(0).getValue())); + } + + /** + * Same as testIncrementColumnValue_AddingNew() except that the keys are + * checked in file not in memcache + * @throws IOException + */ + public void testIncrementColumnValue_AddingNewAfterSFCheck() + throws IOException { + init(this.getName()); + + //Put data in memcache + long value = 1L; + long amount = 3L; + this.store.add(new KeyValue(row, family, qf1, Bytes.toBytes(value))); + this.store.add(new KeyValue(row, family, qf2, Bytes.toBytes(value))); + + flush(1); + + this.store.incrementColumnValue(row, family, qf3, amount); + Get get = new Get(row); + get.addColumn(family, qf3); + NavigableSet qualifiers = + new ConcurrentSkipListSet(Bytes.BYTES_COMPARATOR); + qualifiers.add(qf3); + List result = new ArrayList(); + this.store.get(get, qualifiers, result); + assertEquals(amount, Bytes.toLong(result.get(0).getValue())); + } + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/src/test/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java new file mode 100644 index 00000000000..cadab455f2e --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java @@ -0,0 +1,480 @@ +/* + * Copyright 2009 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver; + +import junit.framework.TestCase; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueTestUtil; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.RowInclusiveStopFilter; +import org.apache.hadoop.hbase.filter.RowWhileMatchFilter; +import org.apache.hadoop.hbase.filter.RowPrefixFilter; +import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.filter.WhileMatchRowFilter; +import org.apache.hadoop.hbase.filter.PrefixRowFilter; +import org.apache.hadoop.hbase.filter.InclusiveStopRowFilter; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NavigableSet; +import java.util.TreeSet; + + +public class TestStoreScanner extends TestCase { + + final byte [] CF = Bytes.toBytes("cf"); + + /** + * Test utility for building a NavigableSet for scanners. + * @param strCols + * @return + */ + NavigableSet getCols(String ...strCols) { + NavigableSet cols = new TreeSet(Bytes.BYTES_COMPARATOR); + for (String col : strCols) { + byte[] bytes = Bytes.toBytes(col); + cols.add(bytes); + } + return cols; + } + + /** + * Test test shows exactly how the matcher's return codes confuses the StoreScanner + * and prevent it from doing the right thing. Seeking once, then nexting twice + * should return R1, then R2, but in this case it doesnt. + * @throws IOException + */ + public void testWontNextToNext() throws IOException { + // build the scan file: + KeyValue [] kvs = new KeyValue[] { + KeyValueTestUtil.create("R1", "cf", "a", 2, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", 1, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R2", "cf", "a", 1, KeyValue.Type.Put, "dont-care") + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, + kvs) + }; + + Scan scanSpec = new Scan(Bytes.toBytes("R1")); + // this only uses maxVersions (default=1) and TimeRange (default=all) + StoreScanner scan = + new StoreScanner(scanSpec, CF, Long.MAX_VALUE, + KeyValue.COMPARATOR, getCols("a"), + scanners); + + List results = new ArrayList(); + scan.next(results); + assertEquals(1, results.size()); + assertEquals(kvs[0], results.get(0)); + // should be ok... + // now scan _next_ again. + results.clear(); + scan.next(results); + assertEquals(1, results.size()); + assertEquals(kvs[2], results.get(0)); + + results.clear(); + scan.next(results); + assertEquals(0, results.size()); + + } + + + public void testDeleteVersionSameTimestamp() throws IOException { + KeyValue [] kvs = new KeyValue [] { + KeyValueTestUtil.create("R1", "cf", "a", 1, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", 1, KeyValue.Type.Delete, "dont-care"), + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) + }; + Scan scanSpec = new Scan(Bytes.toBytes("R1")); + StoreScanner scan = + new StoreScanner(scanSpec, CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + getCols("a"), scanners); + + List results = new ArrayList(); + assertFalse(scan.next(results)); + assertEquals(0, results.size()); + } + + /** + * Test the case where there is a delete row 'in front of' the next row, the scanner + * will move to the next row. + */ + public void testDeletedRowThenGoodRow() throws IOException { + KeyValue [] kvs = new KeyValue [] { + KeyValueTestUtil.create("R1", "cf", "a", 1, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", 1, KeyValue.Type.Delete, "dont-care"), + KeyValueTestUtil.create("R2", "cf", "a", 20, KeyValue.Type.Put, "dont-care") + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) + }; + Scan scanSpec = new Scan(Bytes.toBytes("R1")); + StoreScanner scan = + new StoreScanner(scanSpec, CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + getCols("a"), scanners); + + List results = new ArrayList(); + assertEquals(true, scan.next(results)); + assertEquals(1, results.size()); + assertEquals(kvs[2], results.get(0)); + } + + public void testDeleteVersionMaskingMultiplePuts() throws IOException { + long now = System.currentTimeMillis(); + KeyValue [] kvs1 = new KeyValue[] { + KeyValueTestUtil.create("R1", "cf", "a", now, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", now, KeyValue.Type.Delete, "dont-care") + }; + KeyValue [] kvs2 = new KeyValue[] { + KeyValueTestUtil.create("R1", "cf", "a", now-500, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", now-100, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", now, KeyValue.Type.Put, "dont-care") + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs1), + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs2) + }; + StoreScanner scan = + new StoreScanner(new Scan(Bytes.toBytes("R1")), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + getCols("a"), scanners); + List results = new ArrayList(); + // the two put at ts=now will be masked by the 1 delete, and + // since the scan default returns 1 version we'll return the newest + // key, which is kvs[2], now-100. + assertEquals(true, scan.next(results)); + assertEquals(1, results.size()); + assertEquals(kvs2[1], results.get(0)); + } + public void testDeleteVersionsMixedAndMultipleVersionReturn() throws IOException { + long now = System.currentTimeMillis(); + KeyValue [] kvs1 = new KeyValue[] { + KeyValueTestUtil.create("R1", "cf", "a", now, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", now, KeyValue.Type.Delete, "dont-care") + }; + KeyValue [] kvs2 = new KeyValue[] { + KeyValueTestUtil.create("R1", "cf", "a", now-500, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", now+500, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", now, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R2", "cf", "z", now, KeyValue.Type.Put, "dont-care") + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs1), + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs2) + }; + Scan scanSpec = new Scan(Bytes.toBytes("R1")).setMaxVersions(2); + StoreScanner scan = + new StoreScanner(scanSpec, CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + getCols("a"), scanners); + List results = new ArrayList(); + assertEquals(true, scan.next(results)); + assertEquals(2, results.size()); + assertEquals(kvs2[1], results.get(0)); + assertEquals(kvs2[0], results.get(1)); + } + + public void testWildCardOneVersionScan() throws IOException { + KeyValue [] kvs = new KeyValue [] { + KeyValueTestUtil.create("R1", "cf", "a", 2, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "b", 1, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", 1, KeyValue.Type.DeleteColumn, "dont-care"), + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) + }; + StoreScanner scan = + new StoreScanner(new Scan(Bytes.toBytes("R1")), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + null, scanners); + List results = new ArrayList(); + assertEquals(true, scan.next(results)); + assertEquals(2, results.size()); + assertEquals(kvs[0], results.get(0)); + assertEquals(kvs[1], results.get(1)); + } + public void testWildCardScannerUnderDeletes() throws IOException { + KeyValue [] kvs = new KeyValue [] { + KeyValueTestUtil.create("R1", "cf", "a", 2, KeyValue.Type.Put, "dont-care"), // inc + // orphaned delete column. + KeyValueTestUtil.create("R1", "cf", "a", 1, KeyValue.Type.DeleteColumn, "dont-care"), + // column b + KeyValueTestUtil.create("R1", "cf", "b", 2, KeyValue.Type.Put, "dont-care"), // inc + KeyValueTestUtil.create("R1", "cf", "b", 1, KeyValue.Type.Put, "dont-care"), // inc + // column c + KeyValueTestUtil.create("R1", "cf", "c", 10, KeyValue.Type.Delete, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "c", 10, KeyValue.Type.Put, "dont-care"), // no + KeyValueTestUtil.create("R1", "cf", "c", 9, KeyValue.Type.Put, "dont-care"), // inc + // column d + KeyValueTestUtil.create("R1", "cf", "d", 11, KeyValue.Type.Put, "dont-care"), // inc + KeyValueTestUtil.create("R1", "cf", "d", 10, KeyValue.Type.DeleteColumn, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "d", 9, KeyValue.Type.Put, "dont-care"), // no + KeyValueTestUtil.create("R1", "cf", "d", 8, KeyValue.Type.Put, "dont-care"), // no + + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) + }; + StoreScanner scan = + new StoreScanner(new Scan().setMaxVersions(2), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + null, scanners); + List results = new ArrayList(); + assertEquals(true, scan.next(results)); + assertEquals(5, results.size()); + assertEquals(kvs[0], results.get(0)); + assertEquals(kvs[2], results.get(1)); + assertEquals(kvs[3], results.get(2)); + assertEquals(kvs[6], results.get(3)); + assertEquals(kvs[7], results.get(4)); + } + public void testDeleteFamily() throws IOException { + KeyValue [] kvs = new KeyValue[] { + KeyValueTestUtil.create("R1", "cf", "a", 100, KeyValue.Type.DeleteFamily, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "b", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "d", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "e", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "e", 11, KeyValue.Type.DeleteColumn, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "f", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "g", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "g", 11, KeyValue.Type.Delete, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "h", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "i", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R2", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) + }; + StoreScanner scan = + new StoreScanner(new Scan().setMaxVersions(Integer.MAX_VALUE), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + null, scanners); + List results = new ArrayList(); + assertEquals(true, scan.next(results)); + assertEquals(1, results.size()); + assertEquals(kvs[kvs.length-1], results.get(0)); + } + + public void testDeleteColumn() throws IOException { + KeyValue [] kvs = new KeyValue[] { + KeyValueTestUtil.create("R1", "cf", "a", 10, KeyValue.Type.DeleteColumn, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", 9, KeyValue.Type.Delete, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", 8, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "b", 5, KeyValue.Type.Put, "dont-care") + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs), + }; + StoreScanner scan = + new StoreScanner(new Scan(), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + null, scanners); + List results = new ArrayList(); + assertEquals(true, scan.next(results)); + assertEquals(1, results.size()); + assertEquals(kvs[3], results.get(0)); + } + + public void testSkipColumn() throws IOException { + KeyValue [] kvs = new KeyValue[] { + KeyValueTestUtil.create("R1", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "b", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "d", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "e", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "f", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "g", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "h", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "i", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R2", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + }; + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) + }; + StoreScanner scan = + new StoreScanner(new Scan(), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + getCols("a", "d"), scanners); + + List results = new ArrayList(); + assertEquals(true, scan.next(results)); + assertEquals(2, results.size()); + assertEquals(kvs[0], results.get(0)); + assertEquals(kvs[3], results.get(1)); + results.clear(); + + assertEquals(true, scan.next(results)); + assertEquals(1, results.size()); + assertEquals(kvs[kvs.length-1], results.get(0)); + + results.clear(); + assertEquals(false, scan.next(results)); + } + + KeyValue [] stdKvs = new KeyValue[] { + KeyValueTestUtil.create("R:1", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:1", "cf", "b", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:1", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:1", "cf", "d", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:1", "cf", "e", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:1", "cf", "f", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:1", "cf", "g", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:1", "cf", "h", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:1", "cf", "i", 11, KeyValue.Type.Put, "dont-care"), + + // 9... + KeyValueTestUtil.create("R:2", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:2", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:2", "cf", "c", 10, KeyValue.Type.Put, "dont-care"), + + // 12... + KeyValueTestUtil.create("R:3", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:3", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:3", "cf", "c", 10, KeyValue.Type.Put, "dont-care"), + + // 15 ... + KeyValueTestUtil.create("R:4", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:4", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:4", "cf", "c", 10, KeyValue.Type.Put, "dont-care"), + + // 18 .. + KeyValueTestUtil.create("R:5", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:5", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), + + // 20... + KeyValueTestUtil.create("R:6", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:6", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), + + // 22... + KeyValueTestUtil.create("R:7", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:7", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), + + // 24... + KeyValueTestUtil.create("R:8", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R:8", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), + + // 26 .. + KeyValueTestUtil.create("RA:1", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + + // 27... + KeyValueTestUtil.create("RA:2", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + + // 28.. + KeyValueTestUtil.create("RA:3", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), + }; + private StoreScanner getTestScanner(Scan s, NavigableSet cols) { + KeyValueScanner [] scanners = new KeyValueScanner[] { + new KeyValueScanFixture(KeyValue.COMPARATOR, stdKvs) + }; + + return new StoreScanner(s, CF, Long.MAX_VALUE, KeyValue.COMPARATOR, cols, + scanners); + } + + + // Test new and old row prefix filters. + public void testNewRowPrefixFilter() throws IOException { + Filter f = new RowWhileMatchFilter( + new RowPrefixFilter(Bytes.toBytes("R:"))); + Scan s = new Scan(Bytes.toBytes("R:7")); + s.setFilter(f); + + rowPrefixFilter(s); + } + + public void testOldRowPrefixFilter() throws IOException { + RowFilterInterface f = new WhileMatchRowFilter( + new PrefixRowFilter(Bytes.toBytes("R:"))); + Scan s = new Scan(Bytes.toBytes("R:7")); + s.setOldFilter(f); + + rowPrefixFilter(s); + + } + public void rowPrefixFilter(Scan s) throws IOException { + + StoreScanner scan = getTestScanner(s, null); + + List results = new ArrayList(); + assertTrue(scan.next(results)); + assertEquals(2, results.size()); + assertEquals(stdKvs[22], results.get(0)); + assertEquals(stdKvs[23], results.get(1)); + results.clear(); + + assertTrue(scan.next(results)); + assertEquals(2, results.size()); + assertEquals(stdKvs[24], results.get(0)); + assertEquals(stdKvs[25], results.get(1)); + results.clear(); + + assertFalse(scan.next(results)); + assertEquals(0, results.size()); + } + + // Test new and old row-inclusive stop filter. + public void testNewRowInclusiveStopFilter() throws IOException { + Filter f = new RowWhileMatchFilter(new RowInclusiveStopFilter(Bytes.toBytes("R:3"))); + Scan scan = new Scan(); + scan.setFilter(f); + + rowInclusiveStopFilter(scan); + } + + public void testOldRowInclusiveTopFilter() throws IOException { + RowFilterInterface f = new WhileMatchRowFilter( + new InclusiveStopRowFilter(Bytes.toBytes("R:3"))); + Scan scan = new Scan(); + scan.setOldFilter(f); + + rowInclusiveStopFilter(scan); + } + + public void rowInclusiveStopFilter(Scan scan) throws IOException { + StoreScanner s = getTestScanner(scan, getCols("a")); + + // read crap. + List results = new ArrayList(); + assertTrue(s.next(results)); + assertEquals(1, results.size()); + assertEquals(stdKvs[0], results.get(0)); + results.clear(); + + assertTrue(s.next(results)); + assertEquals(1, results.size()); + assertEquals(stdKvs[9], results.get(0)); + results.clear(); + + assertTrue(s.next(results)); + assertEquals(1, results.size()); + assertEquals(stdKvs[12], results.get(0)); + results.clear(); + + // without aggressive peeking, the scanner doesnt know if the next row is good or not + // under the affects of a filter. + assertFalse(s.next(results)); + assertEquals(0, results.size()); + } + + + +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java b/src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java deleted file mode 100644 index 04da57c1ada..00000000000 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java +++ /dev/null @@ -1,85 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hbase.regionserver; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HBaseClusterTestCase; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TimestampTestBase; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Tests user specifiable time stamps putting, getting and scanning. Also - * tests same in presence of deletes. Test cores are written so can be - * run against an HRegion and against an HTable: i.e. both local and remote. - */ -public class TestTimestamp extends HBaseClusterTestCase { - private static final Log LOG = - LogFactory.getLog(TestTimestamp.class.getName()); - - private static final String COLUMN_NAME = "contents:"; - private static final byte [] COLUMN = Bytes.toBytes(COLUMN_NAME); - private static final int VERSIONS = 3; - - /** - * Test that delete works according to description in hadoop-1784. - * @throws IOException - */ - public void testDelete() throws IOException { - final HRegion r = createRegion(); - try { - final HRegionIncommon region = new HRegionIncommon(r); - TimestampTestBase.doTestDelete(region, region); - } finally { - r.close(); - r.getLog().closeAndDelete(); - } - LOG.info("testDelete() finished"); - } - - /** - * Test scanning against different timestamps. - * @throws IOException - */ - public void testTimestampScanning() throws IOException { - final HRegion r = createRegion(); - try { - final HRegionIncommon region = new HRegionIncommon(r); - TimestampTestBase.doTestTimestampScanning(region, region); - } finally { - r.close(); - r.getLog().closeAndDelete(); - } - LOG.info("testTimestampScanning() finished"); - } - - private HRegion createRegion() throws IOException { - HTableDescriptor htd = createTableDescriptor(getName()); - htd.addFamily(new HColumnDescriptor(COLUMN, VERSIONS, - HColumnDescriptor.DEFAULT_COMPRESSION, false, false, - Integer.MAX_VALUE, HConstants.FOREVER, false)); - return createNewHRegion(htd, null, null); - } -} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestWildcardColumnTracker.java b/src/test/org/apache/hadoop/hbase/regionserver/TestWildcardColumnTracker.java new file mode 100644 index 00000000000..97d5fff3673 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestWildcardColumnTracker.java @@ -0,0 +1,336 @@ +package org.apache.hadoop.hbase.regionserver; + +import java.util.ArrayList; +import java.util.List; +import java.util.TreeSet; + +import org.apache.hadoop.hbase.HBaseTestCase; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode; +import org.apache.hadoop.hbase.util.Bytes; + +public class TestWildcardColumnTracker extends HBaseTestCase +implements HConstants { + private boolean PRINT = false; + + public void testGet_SingleVersion() { + if(PRINT) { + System.out.println("SingleVersion"); + } + byte [] col1 = Bytes.toBytes("col1"); + byte [] col2 = Bytes.toBytes("col2"); + byte [] col3 = Bytes.toBytes("col3"); + byte [] col4 = Bytes.toBytes("col4"); + byte [] col5 = Bytes.toBytes("col5"); + + //Create tracker + List expected = new ArrayList(); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + expected.add(MatchCode.INCLUDE); + int maxVersions = 1; + + ColumnTracker exp = new WildcardColumnTracker(maxVersions); + + //Create "Scanner" + List scanner = new ArrayList(); + scanner.add(col1); + scanner.add(col2); + scanner.add(col3); + scanner.add(col4); + scanner.add(col5); + + //Initialize result + List result = new ArrayList(); + + //"Match" + for(byte [] col : scanner){ + result.add(exp.checkColumn(col, 0, col.length)); + } + + assertEquals(expected.size(), result.size()); + for(int i=0; i< expected.size(); i++){ + assertEquals(expected.get(i), result.get(i)); + if(PRINT){ + System.out.println("Expected " +expected.get(i) + ", actual " + + result.get(i)); + } + } + } + + + public void testGet_MultiVersion() { + if(PRINT) { + System.out.println("\nMultiVersion"); + } + byte [] col1 = Bytes.toBytes("col1"); + byte [] col2 = Bytes.toBytes("col2"); + byte [] col3 = Bytes.toBytes("col3"); + byte [] col4 = Bytes.toBytes("col4"); + byte [] col5 = Bytes.toBytes("col5"); + + //Create tracker + List expected = new ArrayList(); + int size = 5; + for(int i=0; i scanner = new ArrayList(); + scanner.add(col1); + scanner.add(col1); + scanner.add(col1); + scanner.add(col2); + scanner.add(col2); + scanner.add(col2); + scanner.add(col3); + scanner.add(col3); + scanner.add(col3); + scanner.add(col4); + scanner.add(col4); + scanner.add(col4); + scanner.add(col5); + scanner.add(col5); + scanner.add(col5); + + //Initialize result + List result = new ArrayList(); + + //"Match" + for(byte [] col : scanner){ + result.add(exp.checkColumn(col, 0, col.length)); + } + + assertEquals(expected.size(), result.size()); + for(int i=0; i< expected.size(); i++){ + assertEquals(expected.get(i), result.get(i)); + if(PRINT){ + System.out.println("Expected " +expected.get(i) + ", actual " + + result.get(i)); + } + } + } + + public void testUpdate_SameColumns(){ + if(PRINT) { + System.out.println("\nUpdate_SameColumns"); + } + byte [] col1 = Bytes.toBytes("col1"); + byte [] col2 = Bytes.toBytes("col2"); + byte [] col3 = Bytes.toBytes("col3"); + byte [] col4 = Bytes.toBytes("col4"); + byte [] col5 = Bytes.toBytes("col5"); + + //Create tracker + List expected = new ArrayList(); + int size = 10; + for(int i=0; i scanner = new ArrayList(); + scanner.add(col1); + scanner.add(col2); + scanner.add(col3); + scanner.add(col4); + scanner.add(col5); + + //Initialize result + List result = new ArrayList(); + + //"Match" + for(int i=0; i<3; i++){ + for(byte [] col : scanner){ + result.add(wild.checkColumn(col, 0, col.length)); + } + wild.update(); + } + + assertEquals(expected.size(), result.size()); + for(int i=0; i expected = new ArrayList(); + int size = 10; + for(int i=0; i scanner = new ArrayList(); + scanner.add(col0); + scanner.add(col1); + scanner.add(col2); + scanner.add(col3); + scanner.add(col4); + + //Initialize result + List result = new ArrayList(); + + for(byte [] col : scanner){ + result.add(wild.checkColumn(col, 0, col.length)); + } + wild.update(); + + //Create "Scanner1" + List scanner1 = new ArrayList(); + scanner1.add(col5); + scanner1.add(col6); + scanner1.add(col7); + scanner1.add(col8); + scanner1.add(col9); + for(byte [] col : scanner1){ + result.add(wild.checkColumn(col, 0, col.length)); + } + wild.update(); + + //Scanner again + for(byte [] col : scanner){ + result.add(wild.checkColumn(col, 0, col.length)); + } + + //"Match" + assertEquals(expected.size(), result.size()); + for(int i=0; i expected = new ArrayList(); + int size = 5; + for(int i=0; i scanner = new ArrayList(); + scanner.add(col0); + scanner.add(col2); + scanner.add(col4); + scanner.add(col6); + scanner.add(col8); + + //Initialize result + List result = new ArrayList(); + + for(int i=0; i<2; i++){ + for(byte [] col : scanner){ + result.add(wild.checkColumn(col, 0, col.length)); + } + wild.update(); + } + + //Create "Scanner1" + List scanner1 = new ArrayList(); + scanner1.add(col1); + scanner1.add(col3); + scanner1.add(col5); + scanner1.add(col7); + scanner1.add(col9); + for(byte [] col : scanner1){ + result.add(wild.checkColumn(col, 0, col.length)); + } + wild.update(); + + //Scanner again + for(byte [] col : scanner){ + result.add(wild.checkColumn(col, 0, col.length)); + } + + //"Match" + assertEquals(expected.size(), result.size()); + + for(int i=0; i regionThreads = cluster - .getRegionThreads(); - - HRegion region = null; - int server = -1; - for (int i = 0; i < regionThreads.size() && server == -1; i++) { - HRegionServer s = regionThreads.get(i).getRegionServer(); - Collection regions = s.getOnlineRegions(); - for (HRegion r : regions) { - if (Bytes.equals(r.getTableDesc().getName(), Bytes.toBytes(TABLE_NAME))) { - server = i; - region = r; - } - } - } - if (server == -1) { - LOG.fatal("could not find region server serving table region"); - fail(); - } - ((TransactionalRegionServer) regionThreads.get(server).getRegionServer()) - .getFlushRequester().request(region); - } - - /** - * Stop the region server serving TABLE_NAME. - * - * @param abort set to true if region server should be aborted, if false it is - * just shut down. - */ - private void stopOrAbortRegionServer(final boolean abort) { - List regionThreads = cluster - .getRegionThreads(); - - int server = -1; - for (int i = 0; i < regionThreads.size(); i++) { - HRegionServer s = regionThreads.get(i).getRegionServer(); - Collection regions = s.getOnlineRegions(); - LOG.info("server: " + regionThreads.get(i).getName()); - for (HRegion r : regions) { - LOG.info("region: " + r.getRegionInfo().getRegionNameAsString()); - if (Bytes.equals(r.getTableDesc().getName(), Bytes.toBytes(TABLE_NAME))) { - server = i; - } - } - } - if (server == -1) { - LOG.fatal("could not find region server serving table region"); - fail(); - } - if (abort) { - this.cluster.abortRegionServer(server); - - } else { - this.cluster.stopRegionServer(server, false); - } - LOG.info(this.cluster.waitOnRegionServer(server) + " has been " - + (abort ? "aborted" : "shut down")); - } - - protected void verify(final int numRuns) throws IOException { - // Reads - int row1 = Bytes.toInt(table.get(ROW1, COL_A).getValue()); - int row2 = Bytes.toInt(table.get(ROW2, COL_A).getValue()); - int row3 = Bytes.toInt(table.get(ROW3, COL_A).getValue()); - - assertEquals(TOTAL_VALUE - 2 * numRuns, row1); - assertEquals(numRuns, row2); - assertEquals(numRuns, row3); - } - - // Move 2 out of ROW1 and 1 into ROW2 and 1 into ROW3 - private TransactionState makeTransaction(final boolean flushMidWay) - throws IOException { - TransactionState transactionState = transactionManager.beginTransaction(); - - // Reads - int row1 = Bytes.toInt(table.get(transactionState, ROW1, COL_A).getValue()); - int row2 = Bytes.toInt(table.get(transactionState, ROW2, COL_A).getValue()); - int row3 = Bytes.toInt(table.get(transactionState, ROW3, COL_A).getValue()); - - row1 -= 2; - row2 += 1; - row3 += 1; - - if (flushMidWay) { - flushRegionServer(); - } - - // Writes - BatchUpdate write = new BatchUpdate(ROW1); - write.put(COL_A, Bytes.toBytes(row1)); - table.commit(transactionState, write); - - write = new BatchUpdate(ROW2); - write.put(COL_A, Bytes.toBytes(row2)); - table.commit(transactionState, write); - - write = new BatchUpdate(ROW3); - write.put(COL_A, Bytes.toBytes(row3)); - table.commit(transactionState, write); - - return transactionState; - } - - /* - * Run verification in a thread so I can concurrently run a thread-dumper - * while we're waiting (because in this test sometimes the meta scanner looks - * to be be stuck). @param tableName Name of table to find. @param row Row we - * expect to find. @return Verification thread. Caller needs to calls start on - * it. - */ - private Thread startVerificationThread(final int numRuns) { - Runnable runnable = new Runnable() { - public void run() { - try { - // Now try to open a scanner on the meta table. Should stall until - // meta server comes back up. - HTable t = new HTable(conf, TABLE_NAME); - Scanner s = t.getScanner(new byte[][] { COL_A }, - HConstants.EMPTY_START_ROW); - s.close(); - - } catch (IOException e) { - LOG.fatal("could not re-open meta table because", e); - fail(); - } - try { - verify(numRuns); - LOG.info("Success!"); - } catch (Exception e) { - e.printStackTrace(); - fail(); - } - } - }; - return new Thread(runnable); - } -} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/transactional/DisabledTestTransactionalHLogManager.java b/src/test/org/apache/hadoop/hbase/regionserver/transactional/DisabledTestTransactionalHLogManager.java index 9d36d4b15c8..e69de29bb2d 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/transactional/DisabledTestTransactionalHLogManager.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/transactional/DisabledTestTransactionalHLogManager.java @@ -1,308 +0,0 @@ -/** - * Copyright 2008 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.regionserver.transactional; - -import java.io.IOException; -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseTestCase; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.regionserver.HLog; -import org.apache.hadoop.hbase.util.Bytes; - -/** JUnit test case for HLog */ -public class DisabledTestTransactionalHLogManager extends HBaseTestCase implements - HConstants { - private Path dir; - private MiniDFSCluster cluster; - - final byte[] tableName = Bytes.toBytes("tablename"); - final HTableDescriptor tableDesc = new HTableDescriptor(tableName); - final HRegionInfo regionInfo = new HRegionInfo(tableDesc, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - final byte[] row1 = Bytes.toBytes("row1"); - final byte[] val1 = Bytes.toBytes("val1"); - final byte[] row2 = Bytes.toBytes("row2"); - final byte[] val2 = Bytes.toBytes("val2"); - final byte[] row3 = Bytes.toBytes("row3"); - final byte[] val3 = Bytes.toBytes("val3"); - final byte[] col = Bytes.toBytes("col:A"); - - @Override - public void setUp() throws Exception { - cluster = new MiniDFSCluster(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()); - super.setUp(); - this.dir = new Path("/hbase", getName()); - if (fs.exists(dir)) { - fs.delete(dir, true); - } - } - - @Override - public void tearDown() throws Exception { - if (this.fs.exists(this.dir)) { - this.fs.delete(this.dir, true); - } - shutdownDfs(cluster); - super.tearDown(); - } - - /** - * @throws IOException - */ - public void testSingleCommit() throws IOException { - - HLog log = new HLog(fs, dir, this.conf, null); - TransactionalHLogManager logMangaer = new TransactionalHLogManager(log, fs, - regionInfo, conf); - - // Write columns named 1, 2, 3, etc. and then values of single byte - // 1, 2, 3... - long transactionId = 1; - logMangaer.writeStartToLog(transactionId); - - BatchUpdate update1 = new BatchUpdate(row1); - update1.put(col, val1); - logMangaer.writeUpdateToLog(transactionId, update1); - - BatchUpdate update2 = new BatchUpdate(row2); - update2.put(col, val2); - logMangaer.writeUpdateToLog(transactionId, update2); - - BatchUpdate update3 = new BatchUpdate(row3); - update3.put(col, val3); - logMangaer.writeUpdateToLog(transactionId, update3); - - logMangaer.writeCommitToLog(transactionId); - - // log.completeCacheFlush(regionName, tableName, logSeqId); - - log.close(); - Path filename = log.computeFilename(log.getFilenum()); - - Map> commits = logMangaer.getCommitsFromLog( - filename, -1, null); - - assertEquals(1, commits.size()); - assertTrue(commits.containsKey(transactionId)); - assertEquals(3, commits.get(transactionId).size()); - - List updates = commits.get(transactionId); - - update1 = updates.get(0); - assertTrue(Bytes.equals(row1, update1.getRow())); - assertTrue(Bytes.equals(val1, update1.iterator().next().getValue())); - - update2 = updates.get(1); - assertTrue(Bytes.equals(row2, update2.getRow())); - assertTrue(Bytes.equals(val2, update2.iterator().next().getValue())); - - update3 = updates.get(2); - assertTrue(Bytes.equals(row3, update3.getRow())); - assertTrue(Bytes.equals(val3, update3.iterator().next().getValue())); - - } - - /** - * @throws IOException - */ - public void testSingleAbort() throws IOException { - - HLog log = new HLog(fs, dir, this.conf, null); - TransactionalHLogManager logMangaer = new TransactionalHLogManager(log, fs, - regionInfo, conf); - - long transactionId = 1; - logMangaer.writeStartToLog(transactionId); - - BatchUpdate update1 = new BatchUpdate(row1); - update1.put(col, val1); - logMangaer.writeUpdateToLog(transactionId, update1); - - BatchUpdate update2 = new BatchUpdate(row2); - update2.put(col, val2); - logMangaer.writeUpdateToLog(transactionId, update2); - - BatchUpdate update3 = new BatchUpdate(row3); - update3.put(col, val3); - logMangaer.writeUpdateToLog(transactionId, update3); - - logMangaer.writeAbortToLog(transactionId); - - // log.completeCacheFlush(regionName, tableName, logSeqId); - - log.close(); - Path filename = log.computeFilename(log.getFilenum()); - - Map> commits = logMangaer.getCommitsFromLog( - filename, -1, null); - - assertEquals(0, commits.size()); - } - - /** - * @throws IOException - */ - public void testInterlievedCommits() throws IOException { - - HLog log = new HLog(fs, dir, this.conf, null); - TransactionalHLogManager logMangaer = new TransactionalHLogManager(log, fs, - regionInfo, conf); - - long transaction1Id = 1; - long transaction2Id = 2; - logMangaer.writeStartToLog(transaction1Id); - - BatchUpdate update1 = new BatchUpdate(row1); - update1.put(col, val1); - logMangaer.writeUpdateToLog(transaction1Id, update1); - - logMangaer.writeStartToLog(transaction2Id); - - BatchUpdate update2 = new BatchUpdate(row2); - update2.put(col, val2); - logMangaer.writeUpdateToLog(transaction2Id, update2); - - BatchUpdate update3 = new BatchUpdate(row3); - update3.put(col, val3); - logMangaer.writeUpdateToLog(transaction1Id, update3); - - logMangaer.writeCommitToLog(transaction2Id); - logMangaer.writeCommitToLog(transaction1Id); - - // log.completeCacheFlush(regionName, tableName, logSeqId); - - log.close(); - Path filename = log.computeFilename(log.getFilenum()); - - Map> commits = logMangaer.getCommitsFromLog( - filename, -1, null); - - assertEquals(2, commits.size()); - assertEquals(2, commits.get(transaction1Id).size()); - assertEquals(1, commits.get(transaction2Id).size()); - } - - /** - * @throws IOException - */ - public void testInterlievedAbortCommit() throws IOException { - - HLog log = new HLog(fs, dir, this.conf, null); - TransactionalHLogManager logMangaer = new TransactionalHLogManager(log, fs, - regionInfo, conf); - - long transaction1Id = 1; - long transaction2Id = 2; - logMangaer.writeStartToLog(transaction1Id); - - BatchUpdate update1 = new BatchUpdate(row1); - update1.put(col, val1); - logMangaer.writeUpdateToLog(transaction1Id, update1); - - logMangaer.writeStartToLog(transaction2Id); - - BatchUpdate update2 = new BatchUpdate(row2); - update2.put(col, val2); - logMangaer.writeUpdateToLog(transaction2Id, update2); - - logMangaer.writeAbortToLog(transaction2Id); - - BatchUpdate update3 = new BatchUpdate(row3); - update3.put(col, val3); - logMangaer.writeUpdateToLog(transaction1Id, update3); - - logMangaer.writeCommitToLog(transaction1Id); - - // log.completeCacheFlush(regionName, tableName, logSeqId); - - log.close(); - Path filename = log.computeFilename(log.getFilenum()); - - Map> commits = logMangaer.getCommitsFromLog( - filename, -1, null); - - assertEquals(1, commits.size()); - assertEquals(2, commits.get(transaction1Id).size()); - } - - /** - * @throws IOException - */ - public void testInterlievedCommitAbort() throws IOException { - - HLog log = new HLog(fs, dir, this.conf, null); - TransactionalHLogManager logMangaer = new TransactionalHLogManager(log, fs, - regionInfo, conf); - - long transaction1Id = 1; - long transaction2Id = 2; - logMangaer.writeStartToLog(transaction1Id); - - BatchUpdate update1 = new BatchUpdate(row1); - update1.put(col, val1); - logMangaer.writeUpdateToLog(transaction1Id, update1); - - logMangaer.writeStartToLog(transaction2Id); - - BatchUpdate update2 = new BatchUpdate(row2); - update2.put(col, val2); - logMangaer.writeUpdateToLog(transaction2Id, update2); - - logMangaer.writeCommitToLog(transaction2Id); - - BatchUpdate update3 = new BatchUpdate(row3); - update3.put(col, val3); - logMangaer.writeUpdateToLog(transaction1Id, update3); - - logMangaer.writeAbortToLog(transaction1Id); - - // log.completeCacheFlush(regionName, tableName, logSeqId); - - log.close(); - Path filename = log.computeFilename(log.getFilenum()); - - Map> commits = logMangaer.getCommitsFromLog( - filename, -1, null); - - assertEquals(1, commits.size()); - assertEquals(1, commits.get(transaction2Id).size()); - } - - // FIXME Cannot do this test without a global transacton manager - // public void testMissingCommit() { - // fail(); - // } - - // FIXME Cannot do this test without a global transacton manager - // public void testMissingAbort() { - // fail(); - // } - -} diff --git a/src/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java b/src/test/org/apache/hadoop/hbase/thrift/DisabledTestThriftServer.java similarity index 92% rename from src/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java rename to src/test/org/apache/hadoop/hbase/thrift/DisabledTestThriftServer.java index 7c661afe85d..dc3866913b1 100644 --- a/src/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java +++ b/src/test/org/apache/hadoop/hbase/thrift/DisabledTestThriftServer.java @@ -35,14 +35,14 @@ import org.apache.hadoop.hbase.util.Bytes; * Unit testing for ThriftServer.HBaseHandler, a part of the * org.apache.hadoop.hbase.thrift package. */ -public class TestThriftServer extends HBaseClusterTestCase { +public class DisabledTestThriftServer extends HBaseClusterTestCase { // Static names for tables, columns, rows, and values private static byte[] tableAname = Bytes.toBytes("tableA"); private static byte[] tableBname = Bytes.toBytes("tableB"); private static byte[] columnAname = Bytes.toBytes("columnA:"); private static byte[] columnBname = Bytes.toBytes("columnB:"); - private static byte[] badColumnName = Bytes.toBytes("forgotColon"); + private static byte[] badColumnName = Bytes.toBytes("noColon:"); private static byte[] rowAname = Bytes.toBytes("rowA"); private static byte[] rowBname = Bytes.toBytes("rowB"); private static byte[] valueAname = Bytes.toBytes("valueA"); @@ -94,20 +94,6 @@ public class TestThriftServer extends HBaseClusterTestCase { 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); } /** @@ -123,19 +109,31 @@ public class TestThriftServer extends HBaseClusterTestCase { handler.createTable(tableAname, getColumnDescriptors()); // Apply a few Mutations to rowA + // mutations.add(new Mutation(false, columnAname, valueAname)); + // mutations.add(new Mutation(false, columnBname, valueBname)); handler.mutateRow(tableAname, rowAname, getMutations()); // Assert that the changes were made - assertTrue(Bytes.equals(valueAname, handler.get(tableAname, rowAname, columnAname).get(0).value)); + assertTrue(Bytes.equals(valueAname, + handler.get(tableAname, rowAname, columnAname).get(0).value)); TRowResult rowResult1 = handler.getRow(tableAname, rowAname).get(0); assertTrue(Bytes.equals(rowAname, rowResult1.row)); - assertTrue(Bytes.equals(valueBname, rowResult1.columns.get(columnBname).value)); + assertTrue(Bytes.equals(valueBname, + rowResult1.columns.get(columnBname).value)); // Apply a few BatchMutations for rowA and rowB + // rowAmutations.add(new Mutation(true, columnAname, null)); + // rowAmutations.add(new Mutation(false, columnBname, valueCname)); + // batchMutations.add(new BatchMutation(rowAname, rowAmutations)); + // Mutations to rowB + // rowBmutations.add(new Mutation(false, columnAname, valueCname)); + // rowBmutations.add(new Mutation(false, columnBname, valueDname)); + // batchMutations.add(new BatchMutation(rowBname, rowBmutations)); handler.mutateRows(tableAname, getBatchMutations()); // Assert that changes were made to rowA - assertFalse(handler.get(tableAname, rowAname, columnAname).size() > 0); + List cells = handler.get(tableAname, rowAname, columnAname); + assertFalse(cells.size() > 0); assertTrue(Bytes.equals(valueCname, handler.get(tableAname, rowAname, columnBname).get(0).value)); List versions = handler.getVer(tableAname, rowAname, columnBname, MAXVERSIONS); assertTrue(Bytes.equals(valueCname, versions.get(0).value)); @@ -352,9 +350,11 @@ public class TestThriftServer extends HBaseClusterTestCase { */ private List getBatchMutations() { List batchMutations = new ArrayList(); - // Mutations to rowA + // Mutations to rowA. You can't mix delete and put anymore. List rowAmutations = new ArrayList(); rowAmutations.add(new Mutation(true, columnAname, null)); + batchMutations.add(new BatchMutation(rowAname, rowAmutations)); + rowAmutations = new ArrayList(); rowAmutations.add(new Mutation(false, columnBname, valueCname)); batchMutations.add(new BatchMutation(rowAname, rowAmutations)); // Mutations to rowB diff --git a/src/test/org/apache/hadoop/hbase/util/TestMergeTool.java b/src/test/org/apache/hadoop/hbase/util/DisabledTestMergeTool.java similarity index 78% rename from src/test/org/apache/hadoop/hbase/util/TestMergeTool.java rename to src/test/org/apache/hadoop/hbase/util/DisabledTestMergeTool.java index ffe0ca1f36e..7ae9537f3fe 100644 --- a/src/test/org/apache/hadoop/hbase/util/TestMergeTool.java +++ b/src/test/org/apache/hadoop/hbase/util/DisabledTestMergeTool.java @@ -21,6 +21,8 @@ package org.apache.hadoop.hbase.util; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -31,17 +33,24 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.regionserver.HLog; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.util.ToolRunner; /** Test stand alone merge tool that can merge arbitrary regions */ -public class TestMergeTool extends HBaseTestCase { - static final Log LOG = LogFactory.getLog(TestMergeTool.class); - static final byte [] COLUMN_NAME = Bytes.toBytes("contents:"); +public class DisabledTestMergeTool extends HBaseTestCase { + static final Log LOG = LogFactory.getLog(DisabledTestMergeTool.class); +// static final byte [] COLUMN_NAME = Bytes.toBytes("contents:"); + static final byte [] FAMILY = Bytes.toBytes("contents"); + static final byte [] QUALIFIER = Bytes.toBytes("dc"); + private final HRegionInfo[] sourceRegions = new HRegionInfo[5]; private final HRegion[] regions = new HRegion[5]; private HTableDescriptor desc; @@ -54,7 +63,7 @@ public class TestMergeTool extends HBaseTestCase { // Create table description this.desc = new HTableDescriptor("TestMergeTool"); - this.desc.addFamily(new HColumnDescriptor(COLUMN_NAME)); + this.desc.addFamily(new HColumnDescriptor(FAMILY)); /* * Create the HRegionInfos for the regions. @@ -66,32 +75,40 @@ public class TestMergeTool extends HBaseTestCase { // Region 1 will contain the key range [row_0250,row_0400) and overlaps // with Region 0 sourceRegions[1] = - new HRegionInfo(this.desc, Bytes.toBytes("row_0250"), Bytes.toBytes("row_0400")); + new HRegionInfo(this.desc, Bytes.toBytes("row_0250"), + Bytes.toBytes("row_0400")); // Region 2 will contain the key range [row_0100,row_0200) and is adjacent // to Region 0 or the region resulting from the merge of Regions 0 and 1 sourceRegions[2] = - new HRegionInfo(this.desc, Bytes.toBytes("row_0100"), Bytes.toBytes("row_0200")); + new HRegionInfo(this.desc, Bytes.toBytes("row_0100"), + Bytes.toBytes("row_0200")); // Region 3 will contain the key range [row_0500,row_0600) and is not // adjacent to any of Regions 0, 1, 2 or the merged result of any or all // of those regions sourceRegions[3] = - new HRegionInfo(this.desc, Bytes.toBytes("row_0500"), Bytes.toBytes("row_0600")); + new HRegionInfo(this.desc, Bytes.toBytes("row_0500"), + Bytes.toBytes("row_0600")); // Region 4 will have empty start and end keys and overlaps all regions. sourceRegions[4] = - new HRegionInfo(this.desc, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY); + new HRegionInfo(this.desc, HConstants.EMPTY_BYTE_ARRAY, + HConstants.EMPTY_BYTE_ARRAY); /* * Now create some row keys */ this.rows = new byte [5][][]; this.rows[0] = Bytes.toByteArrays(new String[] { "row_0210", "row_0280" }); - this.rows[1] = Bytes.toByteArrays(new String[] { "row_0260", "row_0350", "row_035" }); - this.rows[2] = Bytes.toByteArrays(new String[] { "row_0110", "row_0175", "row_0175", "row_0175"}); - this.rows[3] = Bytes.toByteArrays(new String[] { "row_0525", "row_0560", "row_0560", "row_0560", "row_0560"}); - this.rows[4] = Bytes.toByteArrays(new String[] { "row_0050", "row_1000", "row_1000", "row_1000", "row_1000", "row_1000" }); + this.rows[1] = Bytes.toByteArrays(new String[] { "row_0260", "row_0350", + "row_035" }); + this.rows[2] = Bytes.toByteArrays(new String[] { "row_0110", "row_0175", + "row_0175", "row_0175"}); + this.rows[3] = Bytes.toByteArrays(new String[] { "row_0525", "row_0560", + "row_0560", "row_0560", "row_0560"}); + this.rows[4] = Bytes.toByteArrays(new String[] { "row_0050", "row_1000", + "row_1000", "row_1000", "row_1000", "row_1000" }); // Start up dfs this.dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null); @@ -120,9 +137,9 @@ public class TestMergeTool extends HBaseTestCase { */ for (int j = 0; j < rows[i].length; j++) { byte [] row = rows[i][j]; - BatchUpdate b = new BatchUpdate(row); - b.put(COLUMN_NAME, new ImmutableBytesWritable(row).get()); - regions[i].batchUpdate(b, null); + Put put = new Put(row); + put.add(FAMILY, QUALIFIER, row); + regions[i].put(put); } HRegion.addRegionToMETA(meta, regions[i]); } @@ -173,9 +190,28 @@ public class TestMergeTool extends HBaseTestCase { private void verifyMerge(final HRegion merged, final int upperbound) throws IOException { + //Test + Scan scan = new Scan(); + scan.addFamily(FAMILY); + InternalScanner scanner = merged.getScanner(scan); + List testRes = null; + while(true) { + testRes = new ArrayList(); + boolean hasNext = scanner.next(testRes); + if(!hasNext) { + break; + } + } + + //!Test + for (int i = 0; i < upperbound; i++) { for (int j = 0; j < rows[i].length; j++) { - byte [] bytes = Cell.createSingleCellArray(merged.get(rows[i][j], COLUMN_NAME, -1, -1))[0].getValue(); + Get get = new Get(rows[i][j]); + get.addFamily(FAMILY); + Result result = merged.get(get, null); + assertEquals(1, result.size()); + byte [] bytes = result.sorted()[0].getValue(); assertNotNull(rows[i][j].toString(), bytes); assertTrue(Bytes.equals(bytes, rows[i][j])); } @@ -191,7 +227,10 @@ public class TestMergeTool extends HBaseTestCase { // contain the right data. for (int i = 0; i < regions.length; i++) { for (int j = 0; j < rows[i].length; j++) { - byte[] bytes = Cell.createSingleCellArray(regions[i].get(rows[i][j], COLUMN_NAME, -1, -1))[0].getValue(); + Get get = new Get(rows[i][j]); + get.addFamily(FAMILY); + Result result = regions[i].get(get, null); + byte [] bytes = result.sorted()[0].getValue(); assertNotNull(bytes); assertTrue(Bytes.equals(bytes, rows[i][j])); } @@ -229,4 +268,4 @@ public class TestMergeTool extends HBaseTestCase { log.closeAndDelete(); } } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/util/MigrationTest.java b/src/test/org/apache/hadoop/hbase/util/MigrationTest.java index 34f42077376..e1be537b96c 100644 --- a/src/test/org/apache/hadoop/hbase/util/MigrationTest.java +++ b/src/test/org/apache/hadoop/hbase/util/MigrationTest.java @@ -35,12 +35,15 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.ResultScanner; /** * Runs migration of filesystem from hbase 0.x to 0.x @@ -128,9 +131,11 @@ public class MigrationTest extends HBaseTestCase { HTable t = new HTable(this.conf, TABLENAME); int count = 0; LOG.info("OPENING SCANNER"); - Scanner s = t.getScanner(TABLENAME_COLUMNS); + Scan scan = new Scan(); + scan.addColumns(TABLENAME_COLUMNS); + ResultScanner s = t.getScanner(scan); try { - for (RowResult r: s) { + for (Result r: s) { if (r == null || r.size() == 0) { break; } @@ -168,7 +173,15 @@ public class MigrationTest extends HBaseTestCase { long startcode = -1; boolean changed = false; for (int i = 0; i < retries; i++) { - startcode = Writables.cellToLong(m.get(row, HConstants.COL_STARTCODE)); + Get get = new Get(row); + get.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER); + Result res = m.get(get); + KeyValue [] kvs = res.raw(); + if(kvs.length <= 0){ + return; + } + byte [] value = kvs[0].getValue(); + startcode = Bytes.toLong(value); if (startcode != oldStartCode) { changed = true; break; diff --git a/src/test/org/apache/hadoop/hbase/util/SoftValueSortedMapTest.java b/src/test/org/apache/hadoop/hbase/util/SoftValueSortedMapTest.java index 5cc1792a3d1..132f1cc7c6d 100644 --- a/src/test/org/apache/hadoop/hbase/util/SoftValueSortedMapTest.java +++ b/src/test/org/apache/hadoop/hbase/util/SoftValueSortedMapTest.java @@ -17,7 +17,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import java.util.SortedMap; @@ -30,6 +29,7 @@ public class SoftValueSortedMapTest { map.put(new Integer(i), new Integer(i)); } System.out.println(map.size()); + @SuppressWarnings("unused") byte[] block = new byte[849*1024*1024]; // 10 MB System.out.println(map.size()); } diff --git a/src/test/org/apache/hadoop/hbase/util/TestRootPath.java b/src/test/org/apache/hadoop/hbase/util/TestRootPath.java index 3a684b9db36..8a80038570f 100644 --- a/src/test/org/apache/hadoop/hbase/util/TestRootPath.java +++ b/src/test/org/apache/hadoop/hbase/util/TestRootPath.java @@ -60,4 +60,4 @@ public class TestRootPath extends TestCase { LOG.info("Got expected exception when checking invalid path:", e); } } -} +} \ No newline at end of file diff --git a/src/webapps/master/master.jsp b/src/webapps/master/master.jsp index 30d8a3138cd..f4940cf2f53 100644 --- a/src/webapps/master/master.jsp +++ b/src/webapps/master/master.jsp @@ -12,7 +12,6 @@ import="org.apache.hadoop.hbase.HServerAddress" import="org.apache.hadoop.hbase.HBaseConfiguration" import="org.apache.hadoop.hbase.HColumnDescriptor" - import="org.apache.hadoop.hbase.client.tableindexed.IndexSpecification" import="org.apache.hadoop.hbase.HTableDescriptor" %><% HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER); HBaseConfiguration conf = master.getConfiguration(); @@ -124,19 +123,6 @@ $(document).ready(function(){

              • -
              •  Indexes -
                  -<% Collection idx = htDesc.getIndexes(); - if (idx.size() > 0) - for (IndexSpecification is: idx) { %> -
                •  ID: <%= is.getIndexId() %>
                • -<% } - else { %> -
                •  none
                • -<% } %> -
                -
              • -
            • diff --git a/src/webapps/master/table.jsp b/src/webapps/master/table.jsp index 45cb3b89d2a..ccc37c33286 100644 --- a/src/webapps/master/table.jsp +++ b/src/webapps/master/table.jsp @@ -50,18 +50,18 @@ if (key != null && key.length() > 0) { Writable[] arr = new Writable[1]; arr[0] = new ImmutableBytesWritable(Bytes.toBytes(key)); - master.modifyTable(Bytes.toBytes(tableName), HConstants.MODIFY_TABLE_SPLIT, arr); + master.modifyTable(Bytes.toBytes(tableName), HConstants.Modify.TABLE_SPLIT, arr); } else { - master.modifyTable(Bytes.toBytes(tableName), HConstants.MODIFY_TABLE_SPLIT, null); + master.modifyTable(Bytes.toBytes(tableName), HConstants.Modify.TABLE_SPLIT, null); } %> Split request accepted. <% } else if (action.equals("compact")) { if (key != null && key.length() > 0) { Writable[] arr = new Writable[1]; arr[0] = new ImmutableBytesWritable(Bytes.toBytes(key)); - master.modifyTable(Bytes.toBytes(tableName), HConstants.MODIFY_TABLE_COMPACT, arr); + master.modifyTable(Bytes.toBytes(tableName), HConstants.Modify.TABLE_COMPACT, arr); } else { - master.modifyTable(Bytes.toBytes(tableName), HConstants.MODIFY_TABLE_COMPACT, null); + master.modifyTable(Bytes.toBytes(tableName), HConstants.Modify.TABLE_COMPACT, null); } %> Compact request accepted. <% }