HBASE Changelog

Release 2.4.1 - 2021-01-18

NEW FEATURES:

JIRASummaryPriorityComponent
HBASE-24620Add a ClusterManager which submits command to ZooKeeper and its Agent which picks and execute those Commands.Majorintegration tests

IMPROVEMENTS:

JIRASummaryPriorityComponent
HBASE-25249Adding StoreContextMajor.
HBASE-25449‘dfs.client.read.shortcircuit’ should not be set in hbase-default.xmlMajorconf
HBASE-25329Dump region hashes in logs for the regions that are stuck in transition for more than a configured amount of timeMinor.
HBASE-25476Enable error prone check in pre commitMajorbuild
HBASE-25211Rack awareness in region_moverMajor.
HBASE-25483set the loadMeta log level to debug.MajorMTTR, Region Assignment
HBASE-25435Slow metric value can be configuredMinormetrics
HBASE-25318Configure where IntegrationTestImportTsv generates HFilesMinorintegration tests
HBASE-24850CellComparator perf improvementCriticalPerformance, scan
HBASE-25425Some notes on RawCellTrivial.
HBASE-25420Some minor improvements in rpc implementationMinorrpc
HBASE-25246Backup/Restore hbase cell tags.Majorbackup&restore
HBASE-25328Add builder method to create Tags.Minor.

BUG FIXES:

JIRASummaryPriorityComponent
HBASE-25356HBaseAdmin#getRegion() needs to filter out non-regionName and non-encodedRegionNameMajorshell
HBASE-25279Non-daemon thread in ZKWatcherCriticalZookeeper
HBASE-25504[branch-2.4] Restore method removed by HBASE-25277 to LP(CONFIG) coprocessorsMajorcompatibility, Coprocessors, security
HBASE-25503HBase code download is failing on windows with invalid path errorMajor.
HBASE-24813ReplicationSource should clear buffer usage on ReplicationSourceManager upon terminationMajorReplication
HBASE-25459WAL can't be cleaned in some scenesMajor.
HBASE-25434SlowDelete & SlowPut metric value should use updateDelete & updatePutMajorregionserver
HBASE-25441add security check for some APIs in RSRpcServicesCritical.
HBASE-25432we should add security checks for setTableStateInMeta and fixMetaBlocker.
HBASE-25445Old WALs archive fails in procedure based WAL splitCriticalwal
HBASE-25287Forgetting to unbuffer streams results in many CLOSE_WAIT sockets when loading filesMajor.
HBASE-25447remoteProc is suspended due to OOM ERRORMajorproc-v2
HBASE-24755[LOG][RSGroup]Error message is confusing while adding a offline RS to rsgroupMajorrsgroup
HBASE-25463Fix comment errorMinorshell
HBASE-25457Possible race in AsyncConnectionImpl between getChoreService and closeMajorClient
HBASE-25456setRegionStateInMeta need security checkCritical.
HBASE-25383HBase doesn't update and remove the peer config from hbase.replication.source.custom.walentryfilters if the config is already set on the peer.Major.
HBASE-25404Procedures table Id under master web UI gets word break to single characterMinorUI
HBASE-25277postScannerFilterRow impacts Scan performance a lot in HBase 2.xCriticalCoprocessors, scan
HBASE-25365The log in move_servers_rsgroup is incorrectMinor.
HBASE-25372Fix typo in ban-jersey section of the enforcer plugin in pom.xmlMajorbuild
HBASE-25361[Flakey Tests] branch-2 TestMetaRegionLocationCache.testStandByMetaLocationsMajorflakies

TESTS:

JIRASummaryPriorityComponent
HBASE-25502IntegrationTestMTTR fails with TableNotFoundExceptionMajorintegration tests
HBASE-25334TestRSGroupsFallback.testFallback is flakyMajor.
HBASE-25370Fix flaky test TestClassFinder#testClassFinderDefaultsToOwnPackageMajortest

SUB-TASKS:

