[jira] [Created] (HBASE-28697) Incremental backups delete bulk loaded system table rows too early
Ray Mattingly created HBASE-28697: - Summary: Incremental backups delete bulk loaded system table rows too early Key: HBASE-28697 URL: https://issues.apache.org/jira/browse/HBASE-28697 Project: HBase Issue Type: Bug Affects Versions: 2.6.0 Reporter: Ray Mattingly I've been thinking through the incremental backup order of operations, and I think we delete rows from the bulk loads system table too early and, consequently, make it possible to produce a "successful" incremental backup that is missing bulk loads. To summarize the steps here, starting in {{{}IncrementalTableBackupCilent#execute{}}}: # We take an incremental backup of the WALs generated since the last backup # We ensure any bulk loads done since the last backup are appropriately represented in the new backup by going through the system table and copying the appropriate files to the backup directory # We delete all of the system table rows which told us about these bulk loads # We generate a backup manifest and mark the backup as complete If we began deleting any of the system table rows regarding bulk loads, but fail in steps 3 and 4 before we are able to mark the backup as complete, then we'll be in a precarious spot. If we retry an incremental backup then it may succeed, but it would not know to persist the bulk loaded files for which we have already deleted system table references. We could consider this issue an extension or replacement of https://issues.apache.org/jira/browse/HBASE-28084 in some ways, depending on what solution we land on. I think that we could fix this specific issue by reordering the bulk load table cleanup, but there will always be gotchas like this. Maybe it is simpler to require that the next backup be a full backup after any incremental failure. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28696) BackupSystemTable can create huge delete batches that should be partitioned instead
Ray Mattingly created HBASE-28696: - Summary: BackupSystemTable can create huge delete batches that should be partitioned instead Key: HBASE-28696 URL: https://issues.apache.org/jira/browse/HBASE-28696 Project: HBase Issue Type: Bug Reporter: Ray Mattingly When successfully taking an incremental backup, one of our final steps is to delete bulk load metadata from the system table for the bulk loads that needed to be captured in the given backup. This means that we will basically truncate the entire bulk loads system table in a single batch of the deletes after successfully taking an incremental backup. This logic occurs in {{{}BackupSystemTable#deleteBulkLoadedRows{}}}: {code:java} /* * Removes rows recording bulk loaded hfiles from backup table * @param lst list of table names * @param rows the rows to be deleted */ public void deleteBulkLoadedRows(List rows) throws IOException { try (Table table = connection.getTable(bulkLoadTableName)) { List lstDels = new ArrayList<>(); for (byte[] row : rows) { Delete del = new Delete(row); lstDels.add(del); LOG.debug("orig deleting the row: " + Bytes.toString(row)); } table.delete(lstDels); LOG.debug("deleted " + rows.size() + " original bulkload rows"); } } {code} Depending on your usage, one may run tons of bulk loads between backups, so this design is needlessly fragile. We should partition these deletes so that we never erroneously fail a backup due to this. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28695) Quotas should support hard limits
Ray Mattingly created HBASE-28695: - Summary: Quotas should support hard limits Key: HBASE-28695 URL: https://issues.apache.org/jira/browse/HBASE-28695 Project: HBase Issue Type: Improvement Reporter: Ray Mattingly Quotas only support soft limits. Previously, some limits have accidentally been overly strict in a way that caused difficult UX, and we made them more lenient in https://issues.apache.org/jira/browse/HBASE-28672 It would be nice if quotas allowed users to specify hard throughput limits that they would certainly not exceed. This would be simple for read/write/request number quotas, but would be more difficult for read size throttling where workload estimations are much less precise. It's possible that this can/should only be supported for some combination of read number, write number, request number, and write size throttles. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28687) BackupSystemTable#checkSystemTable should ensure that the backup system tables are enabled
Ray Mattingly created HBASE-28687: - Summary: BackupSystemTable#checkSystemTable should ensure that the backup system tables are enabled Key: HBASE-28687 URL: https://issues.apache.org/jira/browse/HBASE-28687 Project: HBase Issue Type: Improvement Affects Versions: 2.6.0 Reporter: Ray Mattingly If the backup system tables become disabled, then we enter a state which the backup client will not recover from. Without manual intervention, every subsequent backup attempt will fail on [BackupSystemTable's calls to waitForSystemTable|https://github.com/apache/hbase/blob/3a3dd66e21da3f85c72d75605857713716d579fb/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java#L214-L215]. This checkSystemTable method already ensures that the tables exist — it should also ensure that the tables are enabled before we await that condition. Alternatively, we could fast-fail if the tables are disabled rather than awaiting an enabled state that will never occur. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28686) MapReduceBackupCopyJob should support custom DistCp options
Ray Mattingly created HBASE-28686: - Summary: MapReduceBackupCopyJob should support custom DistCp options Key: HBASE-28686 URL: https://issues.apache.org/jira/browse/HBASE-28686 Project: HBase Issue Type: Improvement Affects Versions: 2.6.0 Reporter: Ray Mattingly h4. Problem The MapReduceBackupCopyJob class provides no means for updating DistCp job options. This means that you're stuck with defaults, which isn't always desirable. For example, my workplace would like the freedom to deviate from at least two DistCp defaults: # distcp.direct.write — we would like to set this to true, because writing and renaming tmp files is expensive in S3 (where we store our backups). # we would also like control over the number of mappers that DistCp will run h4. Proposed Solution It is not the prettiest solution, but I'm proposing that we support DistCp customizations via the given backup client configuration like [this.|https://github.com/HubSpot/hbase/compare/hubspot-2.6...HubSpot:hbase:backup-distcp-options] It's necessary to do this conf -> arg conversion because we still want to use [DistCp's run method|https://github.com/HubSpot/hadoop/blob/c4c25b0ea2be1c8bca31d86962597060b2630f62/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java#L134-L171], which expects args, so as to not change any error codes. Hadoop actually does something similar, but in the opposite direction — the DistCp job has logic to convert the args back to configurations (lol). Further, the DistCp API is really unfortunately designed for programmatic use, so it doesn't leave us great alternatives. For example, it doesn't matter what you pass in as DistCpOptions to the constructor if you use the run method, your options will be overwritten based on the args that you pass in. Alternatively, if you pass in the DistCpOptions in the constructor and use DistCp#execute or DistCp#createAndSubmitJob, then you get none of the error specificity! -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28672) Large batch requests can be blocked indefinitely by quotas
Ray Mattingly created HBASE-28672: - Summary: Large batch requests can be blocked indefinitely by quotas Key: HBASE-28672 URL: https://issues.apache.org/jira/browse/HBASE-28672 Project: HBase Issue Type: Improvement Affects Versions: 2.6.0 Reporter: Ray Mattingly At my day job we are trying to implement default quotas for a variety of access patterns. We began by introducing a default read IO limit per-user, per-machine — this has been very successful in reducing hotspots, even on clusters with thousands of distinct users. While implementing a default writes/second throttle, I realized that doing so would put us in a precarious situation where large-enough batches may never succeed. If your batch size is greater than your TimeLimiter's max throughput, then you will always fail in the quota estimation stage. Meanwhile [IO estimates are more optimistic|https://github.com/apache/hbase/blob/bdb3f216e864e20eb2b09352707a751a5cf7460f/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/DefaultOperationQuota.java#L192-L193], deliberately, which can let large requests do targeted oversubscription of an IO quota: {code:java} // assume 1 block required for reads. this is probably a low estimate, which is okay readConsumed = numReads > 0 ? blockSizeBytes : 0;{code} This is okay because the Limiter's availability will go negative and force a longer backoff on subsequent requests. I believe this is preferable UX compared to a doomed throttling loop. In my opinion, we should do something similar in batch request estimation, by estimating a batch request's workload at {{Math.min(batchSize, limiterMaxThroughput)}} rather than simply {{{}batchSize{}}}. -- This message was sent by Atlassian Jira (v8.20.10#820010)
Re: [DISCUSS] Blockers for 2.6.1
I found the bug documented in `HBASE-28643: An unbounded backup failure message can cause an irrecoverable state for the given backup` yesterday, and I think we should probably aim to fix this in the 2.6.1 release. Otherwise that list looks good to me. On Wed, Jun 5, 2024 at 3:35 PM Bryan Beaudreault wrote: > Hey all, > > It's been 2 weeks since 2.6.0 was released. As discussed in the vote > thread, there were a few outstanding backup-related issues. I believe we've > made some progress on some of those. > > I'd like to start compiling a list of important backup-related fixes to > target for the 2.6.1 release so that we can track progress. Can those of > you who are involved (Ray Mattingly, Nick Dimiduck, Dieter De Paepe & team, > and any others) please list any important jiras here? > > With a list of jiras in hand, we can check to make sure blockers & > fixVersions are set and use that to track what we need to drill down before > releasing. > > Here's what I know of so far, let me know if I'm missing anything: > > Not yet started: > - HBASE-28084: incremental backups should be forbidden after deleting > backups > - HBASE-28602: Incremental backup fails when WALs move > - HBASE-28462: (similar to ^, but in a different part of the code) > - HBASE-28538: BackupHFileCleaner is very expensive > > Patch available: > - HBASE-28539: backup merging does not work when using cloud storage as > filesystem > - HBASE-28562: another possible failure cause for incremental backups + > possibly cause of overly big backup metadata > > Resolved: > - HBASE-28502: backed up tables are not listed correctly in backup > metadata, which causes unreliable backup validation > - HBASE-28568: the set of tables included in incremental backups might be > too big >
[jira] [Created] (HBASE-28643) An unbounded backup failure message can cause an irrecoverable state for the given backup
Ray Mattingly created HBASE-28643: - Summary: An unbounded backup failure message can cause an irrecoverable state for the given backup Key: HBASE-28643 URL: https://issues.apache.org/jira/browse/HBASE-28643 Project: HBase Issue Type: Bug Affects Versions: 2.6.0 Reporter: Ray Mattingly The BackupInfo class has a failedMsg field which is a string of unbounded length. When a DistCp job fails then its failure message contains all of its source paths, and its failure message gets propagated to this failedMsg field on the given BackupInfo. If a DistCp job has enough source paths, then this will result in backup status updates being rejected: {noformat} java.lang.IllegalArgumentException: KeyValue size too large at org.apache.hadoop.hbase.client.ConnectionUtils.validatePut(ConnectionUtils.java:513) at org.apache.hadoop.hbase.client.HTable.validatePut(HTable.java:1095) at org.apache.hadoop.hbase.client.HTable.lambda$put$3(HTable.java:564) at org.apache.hadoop.hbase.trace.TraceUtil.trace(TraceUtil.java:187) at org.apache.hadoop.hbase.client.HTable.put(HTable.java:563) at org.apache.hadoop.hbase.backup.impl.BackupSystemTable.updateBackupInfo(BackupSystemTable.java:292) at org.apache.hadoop.hbase.backup.impl.BackupManager.updateBackupInfo(BackupManager.java:376) at org.apache.hadoop.hbase.backup.impl.TableBackupClient.failBackup(TableBackupClient.java:243) at org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient.execute(IncrementalTableBackupClient.java:317) at org.apache.hadoop.hbase.backup.impl.BackupAdminImpl.backupTables(BackupAdminImpl.java:603) at com.hubspot.hbase.recovery.core.backup.BackupManager.lambda$runBackups$2(BackupManager.java:145){noformat} Without the ability to update the backup's state, it will never be returned as a failed backup by the client. This means that any mechanisms designed for repairing or cleaning failed backups won't work properly. I think that a simple fix here would be fine: we should truncate the failedMsg field to a reasonable maximum size. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28513) Secondary replica balancing squashes all other cost considerations
Ray Mattingly created HBASE-28513: - Summary: Secondary replica balancing squashes all other cost considerations Key: HBASE-28513 URL: https://issues.apache.org/jira/browse/HBASE-28513 Project: HBase Issue Type: Improvement Reporter: Ray Mattingly I have a larger write up available [here.|https://git.hubteam.com/gist/rmattingly/8bc9cbe7c422db12ffc9cd1825069bd7] Basically there are a few cost functions with relatively huge default multipliers. For example `PrimaryRegionCountSkewCostFunction` has a default multiplier of 100,000. Meanwhile things like StoreFileCostFunction have a multiplier of 5. Having any multiplier of 100k, while others are single digit, basically makes the latter category totally irrelevant from balancer considerations. I understand that it's critical to distribute a region's replicas across multiple hosts/racks, but I don't think we should do this at the expense of all other balancer considerations. For example, maybe we could have two types of balancer considerations: costs (as we do now), and conditionals (for the more discrete considerations, like ">1 replica of the same region should not exist on a single host"). This would allow us to prioritize replica distribution _and_ maintain consideration for things like storefile balance. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Resolved] (HBASE-28429) Quotas should have a configurable minimum wait interval
[ https://issues.apache.org/jira/browse/HBASE-28429?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ray Mattingly resolved HBASE-28429. --- Resolution: Won't Fix https://issues.apache.org/jira/browse/HBASE-28453 is a better solution > Quotas should have a configurable minimum wait interval > --- > > Key: HBASE-28429 > URL: https://issues.apache.org/jira/browse/HBASE-28429 > Project: HBase > Issue Type: Improvement > Reporter: Ray Mattingly >Assignee: Ray Mattingly >Priority: Major > > At my day job we're attempting to rollout read size throttling by default for > thousands of distinct users across hundreds of multi-tenant clusters. > During our rollout we've observed that throttles with a 1 second refill > interval will yield relatively tiny wait intervals disproportionately often. > From what we've seen, wait intervals are <=5ms on approximately 20-50% of our > RpcThrottlingExceptions; this could sound theoretically promising if latency > is your top priority. But, in reality, this makes it very difficult to > configure a throttle tolerant HBase client because retries become very prone > to near-immediate exhaustion, and throttled clients quickly saturate the > cluster's RPC layer with rapid-fire retries. > One can combat this with the FixedIntervalRateLimiter, but that's a very > heavy handed approach from latency's perspective, and can still yield tiny > intervals that exhaust retries and erroneously fail client operations under > significant load. > With this in mind, I'm proposing that we introduce a configurable minimum > wait interval for quotas, defaulted to 0. This would make quotas much more > usable at scale from our perspective. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28453) Support a middle ground between the Average and Fixed interval rate limiters
Ray Mattingly created HBASE-28453: - Summary: Support a middle ground between the Average and Fixed interval rate limiters Key: HBASE-28453 URL: https://issues.apache.org/jira/browse/HBASE-28453 Project: HBase Issue Type: Improvement Affects Versions: 2.6.0 Reporter: Ray Mattingly Attachments: Screenshot 2024-03-21 at 2.08.51 PM.png, Screenshot 2024-03-21 at 2.30.01 PM.png h3. Background HBase quotas support two rate limiters: a "fixed" and an "average" interval rate limiter. h4. FixedIntervalRateLimiter The fixed interval rate limiter is simpler: it has a TimeUnit, say 1 second, and it refills a resource allotment on the recurring interval. So you may get 10 resources every second, and if you exhaust all 10 resources in the first millisecond of an interval then you will need to wait 999ms to acquire even 1 more resource. h4. AverageIntervalRateLimiter The average interval rate limiter, HBase's default, allows for more flexibly timed refilling of the resource allotment. Extending our previous example, say you have a 10 reads/sec quota and you have exhausted all 10 resources within 1ms of the last full refill. If you request 1 more read then, rather than returning a 999ms wait interval indicating the next full refill time, the rate limiter will recognize that you only need to wait 99ms before 1 read can be available. After 100ms has passed in aggregate since the last full refill, it will support the refilling of 1/10th the limit to facilitate the request for 1/10th the resources. h3. The Problems with Current RateLimiters The problem with the fixed interval rate limiter is that it is too strict from a latency perspective. It results in quota limits to which we cannot fully subscribe with any consistency. The problem with the average interval rate limiter is that, in practice, it is far too optimistic. For example, a real rate limiter might limit to 100MB/sec of read IO per machine. Any multigets that come in will require only a tiny fraction of this limit; for example, a 64kb block is only 0.06% of the total. As a result, the vast majority of wait intervals end up being tiny — like <5ms. This can actually cause an inverse of your intention, where setting up a throttle causes a DDOS of your RPC layer via continuous throttling and ~immediate retrying. I've discussed this problem in https://issues.apache.org/jira/browse/HBASE-28429 and proposed a minimum wait interval as the solution there; after some more thinking, I believe this new rate limiter would be a less hacky solution to this deficit so I'd like to close that Jira in favor of this one. See the attached chart where I put in place a 10k req/sec/machine throttle for this user at 10:43 to try to curb this high traffic, and it resulted in a huge spike of req/sec due to the throttle/retry loop created by the AverageIntervalRateLimiter. h3. PartialIntervalRateLimiter as a Solution I've implemented a RateLimiter which allows for partial chunks of the overall interval to be refilled, by default these chunks are 10% (or 100ms of a 1s interval). I've deployed this to a test cluster at my day job and have seen this really help our ability to full subscribe to a quota limit without executing superfluous retries. See the other attached chart which shows a cluster undergoing a rolling restart from using FixedIntervalRateLimiter to my new PartialIntervalRateLimiter and how it is then able to fully subscribe to its allotted 25MB/sec/machine read IO quota. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28430) RpcThrottlingException messages should describe the throttled access pattern
Ray Mattingly created HBASE-28430: - Summary: RpcThrottlingException messages should describe the throttled access pattern Key: HBASE-28430 URL: https://issues.apache.org/jira/browse/HBASE-28430 Project: HBase Issue Type: Improvement Reporter: Ray Mattingly Right now we catch RpcThrottlingExceptions and have some debug logging in [RegionServerRpcQuotaManager|https://github.com/apache/hbase/blob/98eb3e01b352684de3c647a6fda6208a657c4607/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java#L234-L236] — this is okay support for in depth explanations, but is not trivially transparent to users. For example, at my day job we have proxy APIs which sit between HBase and many microservices. We throttle these microservices in isolation, but the RpcThrottlingExceptions appear to be indiscriminate in the stdout of the proxy API. If we added the given username, table, and namespace to RpcThrottlingException messages then understanding the nature and specificity of any given throttle violation should be much more straightforward. Given that quotas/throttling is most useful in a multi-tenant environment, I would anticipate this being a pretty universal usability pain point. It would be a bit more complicated, but we should also consider including more information about the rate limiter which has been violated. For example, what is the current configured read size limit that we've exceeded? -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28429) Quotas should have a configurable minimum wait interval
Ray Mattingly created HBASE-28429: - Summary: Quotas should have a configurable minimum wait interval Key: HBASE-28429 URL: https://issues.apache.org/jira/browse/HBASE-28429 Project: HBase Issue Type: Improvement Reporter: Ray Mattingly At my day job we're attempting to rollout read size throttling by default for thousands of distinct users across hundreds of multi-tenant clusters. During our rollout we've observed that throttles with a 1 second refill interval will yield relatively tiny wait intervals disproportionately often. From what we've seen, wait intervals are <=5ms on approximately 20-50% of our RpcThrottlingExceptions; this could sound theoretically promising if latency is your top priority. But, in reality, this makes it very difficult to configure a throttle tolerant HBase client because retries become very prone to near-immediate exhaustion, and throttled clients quickly saturate the cluster's RPC layer with rapid-fire retries. One can combat this with the FixedIntervalRateLimiter, but that's a very heavy handed approach from latency's perspective, and can still yield tiny intervals that exhaust retries and erroneously fail client operations under significant load. With this in mind, I'm proposing that we introduce a configurable minimum wait interval for quotas, defaulted to 0. This would make quotas much more usable at scale from our perspective. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28385) Quota estimates are too optimistic for large scans
Ray Mattingly created HBASE-28385: - Summary: Quota estimates are too optimistic for large scans Key: HBASE-28385 URL: https://issues.apache.org/jira/browse/HBASE-28385 Project: HBase Issue Type: Improvement Reporter: Ray Mattingly Fix For: 2.6.0 Let's say you're running a table scan with a throttle of 100MB/sec per RegionServer. Ideally your scans are going to pull down large results, often containing hundreds or thousands of blocks. You will estimate each scan as costing a single block of read capacity, and if your quota is already exhausted then the server will evaluate the backoff required for your estimated consumption (1 block) to be available. This will often be ~1ms, causing your retries to basically be immediate. Obviously it will routinely take much longer than 1ms for 100MB of IO to become available in the given configuration, so your retries will be destined to fail. At worst this can cause a saturation of your server's RPC layer, and at best this causes erroneous exhaustion of the client's retries. We should find a way to make these estimates a bit smarter for large scans. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28370) Default user quotas are refreshing too frequently
Ray Mattingly created HBASE-28370: - Summary: Default user quotas are refreshing too frequently Key: HBASE-28370 URL: https://issues.apache.org/jira/browse/HBASE-28370 Project: HBase Issue Type: Improvement Reporter: Ray Mattingly In [https://github.com/apache/hbase/pull/5666] we introduced default user quotas, but I accidentally called UserQuotaState's default constructor rather than passing in the current timestamp. The consequence is that we're constantly refreshing these default user quotas, and this can be a bottleneck for horizontal cluster scalability. This should be a 1 line fix in QuotaUtil's buildDefaultUserQuotaState method. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28349) Atomic requests should increment read usage in quotas
Ray Mattingly created HBASE-28349: - Summary: Atomic requests should increment read usage in quotas Key: HBASE-28349 URL: https://issues.apache.org/jira/browse/HBASE-28349 Project: HBase Issue Type: Improvement Reporter: Ray Mattingly Right now atomic operations are just treated as a single write from the quota perspective. Since an atomic operation also encompasses a read, it would make sense to increment readNum and readSize counts appropriately. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28215) Region reopen procedure should support some sort of throttling
Ray Mattingly created HBASE-28215: - Summary: Region reopen procedure should support some sort of throttling Key: HBASE-28215 URL: https://issues.apache.org/jira/browse/HBASE-28215 Project: HBase Issue Type: Improvement Reporter: Ray Mattingly The mass reopening of regions caused by a table descriptor modification can be quite disruptive. For latency/error sensitive workloads, like our user facing traffic, we need to be very careful about when we modify table descriptors, and it can be virtually impossible to do it painlessly for busy tables. It would be nice if we supported configurable batching/throttling of reopenings so that the amplitude of any disruption can be kept relatively small. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28175) RpcLogDetails' Message can become corrupt before log is consumed
Ray Mattingly created HBASE-28175: - Summary: RpcLogDetails' Message can become corrupt before log is consumed Key: HBASE-28175 URL: https://issues.apache.org/jira/browse/HBASE-28175 Project: HBase Issue Type: Bug Reporter: Ray Mattingly Assignee: Ray Mattingly The RpcLogDetails class represents a slow (or large) log event which will later be consumed by the SlowLogQueueService. Right now the RpcLogDetails' param field points to the slow call's Message, and this Message is backed by a CodedInputStream which may be overwritten before the given log is consumed. This overwriting of the Message may result in slow query payloads for which the metadata derived post-consumption is inaccurate. To solve this bug I think we need to copy the Message in the RpcLogDetails constructor. I have this bug reproduced in a QA environment and will test out this idea and open a PR shortly if the test results are promising. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28146) ServerManager's rsAdmins map should be thread safe
Ray Mattingly created HBASE-28146: - Summary: ServerManager's rsAdmins map should be thread safe Key: HBASE-28146 URL: https://issues.apache.org/jira/browse/HBASE-28146 Project: HBase Issue Type: Bug Affects Versions: 2.5.5 Reporter: Ray Mattingly Assignee: Ray Mattingly On 2.x [the ServerManager registers admins in a HashMap|https://github.com/apache/hbase/blob/branch-2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java]. This can result in thread safety issues — we recently observed an exception which caused a region to be indefinitely stuck in transition until we could manually intervene. We saw the following exception in the HMaster logs: {code:java} 2023-10-11 02:20:05.213 [RSProcedureDispatcher-pool-325] ERROR org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher: Unexpected error caught, this may cause the procedure to hang forever java.lang.ClassCastException: class java.util.HashMap$Node cannot be cast to class java.util.HashMap$TreeNode (java.util.HashMap$Node and java.util.HashMap$TreeNode are in module java.base of loader 'bootstrap') at java.util.HashMap$TreeNode.moveRootToFront(HashMap.java:1900) ~[?:?] at java.util.HashMap$TreeNode.treeify(HashMap.java:2016) ~[?:?] at java.util.HashMap.treeifyBin(HashMap.java:768) ~[?:?] at java.util.HashMap.putVal(HashMap.java:640) ~[?:?] at java.util.HashMap.put(HashMap.java:608) ~[?:?] at org.apache.hadoop.hbase.master.ServerManager.getRsAdmin(ServerManager.java:723){code} -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28002) Add Get, Mutate, and Multi operations to slow log params
Ray Mattingly created HBASE-28002: - Summary: Add Get, Mutate, and Multi operations to slow log params Key: HBASE-28002 URL: https://issues.apache.org/jira/browse/HBASE-28002 Project: HBase Issue Type: Improvement Reporter: Ray Mattingly In https://issues.apache.org/jira/browse/HBASE-27536 we added the ability to include Scan operations in the slow log params. It would be useful to include more operations too. Beyond just showing the shape of the request to slow log readers, this would also ensure that operation attributes can be inferred. There are a few complications to consider for some operation types: * Mutate: ** we should probably strip the columns from these puts. Otherwise we might produce unpredictably large slow log payloads, and there are potentially security concerns to consider * Multi ** we should also consider stripping columns from these requests ** (configurably?) limiting the number of operations that can be included. For example, maybe we only want to include 5 operations on a slow log payload for a 100 operation MultiRequest for the sake of brevity ** we may want to deduplicate operation attributes. I'm not really sure how we'd do this without the output being misleading -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-28001) Add request attribute support to BufferedMutator
Ray Mattingly created HBASE-28001: - Summary: Add request attribute support to BufferedMutator Key: HBASE-28001 URL: https://issues.apache.org/jira/browse/HBASE-28001 Project: HBase Issue Type: Improvement Reporter: Ray Mattingly In https://issues.apache.org/jira/browse/HBASE-27657 we added support for specifying connection and request attributes. One oversight was including support for doing so via the BufferedMutator class. We should add such support in a follow up PR. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-27975) Region (un)assignment should have a more direct timeout
Ray Mattingly created HBASE-27975: - Summary: Region (un)assignment should have a more direct timeout Key: HBASE-27975 URL: https://issues.apache.org/jira/browse/HBASE-27975 Project: HBase Issue Type: Improvement Reporter: Ray Mattingly h3. Problem We've observed a few cases in which region (un)assignment can hang for significant, and sometimes seemingly indefinite, periods of time. This results in unpredictably long downtime which must be remediated via manually initiated ServerCrashProcedures. h3. Example 1 If a RS is unable to communicate with the NameNode and it is asked to close a region then its RS_CLOSE_REGION thread will get stuck awaiting a NN failover. Due to several default configurations of options like: * hbase.hstore.flush.retries.number * hbase.server.pause * dfs.client.failover.max.attempts * dfs.client.failover.sleep.base.millis * dfs.client.failover.max.attempts this region unassignment attempt will hang for approximately 30 minutes before it allows the failure to bubble up and automatically trigger a ServerCrashProcedure. One can tune the aforementioned options to reduce the TTR here, but it's not a very obvious/direct solution. h3. Example 2 In rare cases our public cloud provider may supply us with machines that have degraded hardware. If we're unable to catch this degradation prior to startup, then we've observed that the degraded RegionServer process may come online; as a result it will be assigned regions which can often never actually be successfully opened. If the RegionServer's assignment handling fails to intentionally fail, then there will never be outside intervention; the assignment will be stuck hanging indefinitely. I've written [a unit test|https://github.com/apache/hbase/compare/master...HubSpot:hbase:rsit-opening-repro] which reproduces this behavior. On this same branch is a unit test demonstrating that a timeout placed on the AssignRegionHandler helps to fast fail and reliably trigger the necessary ServerCrashProcedure. h3. Proposal I want to propose that we add optional and configurable timeouts to the AssignRegion and UnassignRegion event handlers. This would allow us to much more intentionally & clearly prevent long running retries for these downtime inducing procedures and could consequently improve our reliability in both examples. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (HBASE-27786) CompoundBloomFilters break with an error rate that is too high
Ray Mattingly created HBASE-27786: - Summary: CompoundBloomFilters break with an error rate that is too high Key: HBASE-27786 URL: https://issues.apache.org/jira/browse/HBASE-27786 Project: HBase Issue Type: Bug Affects Versions: 2.5.2 Reporter: Ray Mattingly At my company we're beginning to more heavily utilize the bloom error rate configuration. This is because bloom filters are a nice optimization, but for well distributed workloads with relatively dense data (many rows:host), we've found that they can cause lots of memory/GC pressure unless they can entirely fit in the block cache (and consequently not churn memory that's subject to GC). Because it's easier to estimate the memory requirements of changes in existing bloom filters, rather than net new bloom filters, we wanted to begin with very high bloom error rates (and consequently small bloom filters), and then ratchet down as memory availability allowed. This led to us discovering that bloom filters appear to become corrupt at a relatively arbitrary error rate threshold. Blooms with an error rate of 0.61 work as expected, but produce nonsensical results with an error rate of 0.62. I've pushed this branch with test updates to demonstrate the deficit: [https://github.com/apache/hbase/compare/master...HubSpot:hbase:rmattingly/bloom-error-rate-bug] The test changes confirm that the BloomFilterUtil works as expected, at least with respect to its error rate : size relationship. You can see this in the output of {{{}TestBloomFilterChunk#testBloomErrorRateSizeRelationship{}}}: {noformat} previousErrorRate=0.01, previousSize=1048568 currentErrorRate=0.05, currentSize=682109 previousErrorRate=0.05, previousSize=682109 currentErrorRate=0.1, currentSize=524284 previousErrorRate=0.1, previousSize=524284 currentErrorRate=0.2, currentSize=366459 previousErrorRate=0.2, previousSize=366459 currentErrorRate=0.4, currentSize=208634 previousErrorRate=0.4, previousSize=208634 currentErrorRate=0.5, currentSize=157826 previousErrorRate=0.5, previousSize=157826 currentErrorRate=0.75, currentSize=65504 previousErrorRate=0.75, previousSize=65504 currentErrorRate=0.99, currentSize=2289 {noformat} With this in mind, the updates to {{TestCompoundBloomFilter}} tell us that the bug must live somewhere in the {{CompoundBloomFilter}} logic. The output indicates this: {noformat} 2023-04-10T15:07:50,925 INFO [Time-limited test] regionserver.TestCompoundBloomFilter(245): Functional bloom has error rate 0.01 and size 1kb ... 2023-04-10T15:07:56,657 INFO [Time-limited test] regionserver.TestCompoundBloomFilter(245): Functional bloom has error rate 0.61 and size 1kb ... java.lang.AssertionError: False positive is too high: 0.99985334 (greater than 0.65), fake lookup is enabled. Bloom size is 4687kb at org.junit.Assert.fail(Assert.java:89) at org.junit.Assert.assertTrue(Assert.java:42) at org.apache.hadoop.hbase.regionserver.TestCompoundBloomFilter.readStoreFile(TestCompoundBloomFilter.java:243) {noformat} The bloom size change from ~1kb -> 4687kb and total lack of precision is clearly not as intended, and totally inline with what we saw in our HBase clusters that attempted to use high bloom error rates. -- This message was sent by Atlassian Jira (v8.20.10#820010)
Re: Request to be added as a contributor
Amazing, thank you! Have a good one, Ray On Tue, Mar 22, 2022 at 11:53 AM Nick Dimiduk wrote: > I have added you as a contributor. You may need to log out and back in > again for the permission change to take effect. > > Thanks, > Nick > > On Tue, Mar 22, 2022 at 4:49 PM Ray Mattingly > wrote: > > > No sweat! :) > > > > On Tue, Mar 22, 2022 at 11:48 AM Nick Dimiduk > wrote: > > > > > Err. Ray. Sorry :( > > > > > > On Tue, Mar 22, 2022 at 4:45 PM Nick Dimiduk > > wrote: > > > > > > > Hi Matt, > > > > > > > > Thank you for your interest. Do you have an account on our Jira > > instance? > > > > What is your user ID? > > > > > > > > Thanks, > > > > Nick > > > > > > > > On Tue, Mar 22, 2022 at 4:28 PM Ray Mattingly > > > > wrote: > > > > > > > >> Hello, > > > >> > > > >> I'm a SWE at HubSpot and would love to start contributing upstream — > > > >> probably, first, with a solution for > > > >> https://issues.apache.org/jira/browse/HBASE-26874 > <https://issues.apache.org/jira/browse/HBASE-26874> > > > <https://issues.apache.org/jira/browse/HBASE-26874 > <https://issues.apache.org/jira/browse/HBASE-26874> > > > > > >> <https://issues.apache.org/jira/browse/HBASE-26874 > <https://issues.apache.org/jira/browse/HBASE-26874> > > > <https://issues.apache.org/jira/browse/HBASE-26874 > <https://issues.apache.org/jira/browse/HBASE-26874> > > > > > > > > > >> that > > > >> we've been using at HubSpot for a few months now. > > > >> > > > >> Can I please be added as a contributor? > > > >> > > > >> Thanks, > > > >> Ray Mattingly > > > >> > > > > > > > > > >
Re: Request to be added as a contributor
No sweat! :) On Tue, Mar 22, 2022 at 11:48 AM Nick Dimiduk wrote: > Err. Ray. Sorry :( > > On Tue, Mar 22, 2022 at 4:45 PM Nick Dimiduk wrote: > > > Hi Matt, > > > > Thank you for your interest. Do you have an account on our Jira instance? > > What is your user ID? > > > > Thanks, > > Nick > > > > On Tue, Mar 22, 2022 at 4:28 PM Ray Mattingly > > wrote: > > > >> Hello, > >> > >> I'm a SWE at HubSpot and would love to start contributing upstream — > >> probably, first, with a solution for > >> https://issues.apache.org/jira/browse/HBASE-26874 > <https://issues.apache.org/jira/browse/HBASE-26874> > >> <https://issues.apache.org/jira/browse/HBASE-26874 > <https://issues.apache.org/jira/browse/HBASE-26874> > > > >> that > >> we've been using at HubSpot for a few months now. > >> > >> Can I please be added as a contributor? > >> > >> Thanks, > >> Ray Mattingly > >> > > >
Re: Request to be added as a contributor
Yes I do — the user ID is rmdmattingly Thanks! On Tue, Mar 22, 2022 at 11:45 AM Nick Dimiduk wrote: > Hi Matt, > > Thank you for your interest. Do you have an account on our Jira instance? > What is your user ID? > > Thanks, > Nick > > On Tue, Mar 22, 2022 at 4:28 PM Ray Mattingly > wrote: > > > Hello, > > > > I'm a SWE at HubSpot and would love to start contributing upstream — > > probably, first, with a solution for > > https://issues.apache.org/jira/browse/HBASE-26874 > <https://issues.apache.org/jira/browse/HBASE-26874> > > <https://issues.apache.org/jira/browse/HBASE-26874 > <https://issues.apache.org/jira/browse/HBASE-26874> > > > > that > > we've been using at HubSpot for a few months now. > > > > Can I please be added as a contributor? > > > > Thanks, > > Ray Mattingly > > >
Request to be added as a contributor
Hello, I'm a SWE at HubSpot and would love to start contributing upstream — probably, first, with a solution for https://issues.apache.org/jira/browse/HBASE-26874 <https://issues.apache.org/jira/browse/HBASE-26874> that we've been using at HubSpot for a few months now. Can I please be added as a contributor? Thanks, Ray Mattingly