When a Put fails due to a NotServingRegionException, the cached location
of that Region is never cleared. Thus, subsequent calls to resubmit
the Put will fail in the same way as the original, never determining
the new location of the Region.
If the Connection is not closed by the user before the Multiplexer
is discarded, it will leak resources and could cause resource
issues.
Signed-off-by: Sean Busbey <busbey@cloudera.com>
Summary:
* Add VersionInfoUtil to determine if a client has a specified version or better
* Add an exception type to say that the response should be chunked
* Add on client knowledge of retry exceptions
* Add on metrics for how often this happens
Test Plan: Added a unit test
Differential Revision: https://reviews.facebook.net/D51771
Summary:
Create and use a copy on write map for region location.
- Create a copy on write map backed by a sorted array.
- Create a test for both comparing each with a jdk provided map.
- Change MetaCache to use the new map.
Test Plan:
- org.apache.hadoop.hbase.client.TestFromClientSide
- TestHCM
Differential Revision: https://reviews.facebook.net/D49545
Summary:
Use concurrent collections and atomic longs to keep track of edits in buffered mutator.
This keeps buffered mutator as thread safe but it means that shared buffered mutators are not contending on thread locking.
Test Plan: Unit Tests.
Differential Revision: https://reviews.facebook.net/D49467
Summary: Send the list of mutations to AsyncProcess only after done adding the list otherwise there's a lot of contention
Test Plan: UnitTests.
Differential Revision: https://reviews.facebook.net/D49251
* corrects license/notice for source distribution
* adds inception year to correct copyright in generated NOTICE files for jars
* updates project names in poms to use "Apache HBase" instead of "HBase" so jar NOTICE files will be correct
* uses append-resources to include supplemental info on jars with 3rd party works in source
* adds an hbase specific resource bundle for jars that include 3rd party works for binaries
** uses supplemental-model to fill in license gaps
** uses the above and a shade plugin transformation to build proper files for shaded jars.
** uses the above and the assembly plugin to build the proper files for bin assembly
* adds a NOTICE item for things copied out of Hadoop (TODO legal-discuss)
Conflicts:
NOTICE.txt
hbase-checkstyle/pom.xml
hbase-resource-bundle/pom.xml
hbase-server/pom.xml
hbase-thrift/pom.xml
pom.xml
Amending-Author: Sean Busbey <busbey@apache.org>
Signed-off-by: Sean Busbey <busbey@apache.org>
Conflicts:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
* includes additiona branch-1 specific fixes to javadocs
* includes rollback of 2 changes from HBASE-13898 that didn't apply to branch-1
Conflicts:
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
* change pom to use a maven 3 compat version of clover
* add clover to javadoc plugin deps so that instrumented doclet works
* modify IA annotation test to filter out clover instrumentation
* make splitlog counters check for atomiclong before casting
* IA.Public accessible logger instances deprecated
* logger instances modified by tests left in place
* all others made private static final
Signed-off-by: Sean Busbey <busbey@apache.org>
Conflicts:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientIdGenerator.java
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java
hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
hbase-server/src/test/java/org/apache/hadoop/hbase/TestTableDescriptor.java
hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java
hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java
Use the context passed back via ScanResponse that a RegionServer
fills in to denote whether or not more results existing in the
current Region. Add a simple factory to remove a static method
used across both SmallScanner and SmallReverseScanner. Add new
unit tests for both scanner classes to test scans with and
without the new context (as a quick backward-compatibility test).
The RS already returns to the client whether or not it has additional
results to be returned in a subsequent call to scan(), but the ClientScanner
did not use or adhere to this value. Subsequently, this can lead to
bugs around moving to the next region too early. A new method was added
to ClientScanner in the name of testability.
Encapsulate server-state into RegionServerCallable to avoid
modifying parameterization of callable impls.
Signed-off-by: Andrew Purtell <apurtell@apache.org>
Include some basic tests for the method on a testing cluster.
Also update master page to show an alert when balancer is disabled.
Signed-off-by: Enis Soztutar <enis@apache.org>
Conflicts:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
Summary:
The current behavior of a region move shuts down a region and then starts is up in another regionserver. This causes increased latency and possibly timeouts till the new region's cache is fully warmed up. We can make a region move less disruptive by warming the cache in the destination region server before shutting dow the old region.
See https://issues.apache.org/jira/browse/HBASE-13316
Test Plan:
1. Unit Tests
2. Added test for concurrent moves and warmups
3. Manually tested reads/writes happening with concurrent moves
Subscribers: tedyu
Differential Revision: https://reviews.facebook.net/D35967
Signed-off-by: Elliott Clark <eclark@apache.org>
Adds a number of lifecycle-mapping entries which
prevent errors from showing up in Eclipse on a fresh
import of HBase. For plugins defined in the top-level
pom, the mapping is added there; otherwise, the mapping
is pushed down to the child pom.
Signed-off-by: Sean Busbey <busbey@apache.org>
* move mapreduce version of TableInputFormat tests out of mapred
* add ability to get runnable job via MR test shims
* correct the javadoc example for current APIs.
* add tests the run a job based on the extending TableInputFormatBase (as given in the javadocs)
* add tests that run jobs based on the javadocs from 0.98
* fall back to our own Connection if ussers of the deprecated table configuration have a managed connection.
In our pre-1.0 API, HTable is considered a light-weight object that consumed by
a single thread at a time. The HTablePool class provided a means of sharing
multiple HTable instances across a number of threads. As an optimization,
HTable managed a "write buffer", accumulating edits and sending a "batch" all
at once. By default the batch was sent as the last step in invocations of
put(Put) and put(List<Put>). The user could disable the automatic flushing of
the write buffer, retaining edits locally and only sending the whole "batch"
once the write buffer has filled or when the flushCommits() method in invoked
explicitly. Explicit or implicit batch writing was controlled by the
setAutoFlushTo(boolean) method. A value of true (the default) had the write
buffer flushed at the completion of a call to put(Put) or put(List<Put>). A
value of false allowed for explicit buffer management. HTable also exposed the
buffer to consumers via getWriteBuffer().
The combination of HTable with setAutoFlushTo(false) and the HTablePool
provided a convenient mechanism by which multiple "Put-producing" threads could
share a common write buffer. Both HTablePool and HTable are deprecated, and
they are officially replaced in The new 1.0 API by Table and BufferedMutator.
Table, which replaces HTable, no longer exposes explicit write-buffer
management. Instead, explicit buffer management is exposed via BufferedMutator.
BufferedMutator is made safe for concurrent use. Where code would previously
retrieve and return HTables from an HTablePool, now that code creates and
shares a single BufferedMutator instance across all threads.
Conflicts:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java
hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java
hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java
Add some fixup of checkstyle fails
Conflicts:
hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
Issues incorporated and incompatible changes called out on the jira.
* Cleaned up references to HLog
* Deprecates HLogKey but maintains it for compatibility
- Moves all Writeable from WALKey to HLogKey
* Adds utility code to CoprocessorHost to help with evolving Coprocessor APIs
* RSRpcServices roll WAL call now requests the non-meta LogRoller roll all logs
- rolls actually happen asynchronously
- deprecated old api (and noted incompatible behavior change)
- modified api in new Admin interface to reflect lack of return values.
* Moved WAL user facing API to "WAL"
- only 1 sync offered
- WALTrailer removed from API
* make provider used by the WALFactory configurable.
* Move all WAL requests to use opaque ids instead of paths
* WALProvider provides API details for implementers and handles creation of WALs.
* Refactor WALActionsListener to have a basic implementation.
* turn MetricsWAL into a WALActionsListener.
* tests that needs FSHLog implementation details use them directly, others just reference provider + factory
- Some tests moved from Large to Medium based on run time.
* pull out wal disabling into its own no-op class
* update region open to delegate to WALFactory
* update performance test tool to allow for multiple regions
* Removed references to meta-specific wals within wal code
- replaced with generic suffixes
- WALFactory maintains a dedicated WALProvider for meta (and so knows about the distinction)
* maintain backwards compat on HLogPrettyPrinter and mark it deprecated.
- made WALPrettyPrinter IA.Private in favor of `bin/hbase wal`
* move WALUtil stuff that's implementation specific to said implementation
- WALUtil now acts as an integration point between the RegionServer and hte WAL code.
Incorporates contributions from v.himanshu and stack.
Conflicts:
hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java
hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java
hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java
hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationWALReaderManager.java
hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java
hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java
hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFiltering.java
hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplitCompressed.java
src/main/docbkx/ops_mgt.xml
HConnection#getTable (0.98, 0.99)
Backport
Replaced HTable under hbase-*/src/main/java. Skipped tests. Would take
till end of time to do all and some cases are cryptic. Also skipped
some mapreduce where HTable comes through in API. Can do both of
these stragglers in another issue.
Generally, if a utility class or standalone class, tried to pass in a
Connection rather than have the utility or standalone create its own
connection on each invocation; e.g. the Quota stuff. Where not possible,
noted where invocation comes from... if test or hbck, didn't worry about
it.
Some classes are just standalone and nothing to be done to avoid
a Connection setup per invocation (this is probably how it worked
in the new HTable...days anyways). Some classes are not used:
AggregationClient, FavoredNodes... we should just purge this stuff.
Doc on what short circuit connection does (I can just use it...
I thought it was just for short circuit but no, it switches dependent
on where you are connecting).
Changed HConnection to super Interface ClusterConnection where safe (
internal usage by private classes only).
Doc cleanup in example usage so we do new mode rather than the old
fashion.
Used java7 idiom that allows you avoid writing out finally to call close
on implementations of Closeable.
Added a RegistryFactory.. moved it out from being inner class.
Added a utility createGetClosestRowOrBeforeReverseScan method to Scan
to create a Scan that can ...
Renamed getShortCircuitConnection as getConnection – users don't need
to know what implementation does (that it can short-circuit RPC).
The old name gave pause. I was frightened to use it thinking it only
for short-circuit reading – that it would not do remote too.
Squashed commit of the following:
Move from HConnection to ClusterConnection or Connection
Use unmanaged connections where we use managed previous
(used the jdk7 https://docs.oracle.com/javase/7/docs/technotes/guides/language/try-with-resources.html idiom).
In ZKConfig, synchronize on Configuration rather than make a copy.
Making a copy we were dropping hbase configs in certain test context
(could not find the zk ensemble because default port).
In tests, some move to the new style connection setup but mostly
fixes for premature connection close or adding cleanup where it
was lacking.
Conflicts:
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
Summary: This diff ports the Preemptive Fast Fail feature to OSS. In multi threaded clients, we use a feature developed on 0.89-fb branch called Preemptive Fast Fail. This allows the client threads which would potentially fail, fail fast. The idea behind this feature is that we allow, among the hundreds of client threads, one thread to try and establish connection with the regionserver and if that succeeds, we mark it as a live node again. Meanwhile, other threads which are trying to establish connection to the same server would ideally go into the timeouts which is effectively unfruitful. We can in those cases return appropriate exceptions to those clients instead of letting them retry.
Test Plan: Unit tests
Differential Revision: https://reviews.facebook.net/D24177
Signed-off-by: stack <stack@apache.org>
Conflicts:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
Summary: This is a follow up diff for the jira to add the functionality to update configs in an online fashion.
Test Plan: Unit test. This adds an end point on the regionserver. It just tries to call the end point and make sure the call succeeds.
Differential Revision: https://reviews.facebook.net/D24861
Signed-off-by: stack <stack@apache.org>
Conflicts:
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java
Summary: # Clear the cache of the server when failed
Test Plan: Add testcase `TestHTableMultiplexerFlushCache` to reproduce the bug.
Differential Revision: https://reviews.facebook.net/D24603
Signed-off-by: Elliott Clark <elliott@fb.com>
Signed-off-by: Elliott Clark <eclark@apache.org>
Fix javadoc warnings.
Fixup findbugs warnings mostly by adding annotations saying 'working as expected'.
In RpcRetryingCallerWithReadReplicas made following change which findbugs spotted:
- if (completed == null) tasks.wait();
+ while (completed == null) tasks.wait();
In RecoverableZooKeeper, made all zk accesses synchronized -- we were doing it
half-ways previously.
In RatioBasedCompactionPolicy we were making an instance of Random on
each invocation of getNextMajorCompactionTime
Conflicts:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowTooBigException.java