-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-25984: Avoid premature reuse of sync futures in FSHLog #3371
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,74 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.hadoop.hbase.regionserver.wal; | ||
|
|
||
| import java.util.concurrent.TimeUnit; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.hbase.HConstants; | ||
| import org.apache.yetus.audience.InterfaceAudience; | ||
| import org.apache.hbase.thirdparty.com.google.common.cache.Cache; | ||
| import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder; | ||
|
|
||
| /** | ||
| * A cache of {@link SyncFuture}s. This class supports two methods | ||
| * {@link SyncFutureCache#getIfPresentOrNew()} and {@link SyncFutureCache#offer()}. | ||
| * | ||
| * Usage pattern: | ||
| * SyncFuture sf = syncFutureCache.getIfPresentOrNew(); | ||
| * sf.reset(...); | ||
| * // Use the sync future | ||
| * finally: syncFutureCache.offer(sf); | ||
| * | ||
| * Offering the sync future back to the cache makes it eligible for reuse within the same thread | ||
| * context. Cache keyed by the accessing thread instance and automatically invalidated if it remains | ||
| * unused for {@link SyncFutureCache#SYNC_FUTURE_INVALIDATION_TIMEOUT_MINS} minutes. | ||
| */ | ||
| @InterfaceAudience.Private | ||
| public final class SyncFutureCache { | ||
|
|
||
| private static final long SYNC_FUTURE_INVALIDATION_TIMEOUT_MINS = 2; | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 2 min is good estimate? Trying to understand if we might run into overhead (cache entry getting expired followed by same entry getting created for same thread from pool) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If a handler remained idle for 2 mins, that indicates there isn't enough load on the server in which case there is no advantage in keeping this around. Usually only helpful if there is high load and potential for reuse so that we don't need to GC these small objects often. |
||
|
|
||
| private final Cache<Thread, SyncFuture> syncFutureCache; | ||
|
|
||
| public SyncFutureCache(final Configuration conf) { | ||
| final int handlerCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, | ||
| HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT); | ||
| syncFutureCache = CacheBuilder.newBuilder().initialCapacity(handlerCount) | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I thought this guava cache 'slow'. That Ben Manes tried to get a patch to guava that improved it but couldn't get interest so his caffeine cache has means of implementing gauava cache api.... so you can drop in his thing instead. Maybe it doesn't matter here because scale of objects is small? It is critical section though. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ah sweet, I heard about this Guava vs Caffeine thing, let me replace that. (didn't notice any performance issues with the patch but if we get some extra performance why not) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not sure if there is a benefit over a There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It also looks like your on 2.8.1, whereas a putIfAbsent optimization was added in 2.8.2 to avoid locking if the entry is present. That might help. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We don't need a putIfAbsent() either, switched to put() as we are ok with the overwrites here, it doesn't show up in the profiler now. Ya we can't use the
Right, we can't use TL here because of the bug, but WeakKey cache seems like a good alternative, thanks for the pointer. |
||
| .expireAfterWrite(SYNC_FUTURE_INVALIDATION_TIMEOUT_MINS, TimeUnit.MINUTES).build(); | ||
| } | ||
|
|
||
| public SyncFuture getIfPresentOrNew() { | ||
| // Invalidate the entry if a mapping exists. We do not want it to be reused at the same time. | ||
| SyncFuture future = syncFutureCache.asMap().remove(Thread.currentThread()); | ||
| return (future == null) ? new SyncFuture() : future; | ||
|
Comment on lines
+57
to
+58
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This suggestion was great. I think we can make this change in branch-1 as well (sorry I lost the track, not sure if branch-1 PR is still pending for merge or already merged) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There is no branch-1 patch (yet). |
||
| } | ||
|
|
||
| /** | ||
| * Offers the sync future back to the cache for reuse. | ||
| */ | ||
| public void offer(SyncFuture syncFuture) { | ||
| // It is ok to overwrite an existing mapping. | ||
| syncFutureCache.asMap().put(syncFuture.getThread(), syncFuture); | ||
| } | ||
|
|
||
| public void clear() { | ||
| if (syncFutureCache != null) { | ||
| syncFutureCache.invalidateAll(); | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This patch in the current form doesn't get rid of future overwrites as it does not seem to cause any issues in AsyncWAL case (based on code reading), but if the reviewers think we should do that, I can refactor accordingly.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Where is the future overwrite here? The call to 'done'?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@saintstack I documented the race here..
Once done() is called, the future can be reused immediately from another handler (without this patch). That was causing deadlocks in FSHLog. Based on my analysis of AsyncFSWAL, I think the overwrites are possible but it should not affect the correctness as the safe point is attained in a different manner. So wanted to check with Duo who is the expert on that.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry. Should have done more background reading before commenting.