JIRASummaryPriorityComponent
HBASE-25293Followup jira to address the client handling issue when chaning from meta replica to non-meta-replica at the server side.Minor.
HBASE-25353[Flakey Tests] branch-2 TestShutdownBackupMasterMajorflakies

OTHER:

JIRASummaryPriorityComponent
HBASE-25083make sure the next hbase 1.y release has Hadoop 2.10 as a minimum versionMajordocumentation, hadoop2
HBASE-25333Add maven enforcer rule to ban VisibleForTesting importsMajorbuild, pom
HBASE-25452Use MatcherAssert.assertThat instead of org.junit.Assert.assertThatMajortest
HBASE-25400[Flakey Tests] branch-2 TestRegionMoveAndAbandonMajor.
HBASE-25389[Flakey Tests] branch-2 TestMetaShutdownHandlerMajorflakies

Release 2.4.0 - Unreleased (as of 2020-12-03)

NEW FEATURES:

JIRASummaryPriorityComponent
HBASE-25242Add Increment/Append support to RowMutationsCriticalClient, regionserver
HBASE-25278Add option to toggle CACHE_BLOCKS in count.rbMinorshell
HBASE-18070Enable memstore replication for meta replicaCritical.
HBASE-24528Improve balancer decision observabilityMajorAdmin, Balancer, Operability, shell, UI
HBASE-24776[hbtop] Support Batch modeMajorhbtop
HBASE-24602Add Increment and Append support to CheckAndMutateMajor.
HBASE-24760Add a config hbase.rsgroup.fallback.enable for RSGroup fallback featureMajorrsgroup
HBASE-24694Support flush a single column family of tableMajor.
HBASE-24289Heterogeneous Storage for Date Tiered CompactionMajorCompaction
HBASE-24038Add a metric to show the locality of ssd in table.jspMajormetrics
HBASE-8458Support for batch version of checkAndMutate()MajorClient, regionserver

IMPROVEMENTS:

