HBASE-681 NPE in Memcache
HAbstractScanner
- remove HAbstactScanner.iterator() - iterator is not a method on InternalScanner
HRegion
- make getScanner more efficient by iterating only once to find the stores we need to scan
- only pass columns relevant to a store to a HStoreScanner
- remove HScanner.iterator() - iterator is not a method on InternalScanner
Memcache, MemcacheScanner
- Fix NPE in Memcache
- never return HConstants.LATEST_TIMESTAMP as the timestamp value for a row. Instead use the largest timestamp from the cells being returned. This allows a scanner to determine a timestamp that can be used to fetch the same data again should new versions be inserted later.
StoreFileScanner
- getNextViableRow would find a row that matched the row key, but did not consider the requested timestamp. Now if the row it finds has a timestamp greater than the one desired it advances to determine if a row with a timestamp less than or equal to the requested one exists since timestamps are sorted descending.
- removed an unnecessary else
testScanMultipleVersions
- Test program that fails on current trunk but passes when this patch is applied.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@670124 13f79535-47bb-0310-9956-ffa450edef68
-Change HServerLoad's getLoad method to ignore the number of requests, thus causing RegionManager to assign based merely on number of regions per server
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@669533 13f79535-47bb-0310-9956-ffa450edef68
Add in special handling of .META. table that we used have in HQL
(so it prints out the HRegionInfo pretty). Also allow making a scanner
without specifying columns.
M src/java/org/apache/hadoop/hbase/HTableDescriptor.java
Allow getMetadata work if HTable is set against meta tables.
Was failing on isLegalTableName if name was one of the catalog table names.
Needed by shell.
M src/java/org/apache/hadoop/hbase/client/HTable.java
Comment.
M bin/hbase
Remove commented out line.
M bin/HBase.rb
Allow passing just a table name to scanner; let it figure out all families
Added in the special handling of .META. table cells that we used have in HQL
so we can see start/end row, etc. Added in extra testing.
M bin/Formatter.rb
Allow setting width of emitted table in console formatter
M bin/hirb.rb
Allow setting width of emitted table in console formatter
Improved scanner help.:
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@669318 13f79535-47bb-0310-9956-ffa450edef68
-Added new test to TestGet2 to highlight multi-storefile getClosestBefore issue
-Removed erroneous return that caused 2nd and subsequent mapfiles to be skipped
-Split HStore#rowKeyAtOrBeforeFromMapfile into two sub-methods for readability
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@669211 13f79535-47bb-0310-9956-ffa450edef68
HBASE-650 Add String versions of get, scanner, put in HTable
HBASE-656 Do not retry exceptions such as unknown scanner or illegal argument
A src/java/org/apache/hadoop/hbase/ColumnNameParseException.java
A src/java/org/apache/hadoop/hbase/LeaseException.java
Added. Thrown instead of IllegalArgumentExceptions
M src/java/org/apache/hadoop/hbase/Leases.java
Use new LeaseException in place of IllegalArgument
M src/java/org/apache/hadoop/hbase/HStoreKey.java
Use new ColumnNameParse in place of IllegalArgument
M src/java/org/apache/hadoop/hbase/master/ServerManager.java
Log at debug if LeaseException (Not important if it happens).
A src/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
An IOE that shouldn't be retried.
M src/java/org/apache/hadoop/hbase/InvalidColumnNameException.java
M src/java/org/apache/hadoop/hbase/UnknownScannerException.java
Inherit from DoNotRetryIOException else we keep trying.
M src/java/org/apache/hadoop/hbase/util/Bytes.java
(toByteArrays): Added one to handle [] String.
M src/java/org/apache/hadoop/hbase/client/HTable.java
Make String overrides of all methods. Made data members
private (turns out a bunch arent' even used). Stopped it
inheriting from HConstants so we don't have big dump of
all HConstants as first thing in javadoc.
M src/java/org/apache/hadoop/hbase/client/HConnectionManager.java
If instance of DoNotRetryIOException, let the exception out.
M src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
Make String overrides of all methods. Stopped it
inheriting from HConstants so we don't have big dump of
all HConstants as first thing in javadoc.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@661541 13f79535-47bb-0310-9956-ffa450edef68
over on the remote side.
M src/test/org/apache/hadoop/hbase/TestSerialization.java
Add test that HMsg with region and message serializes.
M src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
HMsg types have been enumified. Also use some of the new
static messages. On split and close because of error,
add message to the HMsg we pass back to the master.
M src/java/org/apache/hadoop/hbase/HServerInfo.java
(compareTo): Added.
M src/java/org/apache/hadoop/hbase/HRegionInfo.java
Allow null in compareTo.
M src/java/org/apache/hadoop/hbase/master/ServerManager.java
Use the new HMsg.isType figuring message type.
Redo message logging. Use convenience HMsg statics.
M src/java/org/apache/hadoop/hbase/master/RegionManager.java
Pass back overloaded message if region shutdown because of balancing.
M src/java/org/apache/hadoop/hbase/HServerAddress.java
Make it so that two addresses equate even if one has hostname
and the other IP.
M src/java/org/apache/hadoop/hbase/ipc/HMasterRegionInterface.java
Up the protocol version for regionserver reporting master messages.
M src/java/org/apache/hadoop/hbase/HMsg.java
Enumify the messge types.
Define a few static HMsgs for convenience.
Allow optional message. Improved toString.
(isType, equals, hashCode): Added.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@658465 13f79535-47bb-0310-9956-ffa450edef68
M HMaster
- Moved HMaster.quiescedMetaServers to ServerManager.quiescedServers and changed name since only servers serving user regions get quiesced.
- Removed HMaster.tableInCreation - not used
M ServerManager
- Don't check if quiescedServers.get() >= serversToServerInfo.size() unless master.shutdownRequested is true.
M HRegionServer
- Change order of checks in main loop of HRegionServer.run, so that booleans are checked before we check the number of messages to process
- Don't break out of main loop if restart or stop requested - stop is checked at top of loop and we need to continue rather than break out of main loop if restart was requested
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@655962 13f79535-47bb-0310-9956-ffa450edef68
M MultiRegionTable
Make deterministic by creating the regions directly and not rely on
the asychronous nature of cache flushes, compactions and splits. The
regions are small, but the point of this class is to generate a
table with multiple regions so we can test map / reduce, region
onlining / offlining, etc.
Removed PUNCTUATION from row keys. Not sure why it was there in the
first place, other than perhaps to verify that a row key can have
punctuation in it provided it is not the first character. This will
become moot when row keys change from Text to byte[] anyways.
Incorporate repeated code
{code}
region.close();
region.getLog().closeAndDelete();
{code}
into private method closeRegionAndDeleteLog
M TestSplit
extends HBaseClusterTestCase instead of MultiRegionTable. It didn't
use the output of MultiRegionTable, so all that work was just wasted
by this test.
M TestTableIndex, TestTableMapReduce
The only two tests that currently use MultiRegionTable. Minor
modifications needed because MultiRegionTable now handles starting
and stopping of the mini-DFS cluster. With the new MultiRegionTable
class, if these tests fail now it will be because something they are
testing has regressed and not because MultiRegionTable failed.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@652587 13f79535-47bb-0310-9956-ffa450edef68
knows about workings of memcache
HStore knows about workings of memcache
This patch moves the running of mecache snapshots out of the control
of HRegion and hides the memcache details in HStore. This patch also
does a bunch of work on the merge tool fixing a bug in the metautils along
the way. The merge tool test was failing. We weren't setting into the
HLog the maximum sequence id after we'd opened a region -- as HRS does --
so were losing edits. On the way, refactored the merge tool test to get
rid of duplicated code. Finally, cleans up logging in HStore to aid
debugging; e.g. we always refer to the sequence id as the 'sequence id'
in log messages rather than as seqId or 'sequence record' so can sort
log as see state of sequence id transitions.
Version 2 changes the order in which things are run in memcache. 532
made it so flushing did snapshot and then cleared the snapshot. Now,
we snapshot before we flush a store, then inside in the store flush,
we call getSnapshot and then clearSnapshot.
M src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java
How snapshotting changed. Change test in accordance.
M src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java
Use accessor to get file number.
M src/test/org/apache/hadoop/hbase/util/TestMergeTool.java
Refactored to remove duplicated code so could tell what was going on.
(mergeAndVerify, verifyMerge): Addd.
M src/java/org/apache/hadoop/hbase/regionserver/Memcache.java
(snapshot): Changed so it no longer returns snapshot.
M src/java/org/apache/hadoop/hbase/regionserver/HStore.java
Changed log messages removing the useless and adding info to others.
(this.maxSeqId): We used to add 1 to this in here in HStore. Let
HRegion do it. Its the one that does the machinations w/ sequenceids
anyways. Make flushes return the amount flushed. Use this updating
the regions memcacheSize accounting.
(snapshot): Added method for the region to call.
M src/java/org/apache/hadoop/hbase/regionserver/HLog.java
Made data members private.
(getFileNum): Added accessor for tests.
M src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
Added info to logs..
(snapshotMemcaches): removed.
(internalFlushcache): No longer takes startime. Internally now does
some of what used happen in snapshotMemcaches including sending of
message to stores to snapshot.
(getEntrySize): Added method for calculating size of an update. Used
by HRegion and flushing so both come up w/ same answer.
M src/java/org/apache/hadoop/hbase/util/Merge.java
Add logging of whats happening during merges and fail earlier than we
used if stuff is not right.
Renamed local variables from region1 to r1, etc., so didn't clash
with data members of same name.
M src/java/org/apache/hadoop/hbase/util/MetaUtils.java
Added a TODO
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@650298 13f79535-47bb-0310-9956-ffa450edef68
-Removed HScannerInterface and HInternalScannerInterface
-Created new interfaces Scanner for clients and InternalScanner for internal consumers
-Internal and client scanners no longer share common interface
-Client scanner's next() method and iterables are in RowResults
-Updated tests and internal consumers to use Scanner in place of HScannerInterface
-HTable obtainScanner(*) are now renamed getScanner(*)
-Tests have ScannerIncommon to turn Scanners into InternalScanners for some tests
-Fixed a bug in HMaster that was eating TableExistsExceptions (unrelated)
-Updated TableInputFormat to provide RowResults instead of MapWritables
-Updated TableOutputFormat to take BatchUpdates instead of MapWritables
-Updated TableMap, TableReduce, and friends to correctly hook up to new input/output formats
HBASE-567 Reused BatchUpdate instances accumulate BatchOperations
- Fix to BatchUpdate that allows correct reuse of BatchUpdate instances (readFields didn't clear BatchOperation map)
- Update TestSerialization to prove above is fixed
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@646104 13f79535-47bb-0310-9956-ffa450edef68
-HConnectionManager#locateRegionInMeta no longer throws ISE, instead throws new RegionOfflineException
-Removed duplicated code for catching exceptions for retries
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@644854 13f79535-47bb-0310-9956-ffa450edef68
-Updated HRegionInterface, HRegionServer, HRegion, HStore to provide RowResults as the return of getRow methods
-Updated HTable to expect RowResult objects
-Updated ThriftServer to expect RowResults
-Cleaned up HConnectionManager's interaction with region servers
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@644828 13f79535-47bb-0310-9956-ffa450edef68
Removes startUpdate calls from all but a few places. TestBatchUpdate and TestMultipleUpdates both stay the same, but TMU will be removed when startUpdate is. Parts of TBU will also be whacked when we remove the deprecated methods. HTable still has its startUpdate methods.
Changed the Incommon interface to remove the startUpdate, put, delete, and commit methods, and made a new commit(BatchUpdate).
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@644811 13f79535-47bb-0310-9956-ffa450edef68
HMerge, HRegionServer
- changes that reflect changes to HRegion, CompactSplitThread and Flusher methods
ServerManager
- Return zero length array to region server if it is exiting or quiesced and Master is not yet ready to shut down.
QueueEntry
- removed. no longer used.
CompactSplitThread
- make compactionQueue a queue of HRegion.
- Add Set<HRegion> so we can quickly determine if a region is in the queue. BlockingQueue.contains() does a linear scan of the queue.
- Add a lock and interruptPolitely methods so that compactions/splits in progress are not interrupted.
- Don't add a region to the queue if it is already present.
Flusher
- change queue from DelayQueue to BlockingQueue, with HRegion entries instead of QueueEntry.
- Add Set<HRegion> to quickly determine if a region is already in the queue to avoid linear scan of BlockingQueue.contains().
- Only put regions in the queue for optional cache flush if the last time they were flushed is older than now - optionalFlushInterval.
- Only add regions to the queue if it is not already present.
HRegion
- don't request a cache flush if one has already been requested.
- Add setLastFlushTime so flusher can set it once it has queued an optional flush.
- Replace largestHStore with getLargestHStoreSize: returns long instead of HStoreSize object.
- Add midKey as parameter to splitRegion.
- Reorder start of splitRegion so it doesn't do any work before validating parameters.
- Remove needsSplit and compactIfNeeded - no longer needed.
- compactStores now returns midKey if split is needed.
- snapshotMemcaches now sets flushRequested to false and sets lastFlushTime to now.
- update does not request a cache flush if one has already been requested.
- Override equals and hashCode so HRegions can be stored in a HashSet.
HStore
- loadHStoreFiles now computes max sequence id and the initial size of the store.
- Add getter for family.
- internalCacheFlush updates store size, and logs both size of cache flush and resulting map file size (with debug logging enabled).
- Remove needsCompaction and hasReferences - no longer needed.
- compact() returns midKey if store needs to be split.
- compact() does all checking before actually starting a compaction.
- If store size is greater than desiredMaxFileSize, compact returns the midKey for the store regardless of whether a compaction was actually done.
- Added more synchronization in completeCompaction while iterating over storeFiles.
- completeCompaction computes new store size.
- New method checkSplit replaces method size. Returns midKey if store needs to be split and can be split.
HStoreSize
- removed. No longer needed.
HBaseTestCase
- only set fs if it has not already been set by a subclass.
TestTableIndex, TestTableMapReduce
- call FSUtil.deleteFully to clean up cruft left in local fs, by MapReduce
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@643761 13f79535-47bb-0310-9956-ffa450edef68
M branches/0.1/src/java/org/apache/hadoop/hbase/util/MetaUtils.java
M trunk/src/java/org/apache/hadoop/hbase/util/MetaUtils.java
(changeOnlineStatus): Added.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@643486 13f79535-47bb-0310-9956-ffa450edef68
server reports that it is processing the open request
This is patch reviewed with Jim but with the number of edits between
reports made into a configurable.
Have the HRegionServer pass down a Progressable implementation down into
Region and then down int Store where edits are replayed. Call progress
after every couple of thousand edits.
M src/java/org/apache/hadoop/hbase/HStore.java
Take a Progessable in the constructor. Call it when applying edits.
M src/java/org/apache/hadoop/hbase/HMaster.java
Update commment around MSG_REPORT_PROCESS_OPEN so its expected
that we can get more than one of these messages during a region open.
M src/java/org/apache/hadoop/hbase/HRegion.java
New constructor that takes a Progressable. Pass it to Stores on construction.
M src/java/org/apache/hadoop/hbase/HRegionServer.java
On open of a region, pass in a Progressable that adds a
MSG_REPORT_PROCESS_OPEN every time its called.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@643223 13f79535-47bb-0310-9956-ffa450edef68
on eachiteration, edits are aggregated up into the millions
M src/java/org/apache/hadoop/hbase/HLog.java
(splitLog): If an exception processing a split, catch it.
In finally, close and delete the split. Don't try retrying.
While in some circumstance, we might recover, its also
likely that we just get same exception again. If so, and
multiple files, we'll just accumulate edits until the
kingdom comes.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@643142 13f79535-47bb-0310-9956-ffa450edef68
M src/java/org/apache/hadoop/hbase/HStore.java
(Constructor) If an exception out of reconstructionLog method, log it and
keep going. Presumption is that its result of a lack of HADOOP--1700.
(reconstructionLog): Check for empty log file.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@643110 13f79535-47bb-0310-9956-ffa450edef68
-Changed MiniHBaseCluster to not start up a MiniDFS
-Changed HBaseClusterTestCase to do the work of starting up a MiniDFS.
-Added pre and post setup method to HBaseClusterTestCase so you can control what happen before MiniHBaseCluster is booted up
-Converted AbstractMergeTestCase to be a HBaseClusterTestCase
-Converted any test that used a raw MIniDFS or MiniHBaseCluster to use HBaseClusterTestCase instead
-Split TestTimestamp into two tests - one for clientside (now in o.a.h.h.client) and one for serverside (o.a.h.h.regionserver)
-Merged in Stack's changes to make bin/hbase have hadoop jars first on the classpath
-Updated PerformanceEvaluation (in --miniCluster mode) to start up a DFS first
-Fixed a bug in BaseScanner that would have allowed NPEs to be generated
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@640526 13f79535-47bb-0310-9956-ffa450edef68
-Changed HStore and Memcache methods for computing closest row at or before
-Added more test cases for verifying this functionality
-Simplified the getClosestRowBefore interface so that it does not take timestamps
-Noted that getClosestRowBefore is assumed to work correctly ONLY on tables where updates are always with ascending timestamps (method is still not a part of HTable interface, so not available to clients)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@638598 13f79535-47bb-0310-9956-ffa450edef68
M conf/hbase-default.xml
Add hbase.hbasemaster.maxregionopen property.
M src/java/org/apache/hadoop/hbase/HStore.java
Change way we log. Do way less. Just emit sums of edits applied
and skipped rather than individual edits.
M src/java/org/apache/hadoop/hbase/HRegionServer.java
Make sleeper instance a local rather than data member.
(reportForDuty): Take a sleeper instance.
(run): Removed redundant wrap of a 'for' by a 'while'.
(constructor): If IOE, do not offline the region. Seen to be
an overreaction.
M src/java/org/apache/hadoop/hbase/HLog.java
Don't output map of all files being cleaned everytime a new
entry is added; instead just log new entry. Remove emission
of every 10k edits.
M src/java/org/apache/hadoop/hbase/HMaster.java
Up default for maxregionopen. Was seeing that playing edits
could take a long time (mostly because we used log every
edit) but no harm in this being longer. On REPORT_CLOSE,
emit region info, not just region so can see the properties
(W/o, made it hard to figure who was responsible for offlining).
Add logging of attempt # in shutdown processing.
Add logging of state flags passed to the close region. Helps
debugging. Also in close offline ONLY if we are NOT reassigning
the region (jimk find).
M src/java/org/apache/hadoop/hbase/util/Sleeper.java
Add logging of extraordinary sleeps or calculated periods
(suspicion is that we're sleeping way longer on loaded machies
and the regionserver appears hung).
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@636849 13f79535-47bb-0310-9956-ffa450edef68
-Moved out classes Memcache, StoreFileScanner, MapFileCompactionReader, and HStoreScanner, and interface CompactionReader
-Updated TestHMemcache to use the correct type declaration.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@636815 13f79535-47bb-0310-9956-ffa450edef68
Summary of changes:
HMaster:
- When a row has an empty HRegionInfo (info:regioninfo), log it with the row name and and the other keys still in the row.
- Log the number of rows with empty HRegionInfo
- Delete the rows
- Make RowMap inner class static, change methods to have package scope to avoid synthetic accessors.
- Provide row name to getHRegionInfo so it can issue better log messages
- add method deleteEmptyMetaRows to remove rows with empty HRegionInfo
HRegion
- change removeRegionFromMETA to use deleteAll rather than using a BatchUpdate containing deletes for each cell.
TestEmptyMetaInfo
- new test case
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@636589 13f79535-47bb-0310-9956-ffa450edef68
HLog
- don't overwrite oldlogfile in splitLog if it already exists. Rename it and copy it into the new oldlogfile. Then delete it once it has been copied.
- use FileUtil.fullyDelete to delete region server log directory.
HRegion
- delete oldlogfile once it has been successfully processed
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@636396 13f79535-47bb-0310-9956-ffa450edef68
Other miscellaneous changes included:
IdentityTableReduce
- Added SuppressWarnings("unused") for reporter argument
- Removed unnecessary cast.
AbstractMergeTestBase
- Removed unnecessary compaction
StaticTestEnvironment
- Change logging level for client connections which are too noisy in most cases
TestBloomFilters
- Removed unnecessary config settings
- Modified to use BatchUpdate instead of deprecated startUpdate, etc.
TestScannerAPI
- Modified to use BatchUpdate instead of deprecated startUpdate, etc.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@630394 13f79535-47bb-0310-9956-ffa450edef68
(Recommit. We'd backed it out when it was HADOOP-1398)
This commit includes updating version from 0.1.0-dev to 0.2.0-dev
and an update of the hadoop lib to move it post 0.16.0 branch.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@619526 13f79535-47bb-0310-9956-ffa450edef68
HADOOP-2587 Splits blocked by compactions cause region to be offline for duration of compaction.
Fix bug in TestCompaction in which two mini dfs clusters were being started for the same test.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@611681 13f79535-47bb-0310-9956-ffa450edef68
HADOOP-2392, HADOOP-2324:
Chore
- initialChore() now returns boolean
HMaster
- rather than retry in root and meta scanners, return if a scan fails. It will get retried on the next scan. This has two effects: 1) scanners exit more quickly during shutdown and 2) they don't keep retrying to connect to a dead server, allowing them to recover from a server going down more quickly.
- initialScan in root and meta scanners return boolean and do not progress to maintenanceScan until the initial scan completes successfully.
HRegionServer
- speed up region server exit by reordering join's so that we join with threads in the order that we told them to stop
TestTableMapReduce
- remove overrides of heartbeat and thread wake intervals
HADOOP-2396:
HMaster
- move check for null HRegionInfo before first attempt to dereference it.
HADOOP-2397:
- HMaster$BaseScanner.checkAssigned: don't try to split dead server's log if initial startup has completed.
HADOOP-2353:
HMsg
- change toString() to only output the region name rather than calling HRegionInfo.toString()
StaticTestEnvironment
- make logging a bit less verbose
TestHLog
- was writing to local file system and failing on Windows
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@603428 13f79535-47bb-0310-9956-ffa450edef68
There are a lot of changes in this patch. The memcache has been changed from a per/region object to a per/column object, and HLocking has been removed since we do not have to maintain any locks across RPC calls.
This necessitated major changes to HRegion and HStore
Additionally there were many changes required to the unit tests since they tend to exploit some private interfaces that weren't designed to be public. Some of those interfaces changed so the test cases did as well.
This patch is the result of extensive analysis of the multiple threads in HBase that contend for shared resources: updates, reads, scanners, cache flushing, compaction and region splitting.
Many of the tests are timing sensitive, and since we tend to make "dormant" intervals as short as possible to speed up the Hudson build, we may go through several iterations of getting them right before Hudson is happy. This is especially true since two test cases failed on my dual cpu windows machine while running the tests under Ant, but ran fine under Eclipse.
However, now that the tests are passing locally, I believe the changes are doing the right thing, but may require some parameter tweaks.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@596835 13f79535-47bb-0310-9956-ffa450edef68
- Fix another race condition in processing dead servers,
- Fix error online meta regions: was using region name and not startKey as key for map.put.
- Change TestRegionServerExit to always kill the region server for the META region. This makes the test more deterministic and getting META reassigned was problematic.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@591880 13f79535-47bb-0310-9956-ffa450edef68
HLog.splitLog was generating incorrect file names, HRegion was generating file names that could be far too long especially for local file systems, HMaster had a race condition in which an old HLog would get split by two threads simultaneously.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@586680 13f79535-47bb-0310-9956-ffa450edef68
Set hbase.root in test/hbase-site.xml; when running a test, the default does not work consistantly.
When a HBase mini cluster is started on top of an existing mini dfs cluster, it should not shut down the mini dfs cluster when the mini HBase cluster is shut down.
TestDFSAbort catches exceptions, prints the stack trace and re-throws the exception, so you can see when the exception happened in the log.
Catch runtime exceptions that were escaping from FSUtils.isFileSystemAvailable, enabling more reliable detection of dfs failure. HRegionServer also now checks to see if it is still accepting client requests.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@580745 13f79535-47bb-0310-9956-ffa450edef68
of deletes. Added a deleteAll to remove all cells equal to or older than
passed timestamp. Fixed compaction so deleted cells do not make it out
into compacted output. Ensure also that versions > column max are dropped
compacting.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@574287 13f79535-47bb-0310-9956-ffa450edef68
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java
(addContents): Added overrides that allow specifying a timestamp.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java
Made it so test inherits from HBaseTestCase instead of from HBaseClusterTestCase
so could add in tests that do not use cluster.
(testTimestampScanning): Added test for hadoop-1834 bug.
(testTimestamp): Refactored to remove duplicated code.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
(getNext): Make it respect the timestamp set on construction.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
Removed eclipse yellow flag warnings around empty parens and
auto-boxing longs.
(getNext): Make it respect the timestamp set on construction.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@572980 13f79535-47bb-0310-9956-ffa450edef68
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ConsoleTable.java
Make a PrintStream that outputs utf8. Have all printing use it.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SelectCommand.java
Fix few places where we make Strings w/o stipulating UTF-8 encoding.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@571317 13f79535-47bb-0310-9956-ffa450edef68
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTableMapReduce.java
(localTestSingleRegionTable, localTestMultiRegionTable, verify): Added.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java
Javadoc for addContents and Loader interface and implementations.
Methods have been made static so accessible w/o subclassing.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/MultiRegionTable.java
Guts of TestSplit has been moved here so other tests can have
access to a multiregion table.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java
Bulk moved to MultiRegionTable utility class. Use this new class
instead.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java
Added '@deprecated' javadoc.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
Was throwing RuntimeException when a msgQueue.put was interrupted
but this is a likely event on shutdown. Log a message instead.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
Actually fix for HADOOP-1785... reverse test of row comparison.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@570983 13f79535-47bb-0310-9956-ffa450edef68
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
Fix outputing fail message on each compaction though there was none.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java
(rename): Refactor so return only happens on end..
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
(assignRegions): Make synchronized. In presence of concurrent visits
by regionservers, both visiting threads could grab same set of regions
for assignment.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@569589 13f79535-47bb-0310-9956-ffa450edef68
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpContents.java
Add mention of new 'fs' operator.
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/FsCommand.java
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj
Added support of new 'fs' operator.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserConstants.java
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java
Generated files. Changes come of mods to HBaseShell.jj
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@569484 13f79535-47bb-0310-9956-ffa450edef68
result
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java
Minor fix of a log message.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
(COMPACTION_DIR, WORKING_COMPACTION): Removed.
(compactdir): Renamed compactionDir.
Removed from constructor our checking if a compaction was left undone.
Instead, just ignore it. When compaction reruns whatever as left on
filesystem will just be cleaned up and we'll rerun the compaction
(Likelihood of a crash mid-compaction in exactly the area where
the compaction was recoverable are low -- more robust just redoing
the compaction from scratch).
(compactHelper): We were deleting HBaseRoot/compaction.tmp dir
after a compaction completed. Usually fine but on a cluster of
more than one machine, if two compactions were near-concurrent, one
machine could remove the compaction working directory while another
was mid-way through its compaction. Result was odd failures
during compaction of result file, during the move of the resulting
compacting file or subsequently trying to open reader on the
resulting compaction file (See HADOOP-1765).
a region fsck tool).
(getFilesToCompact): Added.
(processReadyCompaction): Added. Reorganized compaction so that the
window during which loss-of-data is possible is narrowed and even
then, we log a message with how a restore might be performed manually
(TODO: Add a repair tool).
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java
(rename): More checking around rename that it was successful.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
An empty-log gives HLog trouble. Added handling.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
Cleanup of debug level logging.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
Minor javadoc and changed a log from info to debug.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@569446 13f79535-47bb-0310-9956-ffa450edef68
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
Removed some empty lines so I can squeeze more code into a screenful.
(assignedRegions): Factored out some code into own methods so
this method is made a bit shorter. Added early returns near
top -- if nothing to assign, etc. -- so less nesting.
Added fix: Instead of iterating over unassignedRegions after
all the loadings have been calculated, instead iterate over
the locally calculated map, regionsToAssign (Otherwise, we
were running over the same territory each time through the
loop and were thus giving out same region multiple times).
(regionsPerServer, assignRegionsToOneServer,
getRegionsToAssign): Added.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@568404 13f79535-47bb-0310-9956-ffa450edef68
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
Use new HColumnDescriptor accessors rather than make direct accesses
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
(COMPRESSION_NONE, COMPRESSION_RECORD, COMPRESSION_BLOCK): Removed.
Use enum ordinals instead. Removed mapping between these defines and
enum equivalents. Made data members private and added accessors.
(DEFAULT_IN_MEMORY, DEFAULT_COMPRESSION_TYPE,
DEFAULT_BLOOM_FILTER_DESCRIPTOR, DEFAULT_MAX_VALUE_LENGTH): Added.
M hbase/src/test/org/apache/hadoop/hbase/TestToString.java
Fix because enum strings are upper-case (was expecting lowercase).
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@568275 13f79535-47bb-0310-9956-ffa450edef68
Added handling for legal null value scanning META table and added
logging of unexpected exceptions that arise scanning.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java
Refactored to do a staged removal of daughter references.
(compact, recalibrate): Added.
(getSplitParent): Refactored as getSplitParentInfo.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java
Added formatting of the find table result string so shorter
(when 30-odd regions fills page with its output).
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java
Formatting to clean eclipse warnings.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
The split column in a parent meta table entry can be null (Happens
if a daughter split no longer has references -- it removes its
entry from parent). Add handling and clean up around split
management code. Added logging of unexpected exceptions
scanning a region.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
Added fix for NPE when client asks for scanner but passes
non-existent columns.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/Writables.java
(getHRegionInfo, getHRegionInfoOrNull): Added.:
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@567308 13f79535-47bb-0310-9956-ffa450edef68
HADOOP-1725 Client find of table regions should not include offlined, split parents
Changes:
New class MapWritable replaces KeyedData and KeyedDataArrayWritable
HBaseAdmin, HConnectionManager, HMaster, HRegionInterface,
HRegionServer, HTable, TestScanner2:
- getRow returns MapWritable instead of array of KeyedData
- next returns MapWritable instead of array of KeyedData
GroupingTableMap, IdentityTableMap, IdentityTableReduce,
TableInputFormat, TableMap, TableOutputCollector, TableOutputFormat,
TestTableMapReduce:
- use MapWritable instead of KeyedData and KeyedDataArrayWritable
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@566878 13f79535-47bb-0310-9956-ffa450edef68
Disentangles flushes and compactions; flushes can proceed while a
compaction is happening. Also, don't compact unless we hit
compaction threshold: i.e. don't automatically compact on HRegion
startup so regions can come online the faster.
M src/contrib/hbase/conf/hbase-default.xml
(hbase.hregion.compactionThreashold): Moved to be a hstore property
as part of encapsulating compaction decision inside hstore.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java
Refactored. Moved here generalized content loading code that can
be shared by tests. Add to setup and teardown the setup and removal
of local test dir (if it exists).
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompare.java
Added test of HStoreKey compare (It works other than one would at
first expect).
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java
Bulk of content loading code has been moved up into the parent class.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java
(tableExists): Restore to a check of if the asked-for table is in list of
tables. As it was, a check for tableExists would just wait on all timeouts
and retries to expire and then report table does not exist.. Fixed up
debug message listing regions of a table. Added protection against meta
table not having a COL_REGINFO (Seen in cluster testing -- probably a bug
in row removal).
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java
Loading store files, even if it was noticed that there was no corresponding
map file, was still counting file as valid. Also fix merger -- was
constructing MapFile.Reader directly rather than asking HStoreFile for
the reader (HStoreFile knows how to do MapFile references)
(rename): Added check that move succeeded and logging. In cluster-testing,
the hdfs move of compacted file into place has failed on occasion (Need
more info).
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
Encapsulate ruling on whether a compaction should take place inside HStore.
Added reading of the compactionThreshold her. Compaction threshold is
currently just number of store files. Later may include other factors such
as count of reference files. Cleaned up debug messages around
reconstruction log. Removed compaction if size > 1 from constructor. Let
compaction happen after we've been deployed (Compactions that happen while
we are online can continue to take updates. Compaction in the constructor
puts off our being able to take in updates).
(close): Changed so it now returns set of store files. This used to be done
by calls to flush. Since flush and compaction have been disentangled, a
compaction can come in after flush and the list of files could be off.
Having it done by close, can be sure list of files is complete.
(flushCache): No longer returns set of store files. Added 'merging compaction'
where we pick an arbitrary store file from disk and merge into it the content
of memcache (Needs work).
(getAllMapFiles): Renamed getAllStoreFiles.
(needsCompaction): Added.
(compactHelper): Added passing of maximum sequence number if already
calculated. If compacting one file only, we used skip without rewriting
the info file. Fixed.
Refactored. Moved guts to new compact(outFile, listOfStores) method.
(compact, CompactionReader): Added overrides and interface to support
'merging compaction' that takes files and memcache. In compaction,
if we failed the move of the compacted file, all data had already been
deleted. Changing, so deletion happens after confirmed move of
compacted file.
(getFull): Fixed bug where NPE when read of maps came back null.
Revealed by our NOT compacting stores on startup. Meant could be two
backing stores one of which had no data regards queried key.
(getNMaps): Renamed countOfStoreFiles.
(toString): Added.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreKey.java
Added comment on 'odd'-looking comparison.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
Javadoc edit.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
Only return first 128 bytes of value when toStringing (On cluster,
was returning complete web pages in log).
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
Removed confusing debug message (made sense once -- but not now).
Test rootRegionLocation for null before using it (can be null).
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
Added comment that delete behavior needs study.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
Fixed merge so it doesn't do the incremental based off files
returned by flush. Instead all is done in the one go after
region closes (using files returned by close).
Moved duplicated code to new filesByFamily method.
(WriteState): Removed writesOngoing in favor of compacting and
flushing flags.
(flushCache): No longer returns list of files.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/Writables.java
Fix javadoc.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@566459 13f79535-47bb-0310-9956-ffa450edef68
Splits are now near-instantaneous. On split, daughter splits create
'references' to store files up in the parent region using new 'HalfMapFile'
class to proxy accesses against the top-half or bottom-half of
backing MapFile. Parent region is deleted after all references in daughter
regions have been let go.
Below includes other cleanups and at least one bug fix for fails adding
>32k records and improvements to make it more likely TestRegionServerAbort
will complete..
A src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java
Added. Tests new Reference HStoreFiles. Test new HalfMapFileReader inner
class of HStoreFile. Test that we do the right thing when HStoreFiles
are smaller than a MapFile index range (i.e. there is not 'MidKey').
Test we do right thing when key is outside of a HalfMapFile.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java
getHRegionDir moved from HStoreFile to HRegion.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBatchUpdate.java
Let out exception rather than catch and call 'fail'.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java
Refactored so can start and stop a minihbasecluster w/o having to
subclass this TestCase. Refactored methods in this class to use the
newly added methods listed below.
(MasterThread, RegionServerThread, startMaster, startRegionServers
shutdown): Added.
Added logging of abort, close and wait. Also on abort/close
was doing a remove that made it so subsequent wait had nothing to
wait on.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java
Added tests that assert all works properly at region level on
multiple levels of splits and then do same on a cluster.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java
Removed catch and 'fail()'.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java
Javadoc to explain how split now works. Have constructors flow
into each other rather than replicate setup per instance. Moved
in here operations such as delete, rename, and length of store files
(No need of clients to remember to delete map and info files).
(REF_NAME_PARSER, Reference, HalfMapFile, isReference,
writeReferenceFiles, writeSplitInfo, readSplitInfo,
createOrFail, getReader, getWriter, toString): Added.
(getMapDir, getMapFilePath, getInfoDir, getInfoFilePath): Added
a bunch of overrides for reference handling.
(loadHStoreFiles): Amended to load references off disk.
(splitStoreFiles): Redone to instead write references into
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
Rename maps as readers and mapFiles as storefiles.
Moved BloomFilterReader and Writer into HStoreFile. Removed
getMapFileReader and getMapFileWriter (They are in HStoreFile now).
(getReaders): Added.
(HStoreSize): Added. Data Structure to hold aggregated size
of all HStoreFiles in HStore, the largest, its midkey, and
if the HStore is splitable (May not be if references).
Previous we only did largest file; less accurate.
(getLargestFileSize): Renamed size and redone to aggregate
sizes, etc.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
Have constructors waterfall down through each other rather than
repeat initializations.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java
Use new HStoreSize structure.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
Added delayed remove of HRegion (Now done in HMaster as part of
meta scan). Change LOG.error and LOG.warn so they throw stack trace
instead of just the Exception.toString as message.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
(COLUMN_FAMILY_STR): Added.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
Added why to log of splitting.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
Short is not big enough to hold edits tha could contain a sizable
web page.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java
(getTableName): Added.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
Added constructor to BaseScanner that takes name of table we're
scanning (ROOT or META usually). Added to scanOneRegion handling
of split regions. Collect splits to check while scanning and
then outside of the scanning, so we can modify the META table
is needed, do the checks of daughter regions and update on
change of state. Made LOG.warn and LOG.error print stack trace.
(isSplitParent, cleanupSplits, hasReferences): Added.
Added toString to each of the PendingOperation implementations.
In the ShutdownPendingOperation scan of meta data, removed
check of startcode (if the server name is that of the dead
server, it needs reassigning even if start code is good).
Also, if server name is null -- possible if we are missing
edits off end of log -- then the region should be reassigned
just in case its from the dead server. Also, if reassigning,
clear from pendingRegions. Server may have died after sending
region is up but before the server confirms receipt in the
meta scan. Added mare detail to each log. In OpenPendingOperation
we were trying to clear pendingRegion in wrong place -- it was
never executed (regions were always pending).
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java
Add split boolean. Output offline and split status in toString.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
Comments.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
Moved getRegionDir here from HStoreFile.
(COL_SPLITA, COL_SPLITB): Added.
(closeAndSplit): Refactored to use new fast split method.
StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
(splitStoreFile): Moved into HStoreFile.
(getSplitRegionDir, getSplitsDir, toString): Added.
(needsSplit): Refactored to exploit new HStoreSize structure.
Also manages notion of 'unsplitable' region.
(largestHStore): Refactored.
(removeSplitFromMETA, writeSplitToMETA, getSplit, hasReference): Added.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/Keying.java
(intToBytes, getBytes): Added.
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/Writables.java
Utility reading and writing Writables.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@564012 13f79535-47bb-0310-9956-ffa450edef68
HTable
* added public method close
* added protected method checkClosed
* make getConnection public
HConnectionManager
* a call to getTableServers or reloadTableServers will cause information for closed
tables to be reloaded
TestHTable
* new test case
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@562294 13f79535-47bb-0310-9956-ffa450edef68
(except TestHClient and HBaseShell) have been converted to use the new client
side objects (HTable/HBaseAdmin/HConnection) instead of HClient.
HBaseAdmin
- Expose connection methods getMaster, isMasterRunning and listTables
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@562041 13f79535-47bb-0310-9956-ffa450edef68
Modified:
HConstants
static final Text[] COL_REGIONINFO_ARRAY = new Text [] {COL_REGIONINFO};
static final Text EMPTY_START_ROW = new Text();
HMaster
- don't process a region server exit message if the lease has timed
out. Otherwise we end up with two pending server shutdown messages
to process and chaos ensues.
- don't reassign the root region when the server's lease expires. The
lease expiration handler will queue a PendingServerShutdown
operation that must run before the root region is reassigned because
the HLog of the dead server must be split before any regions served
by the dead server are reassigned.
- added some additional debug level logging
HBaseClusterTestCase
- call HConnectionManager.deleteConnection(conf) in tearDown() so that
multiple tests can be run from the same test class.
TestScanner2
- changes to make test compatible with the change from inner class
HClient.RegionLocation to public class HRegionLocation
Leases
- cancelLease just returns if the lease is not found instead of
throwing an IOException
New:
HConnection - an interface that describes the operations performed by
a connection implementation
HConnectionManager - manages connections for multiple HBase instances
and returns an object that implements HConnection from its static
method getConnection
HBaseAdmin - the HBase administrative methods refactored out of
HClient. Each HBaseAdmin object can control a single HBase
instance. To manipulate multiple instances, create multiple HBaseAdmin
objects.
HTable - The data manipulation methods refactored out of HClient. Each
HTable object talks to a single table in a single HBase
instance. Create multiple HTable objects to use more than one table.
HRegionLocation - an inner class refactored out of HClient. Each
HRegionLocation has an HRegionInfo object and an HServerAddress
object.
HClient - totally re-implemented in terms of the new classes
above. HClient is now deprecated.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@561935 13f79535-47bb-0310-9956-ffa450edef68
Detailed changes:
MiniHBaseCluster
- rewrite abortRegionServer, stopRegionServer - they now remove the
server from the map of servers.
- rewrite waitOnRegionServer - now removes thread from map of threads
TestCleanRegionServerExit
- reduce Hadoop ipc client timeout and number of retries
- use rewritten stopRegionServer and waitOnRegionServer from MiniHBaseCluster
- add code to verify that failover worked
- moved testRegionServerAbort to separate test file
TestRegionServerAbort
- new test. Uses much the same code as TestCleanRegionServerExit but
aborts the region server instead of shutting it down
cleanly. Includes code to verify that failover worked.
hbase-site.xml (in src/contrib/hbase/src/test)
- reduce master lease timeout and time between lease timeout checks so
that tests will run quicker.
HClient
- Major restructing of code that determines what region server to
contact for a specific region. The main method findServersForTable
is now recursive so that it will find the meta and root regions if
they have not already been located or will re-find them if they have
been reassigned and the old server can no longer be contacted.
- re-ordered administrative and general purpose methods so they are no
longer located in seemingly random order.
- re-ordered code in ClientScanner.loadRegions so that if the location
of the region changes, it will actually try to connect to the new
server rather than continually trying to use the connection to the
old server.
HLog
- use HashMap<Text, SequenceFile.Writer> instead of
TreeMap<Text, SequenceFile.Writer> because the TreeMap would return
a value for a key it did not have (it was the value of another
key). I have observed this before when the key is Text, but could
not create a simple test case that reproduced the problem.
- added some new DEBUG level logging
- removed call to rollWriter() from closeAndDelete(). We don't need to
start a new writer if we are closing the log.
HLogKey
- cleaned up per HADOOP-1466 (I initially modified it to add some
debug logging which was later removed, but when I was making the
modifications I took the opportunity to clean up the file)
- changed toString() format
HMaster
- better handling of RemoteException
- modified BaseScanner
- now knows if it is scanning the root or a meta region
- scanRegion no longer returns a value
- if scanning the root region, it counts the number of meta regions
it finds and sets a new AtomicInteger, numberOfMetaRegions when the
scan is complete.
- added abstract methods initialScan and maintenanceScan this allowed
run method to be implemented in the base class.
- boolean rootScanned is now volatile
- modified RootScanner
- moved actual scan into private method for readability (scanRoot)
- implementation of abstract methods just call scanRoot
- add constructor for inner static class MetaRegion
- use a BlockingQueue to queue up work for the MetaScanner
- clean up handling of an unexpected region server exit
- PendingOperation.process now returns a boolean so that HMaster.run
can determine if the operation completed or needs to be retried later
- PendingOperation processing no longer does a wait inside the process
method since this might cause a deadlock if the current operation is
waiting for another operation that has yet to be processed
HMsg
- removed MSG_REGIONSERVER_STOP_IN_ARRAY, MSG_NEW_REGION
- added MSG_REPORT_SPLIT
HRegionServer
- changed reportSplit to contain old region and new regions
- use IP from default interface rather than host name
- abort calls HLog.close() instead of HLog.rollWriter()
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@559819 13f79535-47bb-0310-9956-ffa450edef68
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ClearCommand.java
Added.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj
Add 'clear' command handling.
(clearCommand): Added.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserConstants.java
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java
Add 'clear' command handling (Generated by javacc).
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpContents.java
Add 'CLEAR' help.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpManager.java
Clear screen before outputting version string.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@556356 13f79535-47bb-0310-9956-ffa450edef68
Change format of region names from TABLENAME_STARTROW_ENDROW-RANDOMID
to TABLENAME,STARTROW,ENDROW-RANDOMID. Makes it so lone table name will
sort before any region of said table.
M src/contrib/hbase/src/test/hbase-site.xml
(hbase.client.retries.number): Removed. Wasdefault value for this property.
(hbase.master.meta.thread.rescanfrequency, hbase.server.thread.wakefrequency,
hbase.regionserver.handler.count): Add values that are less than
default so unit tests are even more responsive (and finished quicker).
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestToString.java
Change test so it expects region info name that has ',' delimiters
rather than '_' delimiters.
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTable.java
Rename testTable as testCreateTable.
(testTableNameClash): Test for this issue.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java
Change format of region names so delimiter is ',' rather than '_'.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@556348 13f79535-47bb-0310-9956-ffa450edef68
M src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTable.java
(testTable): Add checking of actual exceptions thrown and
assertions that we are getting right behavior. Add a test
that has ten clients concurrently trying to create same table.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
javadoc edit. Fix debug message that could give impression
table was found when it wasn't.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
Added comment on table nameing.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
(createTable): Refactored. Bulk moved to a private override.
Changed how check for existance is done.
M rc/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java
(getTableNameFromRegionName): Utility method added.
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/TableExistsException.java
Added.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@556334 13f79535-47bb-0310-9956-ffa450edef68
M src/contrib/hbase/NOTICE.txt
Add notice of udanax contributions.
Msrc/contrib/hbase/conf/hbase-default.xml
(hbaseshell.jline.bell.enabled): Added.
M src/contrib/hbase/CHANGES.txt
(hadoop-1375) Added.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/package.html
Add note on how to start up hbase shell
M src/contrib/hbase/bin/hbase
Add 'shell'. Remove 'client' (shell does what it used do and more).
Removed all reader and logreader until better developed. Starting
up a reader or logreader on a running hbase system could do damage).
M src/contrib/hbase/build.xml
Add a javacc target to generate content of shell/generated subpackage.
A src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestHBaseShell.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/Shell.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DeleteCommand.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/CreateCommand.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DropCommand.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/InsertCommand.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/CommandFactory.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpContents.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ExitCommand.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ConsoleTable.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DescCommand.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SelectCommand.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/Command.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/BasicCommand.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpManager.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ReturnMsg.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpCommand.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj
Added.
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Token.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/TokenMgrError.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/SimpleCharStream.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParseException.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserConstants.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java
Added javacc generated files.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@555415 13f79535-47bb-0310-9956-ffa450edef68
Adds a row/column filter interface and two implementations: A pager and a
row/column-value regex filter.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
(openScanner): Add override that specifies a row fliter.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
(obtainScanner): Add override that specifies a row fliter.
(ColumnScanner): Add filter parameter to constructor.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
(getScanner): Add override with filter parameter.
(next): Add handling of filtering.
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java
Row-filter interface, exception and implementations.
A src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java
A src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java
Simple pager and regex filter tests.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@553620 13f79535-47bb-0310-9956-ffa450edef68
On shutdown, region servers and masters were just cancelling leases
without letting 'lease expired' code run -- code to clean up
outstanding locks in region server. Outstanding read locks were
getting in the way of region server getting necessary write locks
needed for the shutdown process. Also, cleaned up messaging around
shutdown so its clean -- no timeout messages as region servers try
to talk to a master that has already shutdown -- even when region
servers take their time going down.
M src/contrib/hbase/conf/hbase-default.xml
Make region server timeout 30 seconds instead of 3 minutes.
Clients retry anyways. Make so its likely region servers report
in their shutdown message before their lease expires on master.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/Leases.java
(closeAfterLeasesExpire): Added.
* src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
Added comments.
(stop): Converted from public to default access (master shuts
down regionservers).
(run): Use leases.closeAfterLeasesExpire instead of leases.close.
Changed log of main thread exit from DEBUG to INFO.
* src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
(letRegionsServersShutdown): Add better explaination of shutdown
process to method doc. Changed timeout waits from
hbase.regionserver.msginterval to threadWakeFrequency.
(regionServerReport): If closing, we used to immediately respond
to region server with a MSG_REGIONSERVER_STOP. This meant that
we avoided handling of the region servers MSG_REPORT_EXITING sent
on shutdown so region servers had no chance to cancel their lease
in the master. Reordered. Moved sending of MSG_REGIONSERVER_STOP
to after handling of MSG_REPORT_EXITING. Also, in handling of
MSG_REGIONSERER_STOP removed cancelling of leases. Let leases
expire normally (or get cancelled when the region server comes in
with MSG_RPORT_EXITING).
* src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMsg.java
(MSG_REGIONSERVER_STOP_IN_ARRAY): Added.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@552376 13f79535-47bb-0310-9956-ffa450edef68
AbstractMergeTestBase, HBaseTestCase: move createNewHRegion to HBaseTestCase
MiniHBaseCluster: add deleteOnExit, getDFSCluster, fix Javadoc
TestScanner2: moved KeyedData to org.apache.hadoop.hbase.io
TestTableMapReduce: new test case to test map/reduce interface to HBase
hbase-site.xml: change hbase.client.pause from 3 to 5 seconds, hbase.client.retries.number to 5 so that tests will not time out or run out of retries
HClient: moved KeyedData to org.apache.hadoop.hbase.io, fix javadoc, add method getStartKeys
HMaster: moved KeyedData to org.apache.hadoop.hbase.io, remove unused variables, remove extraneous throws clause,
HRegionInterface, HRegionServer: moved KeyedData to org.apache.hadoop.hbase.io
KeyedData: moved KeyedData to org.apache.hadoop.hbase.io
KeyedDataArrayWritable: new class to support HBase map/reduce
org.apache.hadoop.hbase.mapred: new classes for map/reduce
- GroupingTableMap
- IdentityTableMap
- IdentityTableReduce
- TableInputFormat
- TableMap
- TableOutputCollector
- TableOutputFormat
- TableReduce
- TableSplit
hbase/bin/hbase: changes for map/reduce
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@552127 13f79535-47bb-0310-9956-ffa450edef68
* src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMemcache.java
(findFirstRow): Change compare from startsWith -- looking for
an 'exact' match -- to instead use compareTo and if >= 0, then
we are at first key.
* src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
Add logging of problematic directory if server won't start because
log file exists.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@551039 13f79535-47bb-0310-9956-ffa450edef68
so that region servers are assigned the regions have a log to apply edits from. Enhance fail over
capabilities.
For all the files modified, clean up javadoc, class method and field visibility.
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@546192 13f79535-47bb-0310-9956-ffa450edef68