Passed all regression tests and PerformanceEvaluation running with multiple region servers. table.jsp now displays the correct information
HRegionInfo:
- removed getTableNameFromRegionName and parseMetaRegionRow we have the information in the meta table, just use it.
HServerInfo:
- I had originally made some changes here but removed them. The only remaining changes are javadoc
MetaScanner:
- build region name using ZEROES instead of NINES. When you scan you need a row name that sorts before the first row rather than after the last row.
- scan using COLUMN_FAMILY_ARRAY instead of COL_REGIONINFO_ARRAY. This way you also get the server name and start code
- change api for MetaScannerVisitor so that processRow only gets the RowResult. If you have the RowResult you have everything you need.
HConnectionManager:
- change listTables' MetaScannerVisitor to conform to new processRow api
HTable:
- change getStartKeys' MetaScannerVisitor to conform to new processRow api
- getRegionsInfo: use new processRow api, and get the server address out of the RowResult, rather than relying on the one that is cached in HConnectionManager
ScannerCallable:
- make constructor public, add javadoc
HMaster
- change createTable to get the HRegionInfo out of the RowResult and compare table name from HRegionInfo with that in the HRegionInfo of the table being created, instead of parsing the table name out of the row
TestHTable
- modify test's MetaScannerVisitor to conform to the new processRow api
TestHRegionInfo
- remove testParse as parse method no longer exists
table.jsp
- change catch of IOException to catch Exception and print stack trace. At least you'll be able to see why the server crashes if it does.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@678208 13f79535-47bb-0310-9956-ffa450edef68
Changed InternalScanner API:
from
{code}
boolean next(HStoreKey, SortedMap<byte[], byte>)
{code}
to
{code}
boolean next(HStoreKey, SortedMap<byte[], Cell>)
{code}
Files changed as a result of API change:
HMerge.java, ColumnValueFilter, PageRowFilter, RegExpRowFilter, RowFilterInterface, RowFilterSet, StopRowFilter, WhileMatchRowFilter, HAbstractScanner, HRegion$HScanner, HRegionServer, HStoreScanner, InternalScanner, Memcache, StoreFileScanner, MetaUtils, HBaseTestCase, TestScannerAPI, TimestampTestBase, TestRegExpRowFilter, TestRowFilterAfterWrite, TestRowFilterOnMultipleFamilies, TestRowFilterSet, TestGet2, TestHMemcache, TestHRegion, TestScanner, TestSplit
Update comments in Flusher
HRegion.internalFlushCache, HStore.flushCache, HStore.internalFlushCache now returns true only if a flush was completed and a compaction is needed.
HRegion.internalFlushCache now includes region name in DroppedSnapshotException
When creating a reader during compaction, don't bother with bloom filter since we won't use it.
StoreFileScanner locks the store for read while it is in the constructor. It also does not load the bloom filter when it opens readers on the store files.
TestScannerTimes - new regression test for HBASE-737
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@677113 13f79535-47bb-0310-9956-ffa450edef68
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