JIRASummaryPriorityComponent
HBASE-25339Method parameter and member variable are duplicated in checkSplittable() of SplitTableRegionProcedureMinor.
HBASE-25237‘hbase master stop’ shuts down the cluster, not the master onlyMajor.
HBASE-25324Remove unnecessary array to list conversion in SplitLogManagerMinor.
HBASE-25263Change encryption key generation algorithm used in the HBase shellMajorencryption, shell
HBASE-25323Fix potential NPE when the zookeeper path of RegionServerTracker does not exist when startMinor.
HBASE-25281Bulkload split hfile too many times due to unreasonable split pointMinortooling
HBASE-25325Remove unused class ClusterSchemaExceptionMinor.
HBASE-25213Should request Compaction when bulkLoadHFiles is doneMinor.
HBASE-24877Add option to avoid aborting RS process upon uncaught exceptions happen on replication sourceMajorReplication
HBASE-24664Some changing of split region by overall region size rather than only one store sizeMajorregionserver
HBASE-25026Create a metric to track full region scans RPCsMinor.
HBASE-25289[testing] Clean up resources after tests in rsgroup_shell_test.rbMajorrsgroup, test
HBASE-25261Upgrade Bootstrap to 3.4.1Majorsecurity, UI
HBASE-25272Support scan on a specific replicaCriticalClient, scan
HBASE-25267Add SSL keystore type and truststore related configs for HBase RESTServerMajorREST
HBASE-25181Add options for disabling column family encryption and choosing hash algorithm for wrapped encryption keys.Majorencryption
HBASE-25254Rewrite TestMultiLogThreshold to remove the LogDelegate in RSRpcServicesMajorlogging, test
HBASE-25252Move HMaster inner classes outMinormaster
HBASE-25240gson format of RpcServer.logResponse is abnormalMinor.
HBASE-25210RegionInfo.isOffline is now a duplication with RegionInfo.isSplitMajormeta
HBASE-25212Optionally abort requests in progress after deciding a region should closeMajorregionserver
HBASE-24859Optimize in-memory representation of mapreduce TableSplit objectsMajormapreduce
HBASE-24967The table.jsp cost long time to load if the table include closed regionsMajorUI
HBASE-25167Normalizer support for hot config reloadingMajormaster, Normalizer
HBASE-24419Normalizer merge plans should consider more than 2 regions when possibleMajormaster, Normalizer
HBASE-25224Maximize sleep for checking meta and namespace regions availabilityMajormaster
HBASE-25223Use try-with-resources statement in snapshot packageMinor.
HBASE-25201YouAreDeadException should be moved to hbase-server moduleMajorClient
HBASE-25193Add support for row prefix and type in the WAL Pretty Printer and some minor fixesMinorwal
HBASE-25128RSGroupInfo's toString() and hashCode() does not take into account configuration map.Minorrsgroup
HBASE-24628Region normalizer now respects a rate limitMajorNormalizer
HBASE-25179Assert format is incorrect in HFilePerformanceEvaluation class.MinorPerformance, test
HBASE-25065WAL archival to be done by a separate threadMajorwal
HBASE-14067bundle ruby files for hbase shell into a jar.Majorshell
HBASE-24875Remove the force param for unassign since it dose not take effect any moreMajorClient
HBASE-24025Improve performance of move_servers_rsgroup and move_tables_rsgroup by using async region move APIMajorrsgroup
HBASE-25160Refactor AccessController and VisibilityControllerMajor.
HBASE-25146Add extra logging at info level to HFileCorruptionChecker in order to report progressMajorhbck, hbck2
HBASE-24054The Jetty's version number leak occurred while using the thrift serviceMinor.
HBASE-25091Move LogComparator from ReplicationSource to AbstractFSWALProvider#.WALsStartTimeComparatorMinor.
HBASE-24981Enable table replication fails from 1.x to 2.x if table already exist at peer.MajorReplication
HBASE-25109Add MR Counters to WALPlayer; currently hard to tell if it is doing anythingMajor.
HBASE-25082Per table WAL metrics: appendCount and appendSizeMajormetrics
HBASE-25079Upgrade Bootstrap to 3.3.7Majorsecurity, UI
HBASE-24976REST Server failes to start without any error messageMajorREST
HBASE-25066Use FutureUtils.rethrow in AsyncTableResultScanner to better catch the stack traceMajorClient, Scanners
HBASE-25069Display region name instead of encoded region name in HBCK report page.Minorhbck
HBASE-24991Replace MovedRegionsCleaner with guava cacheMinor.
HBASE-25057Fix typo “memeber”Trivialdocumentation
HBASE-24764Add support of adding base peer configs via hbase-site.xml for all replication peers.MinorReplication
HBASE-25037Lots of thread pool are changed to non daemon after HBASE-24750 which causes trouble when shutting downMajor.
HBASE-24831Avoid invoke Counter using reflection in SnapshotInputFormatMajor.
HBASE-25002Create simple pattern matching query for retrieving metrics matching the patternMajor.
HBASE-25022Remove ‘hbase.testing.nocluster’ configMajortest
HBASE-25006Make the cost functions optional for StochastoicBalancerMajor.
HBASE-24974Provide a flexibility to print only row key and filter for multiple tables in the WALPrettyPrinterMinorwal
HBASE-24994Add hedgedReadOpsInCurThread metricMinormetrics
HBASE-25005Refactor CatalogJanitorMajormaster, meta
HBASE-24992log after Generator success when running ITBLLTrivial.
HBASE-24937table.rb use LocalDateTime to replace InstantMinorshell
HBASE-24940runCatalogJanitor() API should return -1 to indicate already running statusMajor.
HBASE-24973Remove read point parameter in method StoreFlush#performFlush and StoreFlush#createScannerMinor.
HBASE-24569Get hostAndWeights in addition using localhost if it is null in local modeMinorregionserver
HBASE-24949Optimize FSTableDescriptors.get to not always go to fs when cache missMajormaster
HBASE-24898Use EnvironmentEdge.currentTime() instead of System.currentTimeMillis() in CurrentHourProviderMajortooling
HBASE-24942MergeTableRegionsProcedure should not call clean merge regionMajorproc-v2, Region Assignment
HBASE-24811Use class access static field or methodMinor.
HBASE-24686[LOG] Log improvement in Connection#closeMajorClient, logging
HBASE-24912Enlarge MemstoreFlusherChore/CompactionChecker period for unit testMajor.
HBASE-24627Normalize one table at a timeMajorNormalizer
HBASE-24872refactor valueOf PoolTypeMinorClient
HBASE-24854Correct the help content of assign and unassign commands in hbase shellMinorshell
HBASE-24750All executor service should start using guava ThreadFactoryMajor.
HBASE-24709Support MoveCostFunction use a lower multiplier in offpeak hoursMajorBalancer
HBASE-24824Add more stats in PE for read replicaMinorPE, read replicas
HBASE-21721FSHLog : reduce write#syncs() timesMajor.
HBASE-24404Support flush a single column family of regionMajorshell
HBASE-24826Add some comments for processlist in hbase shellMinorshell
HBASE-24659Calculate FIXED_OVERHEAD automaticallyMajor.
HBASE-24827BackPort HBASE-11554 Remove Reusable poolmap Rpc client type.MajorClient
HBASE-24823Port HBASE-22762 Print the delta between phases in the split/merge/compact/flush transaction journals to master branchMinor.
HBASE-24795RegionMover should deal with unknown (split/merged) regionsMajor.
HBASE-24821Simplify the logic of getRegionInfo in TestFlushFromClient to reduce redundancy codeMinortest
HBASE-24704Make the Table Schema easier to view even there are multiple familiesMajorUI
HBASE-24695FSHLog - close the current WAL file in a background threadMajor.
HBASE-24803Unify hbase-shell ::Shell::Commands::Command#help behaviorMinorshell
HBASE-11686Shell code should create a binding / irb workspace instead of polluting the root namespaceMinorshell
HBASE-20226Performance Improvement Taking Large Snapshots In Remote FilesystemsMinorsnapshots
HBASE-24669Logging of ppid should be consistent across all occurrencesMinorOperability, proc-v2
HBASE-24757ReplicationSink should limit the batch rowcount for batch mutations based on hbase.rpc.rows.warning.thresholdMajor.
HBASE-24777InfoServer support ipv6 host and portMinorUI
HBASE-24758Avoid flooding replication source RSes logs when no sinks are availableMajorReplication
HBASE-24743Reject to add a peer which replicate to itself earlierMajor.
HBASE-24696Include JVM information on Web UI under “Software Attributes”MinorUI
HBASE-24747Log an ERROR if HBaseSaslRpcServer initialisation fails with an uncaught exceptionMajor.
HBASE-24586Add table level locality in table.jspMajorUI
HBASE-24663Add procedure process time statistics UIMajor.
HBASE-24653Show snapshot owner on Master WebUIMajor.
HBASE-24431RSGroupInfo add configuration map to store something extraMajorrsgroup
HBASE-24671Add excludefile and designatedfile options to graceful_stop.shMajor.
HBASE-24560Add a new option of designatedfile in RegionMoverMajor.
HBASE-24382Flush partial stores of region filtered by seqId when archive wal due to too many walsMajorwal
HBASE-24208Remove RS entry from zk draining servers node after RS been stoppedMajor.
HBASE-24456Immutable Scan as unmodifiable subclass or wrapper of ScanMajor.
HBASE-24471The way we bootstrap meta table is confusingMajormaster, meta, proc-v2
HBASE-24350HBase table level replication metrics for shippedBytes are always 0MajorReplication
HBASE-24311Add more details in MultiVersionConcurrencyControl STUCK log messageMajor.

BUG FIXES:

JIRASummaryPriorityComponent
HBASE-25355[Documentation] Fix spelling errorTrivialdocumentation
HBASE-25230Embedded zookeeper server not clean up the old dataMinorZookeeper
HBASE-25349[Flakey Tests] branch-2 TestRefreshRecoveredReplication.testReplicationRefreshSource:141 Waiting timed out after [60,000] msecMajorflakies
HBASE-25332one NPEMajorZookeeper
HBASE-25345[Flakey Tests] branch-2 TestReadReplicas#testVerifySecondaryAbilityToReadWithOnFilesMajortest
HBASE-25307ThreadLocal pooling leads to NullPointerExceptionMajorClient
HBASE-25341Fix ErrorProne error which causes nightly to failMajortest
HBASE-25330RSGroupInfoManagerImpl#moveServers return is not set of servers movedMajorrsgroup
HBASE-25321The sort icons not shown after Upgrade JQuery to 3.5.1MajorUI
HBASE-24268REST and Thrift server do not handle the “doAs” parameter case insensitivelyMinorREST, Thrift
HBASE-25050We initialize Filesystems more than once.Minor.
HBASE-25311ui throws NPEMajor.
HBASE-25306The log in SimpleLoadBalancer#onConfigurationChange is wrongMajor.
HBASE-25300‘Unknown table hbase:quota’ happens when desc table in shell if quota disabledMajorshell
HBASE-25255Master fails to initialize when creating rs group tableCriticalmaster, rsgroup
HBASE-25275Upgrade asciidoctorBlockerwebsite
HBASE-25276Need to throw the original exception in HRegion#openHRegionMajor.
HBASE-20598Upgrade to JRuby 9.2Majordependencies, shell
HBASE-25216The client zk syncer should deal with meta replica count changeMajormaster, Zookeeper
HBASE-25238Upgrading HBase from 2.2.0 to 2.3.x fails because of “Message missing required fields: state”Critical.
HBASE-25234[Upgrade]Incompatibility in reading RS report from 2.1 RS when Master is upgraded to a version containing HBASE-21406Major.
HBASE-25053WAL replay should ignore 0-length filesMajormaster, regionserver
HBASE-25090CompactionConfiguration logs unrealistic store file sizesMinorCompaction
HBASE-24977Meta table shouldn't be modified as read onlyMajormeta
HBASE-25176MasterStoppedException should be moved to hbase-client moduleMajorClient
HBASE-25206Data loss can happen if a cloned table loses original split region(delete table)Majorproc-v2, Region Assignment, snapshots
HBASE-25207Revisit the implementation and usage of RegionStates.includeMajorRegion Assignment
HBASE-25186TestMasterRegionOnTwoFileSystems is failing after HBASE-25065Blockermaster
HBASE-25204Nightly job failed as the name of jdk and maven changedMajor.
HBASE-25093the RSGroupBasedLoadBalancer#retainAssignment throws NPEMajorrsgroup
HBASE-25117ReplicationSourceShipper thread can not be finishedMajor.
HBASE-25168Unify WAL name timestamp parsersMajor.
HBASE-23834HBase fails to run on Hadoop 3.3.0/3.2.2/3.1.4 due to jetty version mismatchMajordependencies
HBASE-25165Change ‘State time’ in UI so sortsMinorUI
HBASE-25048[HBCK2] Bypassed parent procedures are not updated in storeMajorhbck2, proc-v2
HBASE-25147Should store the regionNames field in state data for ReopenTableRegionsProcedureMajorproc-v2
HBASE-25115HFilePrettyPrinter can't seek to the row which is the first row of a hfileMajorHFile, tooling
HBASE-25135Convert the internal seperator while emitting the memstore read metrics to #Minor.
HBASE-24665MultiWAL : Avoid rolling of ALL WALs when one of the WAL needs a rollMajorwal
HBASE-25096WAL size in RegionServer UI is wrongMajor.
HBASE-25077hbck.jsp page loading fails, logs NPE in master log.Majorhbck
HBASE-25088CatalogFamilyFormat/MetaTableAccessor.parseRegionInfoFromRegionName incorrectly setEndKey to regionIdCriticalmeta
HBASE-25097Wrong RIT page number in Master UIMinorUI
HBASE-24896‘Stuck’ in static initialization creating RegionInfo instanceMajor.
HBASE-24956ConnectionManager#locateRegionInMeta waits for user region lock indefinitely.MajorClient
HBASE-24481HBase Rest: Request for region detail of a table which doesn't exits is success(200 success code) instead of 404Minor.
HBASE-25047WAL split edits number is negative in RegionServerUIMinorUI, wal
HBASE-25021Nightly job should skip hadoop-2 integration test for masterMajorbuild, scripts
HBASE-25012HBASE-24359 causes replication missed log of some RemoteExceptionMajorReplication
HBASE-25009Hbck chore logs wrong message when loading regions from RS reportMinor.
HBASE-25014ScheduledChore is never triggered when initalDelay > 1.5*periodMajor.
HBASE-25016Should close ResultScanner in MetaTableAccessor.scanByRegionEncodedNameCriticalmaster, meta
HBASE-24958CompactingMemStore.timeOfOldestEdit error updateCriticalregionserver
HBASE-24995MetaFixer fails to fix overlaps when multiple tables have overlapsMajorhbck2
HBASE-24719Renaming invalid rsgroup throws NPE instead of proper error messageMajor.
HBASE-19352Port HADOOP-10379: Protect authentication cookies with the HttpOnly and Secure flagsMajor.
HBASE-24971Upgrade JQuery to 3.5.1Majorsecurity, UI
HBASE-24968One of static initializers of CellComparatorImpl referring to subclass MetaCellComparatorMajor.
HBASE-24916Region hole contains wrong regions pair when hole is created by first region deletionMajorhbck2
HBASE-24885STUCK RIT by hbck2 assignsMajorhbck2, Region Assignment
HBASE-24926Should call setFailure in MergeTableRegionsProcedure when isMergeable returns falseMajormaster, proc-v2
HBASE-24884BulkLoadHFilesTool/LoadIncrementalHFiles should accept -D options from command line parametersMinor.
HBASE-24583Normalizer can't actually merge empty regions when neighbor is larger than average sizeMajormaster, Normalizer
HBASE-24844Exception on standalone (master) shutdownMinorZookeeper
HBASE-24856Fix error prone error in FlushTableSubprocedureMajor.
HBASE-24838The pre commit job fails to archive surefire reportsCriticalbuild, scripts
HBASE-23157WAL unflushed seqId tracking may wrong when Durability.ASYNC_WAL is usedMajorregionserver, wal
HBASE-24625AsyncFSWAL.getLogFileSizeIfBeingWritten does not return the expected synced file length.CriticalReplication, wal
HBASE-24830Some tests involving RS crash fail with NullPointerException after HBASE-24632 in branch-2Major.
HBASE-24788Fix the connection leaks on getting hbase admin from unclosed connectionMajormapreduce
HBASE-24805HBaseTestingUtility.getConnection should be threadsafeMajortest
HBASE-24808skip empty log cleaner delegate class names (WAS => cleaner.CleanerChore: Can NOT create CleanerDelegate= ClassNotFoundException)Trivial.
HBASE-24767Change default to false for HBASE-15519 per-user metricsMajormetrics
HBASE-24713RS startup with FSHLog throws NPE after HBASE-21751Minorwal
HBASE-24794hbase.rowlock.wait.duration should not be <= 0Minorregionserver
HBASE-24797Move log code out of loopMinorNormalizer
HBASE-24752NPE/500 accessing webui on master startupMinormaster
HBASE-24766Document Remote Procedure ExecutionMajordocumentation
HBASE-11676Scan FORMATTER is not applied for columns using non-printable name in shellMinorshell
HBASE-24738[Shell] processlist command fails with ERROR: Unexpected end of file from server when SSL enabledMajorshell
HBASE-24675On Master restart all servers are assigned to default rsgroup.Majorrsgroup
HBASE-22146SpaceQuotaViolationPolicy Disable is not working in Namespace levelMajor.
HBASE-24742Improve performance of SKIP vs SEEK logicMajorPerformance, regionserver
HBASE-24710Incorrect checksum calculation in saveVersion.shMajorscripts
HBASE-24714Error message is displayed in the UI of table's compaction state if any region of that table is not open.MajorCompaction, UI
HBASE-24748Add hbase.master.balancer.stochastic.moveCost.offpeak to doc as support dynamically changeMinordocumentation
HBASE-24746The sort icons overlap the col name in master UIMajorUI
HBASE-24721rename_rsgroup overwriting the existing rsgroup.Major.
HBASE-24615MutableRangeHistogram#updateSnapshotRangeMetrics doesn't calculate the distribution for last bucket.Majormetrics
HBASE-24705MetaFixer#fixHoles() does not include the case for read replicas (i.e, replica regions are not created)Majorread replicas
HBASE-24720Meta replicas not cleaned when disabledMinorread replicas
HBASE-24693regioninfo#isLast() has a logic errorMinor.
HBASE-23744FastPathBalancedQueueRpcExecutor should enforce queue length of 0Minor.
HBASE-22738Fallback to default group to choose RS when there are no RS in current groupMajorrsgroup
HBASE-23126IntegrationTestRSGroup is useless nowMajorrsgroup
HBASE-24518waitForNamespaceOnline() should return false if any region is offlineMajor.
HBASE-24564Make RS abort call idempotentMajorregionserver
HBASE-24340PerformanceEvaluation options should not mandate any specific orderMinor.
HBASE-24130rat plugin complains about having an unlicensed file.Minor.
HBASE-24017Turn down flakey rerun rate on all but hot branchesMajor.

TESTS:

JIRASummaryPriorityComponent
HBASE-24015Coverage for Assign and Unassign of Regions on RegionServer on failureMajoramv2
HBASE-25156TestMasterFailover.testSimpleMasterFailover is flakyMajortest
HBASE-24979Include batch mutatations in client operation timeout testsMajor.
HBASE-24894[Flakey Test] TestStochasticLoadBalancer.testMoveCostMultiplierMajorBalancer, master, test

SUB-TASKS:

JIRASummaryPriorityComponent
HBASE-25127Enhance PerformanceEvaluation to profile meta replica performance.Major.
HBASE-25284Check-in “Enable memstore replication...” designMajor.
HBASE-25126Add load balance logic in hbase-client to distribute read load over meta replica regions.Major.
HBASE-25291Document how to enable the meta replica load balance mode for the client and clean up around hbase:meta read replicasMajor.
HBASE-25253Deprecated master carrys regions related methods and configsMajorBalancer, master
HBASE-25203Change the reference url to flaky list in our jenkins jobsMajorflakies, jenkins
HBASE-25194Do not publish workspace in flaky find jobMajorjenkins
HBASE-25169Update documentation about meta region replicaMajordocumentation
HBASE-25164Make ModifyTableProcedure support changing meta replica countMajormeta, read replicas
HBASE-25162Make flaky tests run more aggressivelyMajorjenkins, scripts, test
HBASE-25163Increase the timeout value for nightly jobsMajorjenkins, scripts, test
HBASE-22976[HBCK2] Add RecoveredEditsPlayerMajorhbck2, walplayer
HBASE-25124Support changing region replica count without disabling tableMajormeta, proc-v2
HBASE-23959Fix javadoc for JDK11Major.
HBASE-25151warmupRegion frustrates registering WALs on the catalog replicationsourceMajorread replicas
HBASE-25154Set java.io.tmpdir to project build directory to avoid writing std*deferred files to /tmpMajorbuild, test
HBASE-25121Refactor MetaTableAccessor.addRegionsToMeta and its usage placesMajormeta
HBASE-25055Add ReplicationSource for meta WALs; add enable/disable when hbase:meta assigned to RSMajor.
HBASE-25133Migrate HBase Nightly jenkins job from Hadoop to hbaseMajorjenkins, scripts
HBASE-25132Migrate flaky test jenkins job from Hadoop to hbaseMajorjenkins, scripts
HBASE-25103Remove ZNodePaths.metaReplicaZNodesMajor.
HBASE-25107Migrate flaky reporting jenkins job from Hadoop to hbaseMajorjenkins, scripts
HBASE-25068Pass WALFactory to Replication so it knows of all WALProviders, not just default/user-spaceMinor.
HBASE-25067Edit of log messages around async WAL Replication; checkstyle fixes; and a bugfixMajor.
HBASE-24857Fix several problems when starting webUIMinorcanary, UI
HBASE-24964Remove MetaTableAccessor.tableExistsMajormeta
HBASE-24765Dynamic master discoveryMajorClient
HBASE-24945Remove MetaTableAccessor.getRegionCountMajormapreduce, meta
HBASE-24944Remove MetaTableAccessor.getTableRegionsAndLocations in hbase-rest moduleMajormeta, REST
HBASE-24918Make RegionInfo#UNDEFINED IA.PrivateMajor.
HBASE-24806Small Updates to Functionality of Shell IRB WorkspaceMajorshell
HBASE-24876Fix the flaky job url in hbase-personality.shMajor.
HBASE-24841Change the jenkins job urls in our jenkinsfileMajorbuild, scripts
HBASE-24680Refactor the checkAndMutate code on the server sideMajor.
HBASE-24817Allow configuring WALEntry filters on ReplicationSourceMajorReplication, wal
HBASE-24632Enable procedure-based log splitting as default in hbase3Majorwal
HBASE-24718Generic NamedQueue framework for recent in-memory history (refactor slowlog)Major.
HBASE-24698Turn OFF Canary WebUI as defaultMajorcanary
HBASE-24650Change the return types of the new checkAndMutate methods introduced in HBASE-8458MajorClient
HBASE-24013Bump branch-2 version to 2.4.0-SNAPSHOTMajor.

OTHER:

JIRASummaryPriorityComponent
HBASE-25099Change meta replica count by altering meta table descriptorMajormeta, read replicas
HBASE-25320Upgrade hbase-thirdparty dependency to 3.4.1Blockerdependencies
HBASE-24640Purge use of VisibleForTestingMajorcommunity
HBASE-24081Provide documentation for running Yetus with HBaseMajordocumentation
HBASE-24667Rename configs that support atypical DNS set ups to put them in hbase.unsafeMajorconf, Operability
HBASE-25228Delete dev-support/jenkins_precommit_jira_yetus.shMinorbuild
HBASE-24200Upgrade to Yetus 0.12.0Minorbuild
HBASE-25120Remove the deprecated annotation for MetaTableAccessor.getScanForTableNameMajormeta
HBASE-25073Should not use XXXService.Interface.class.getSimpleName as stub key prefix in AsyncConnectionImplMajorClient
HBASE-25072Remove the unnecessary System.out.println in MasterRegistryMinorClient
HBASE-25004Log RegionTooBusyException detailsMajor.
HBASE-24993Remove OfflineMetaRebuildTestCoreMajortest
HBASE-24809Yetus IA Javadoc links are no longer availableMinor.
HBASE-14847Add FIFO compaction section to HBase bookMajordocumentation
HBASE-24843Sort the constants in `hbase_constants.rb`Minorshell
HBASE-24835Normalizer should log a successful run at INFO levelMinorNormalizer
HBASE-24779Improve insight into replication WAL readers hung on checkQuotaMinorReplication
HBASE-24662Update DumpClusterStatusAction to notice changes in region server countMajorintegration tests
HBASE-24658Update PolicyBasedChaosMonkey to handle uncaught exceptionsMinorintegration tests
HBASE-24648Remove the legacy ‘forceSplit’ related code at region server sideMajorregionserver
HBASE-24492ProtobufLogReader.readNext does not need loopingMinorReplication, wal
HBASE-22033Update to maven-javadoc-plugin 3.2.0 and switch to non-forking aggregate goalsMajorbuild, website

Release 2.3.0

CHANGES.md at rel/2.3.0 (e0e1382)

Release 2.2.0

CHANGES.md at rel/2.2.0 (3ec6932)

Release 2.1.0

CHANGES.md at rel/2.1.0 (e1673bb)

Release 2.0.0

CHANGES.md at rel/2.0.0 (7483b11)

Release 1.0.0

CHANGES.txt at rel/1.0.0 (6c98bff)