-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-25998: Redo synchronization in SyncFuture #3382
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 |
|---|---|---|
| @@ -1,4 +1,4 @@ | ||
| /** | ||
| /* | ||
| * 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 | ||
|
|
@@ -19,7 +19,8 @@ | |
|
|
||
| import java.util.concurrent.ExecutionException; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| import java.util.concurrent.locks.Condition; | ||
| import java.util.concurrent.locks.ReentrantLock; | ||
| import org.apache.hadoop.hbase.exceptions.TimeoutIOException; | ||
| import org.apache.yetus.audience.InterfaceAudience; | ||
|
|
||
|
|
@@ -44,15 +45,23 @@ | |
| */ | ||
| @InterfaceAudience.Private | ||
| class SyncFuture { | ||
| // Implementation notes: I tried using a cyclicbarrier in here for handler and sync threads | ||
| // to coordinate on but it did not give any obvious advantage and some issues with order in which | ||
| // events happen. | ||
|
|
||
| private static final long NOT_DONE = -1L; | ||
| private Thread t; | ||
|
|
||
| /** | ||
| * The transaction id of this operation, monotonically increases. | ||
| * Lock protecting the thread-safe fields. | ||
| */ | ||
| private final ReentrantLock doneLock; | ||
|
|
||
| /** | ||
| * Condition to wait on for client threads. | ||
| */ | ||
| private final Condition doneCondition; | ||
|
|
||
| /* | ||
| * Fields below are protected by {@link SyncFuture#doneLock}. | ||
| */ | ||
| private long txid; | ||
|
|
||
| /** | ||
| * The transaction id that was set in here when we were marked done. Should be equal or > txnId. | ||
|
|
@@ -65,16 +74,30 @@ class SyncFuture { | |
| */ | ||
| private Throwable throwable; | ||
|
|
||
| private Thread t; | ||
| /* | ||
| * Fields below are created once at reset() and accessed without any lock. Should be ok as they | ||
| * are immutable for this instance of sync future until it is reset. | ||
|
Contributor
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. Is it worth it doing all this handling just so we reuse SyncFuture instances? Maybe a later experiment would be trying to create a SyncFuture every time? Just a thought.
Contributor
Author
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. Ya, the whole SyncFuture caching seems unnecessary, I was discussing this with my colleagues at work. cc: @apurtell @d-c-manning I don't have the full historical context here but if I were to guess I think it is because we don't want GC to (over) work with millions of these small sync future objects since they are created once per mutation. May be the latest versions of Java garbage collectors don't need these kind of optimizations, agree that this needs more performance analysis. Let me create a jira to perf analyze the removal of this. |
||
| */ | ||
|
|
||
| /** | ||
| * The transaction id of this operation, monotonically increases. | ||
| */ | ||
| private long txid; | ||
|
|
||
| private boolean forceSync; | ||
|
|
||
| SyncFuture() { | ||
| this.doneLock = new ReentrantLock(); | ||
|
Contributor
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. These two fields could be marked as final if we never change them after construction. |
||
| this.doneCondition = doneLock.newCondition(); | ||
| } | ||
|
|
||
| /** | ||
| * Call this method to clear old usage and get it ready for new deploy. | ||
| * | ||
| * @param txid the new transaction id | ||
| * @return this | ||
| */ | ||
| synchronized SyncFuture reset(long txid) { | ||
| SyncFuture reset(long txid, boolean forceSync) { | ||
| if (t != null && t != Thread.currentThread()) { | ||
| throw new IllegalStateException(); | ||
| } | ||
|
|
@@ -83,30 +106,26 @@ synchronized SyncFuture reset(long txid) { | |
| throw new IllegalStateException("" + txid + " " + Thread.currentThread()); | ||
| } | ||
| this.doneTxid = NOT_DONE; | ||
| this.forceSync = forceSync; | ||
| this.txid = txid; | ||
| this.throwable = null; | ||
| return this; | ||
| } | ||
|
|
||
| @Override | ||
| public synchronized String toString() { | ||
| public String toString() { | ||
| return "done=" + isDone() + ", txid=" + this.txid + " threadID=" + t.getId() + | ||
| " threadName=" + t.getName(); | ||
| } | ||
|
|
||
| synchronized long getTxid() { | ||
| long getTxid() { | ||
| return this.txid; | ||
| } | ||
|
|
||
| synchronized boolean isForceSync() { | ||
| boolean isForceSync() { | ||
| return forceSync; | ||
|
Contributor
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 should be covered by the lock? The comment at head of the class says this data member should be covered by the lock? ("all below")
Contributor
Author
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 also this comment.. |
||
| } | ||
|
|
||
| synchronized SyncFuture setForceSync(boolean forceSync) { | ||
| this.forceSync = forceSync; | ||
| return this; | ||
| } | ||
|
|
||
| /** | ||
| * Returns the thread that owned this sync future, use with caution as we return the reference to | ||
| * the actual thread object. | ||
|
|
@@ -122,55 +141,67 @@ Thread getThread() { | |
| * @return True if we successfully marked this outstanding future as completed/done. Returns false | ||
| * if this future is already 'done' when this method called. | ||
| */ | ||
| synchronized boolean done(final long txid, final Throwable t) { | ||
| if (isDone()) { | ||
| return false; | ||
| } | ||
| this.throwable = t; | ||
| if (txid < this.txid) { | ||
| // Something badly wrong. | ||
| if (throwable == null) { | ||
| this.throwable = | ||
| new IllegalStateException("done txid=" + txid + ", my txid=" + this.txid); | ||
| boolean done(final long txid, final Throwable t) { | ||
| doneLock.lock(); | ||
| try { | ||
| if (doneTxid != NOT_DONE) { | ||
| return false; | ||
| } | ||
| this.throwable = t; | ||
| if (txid < this.txid) { | ||
| // Something badly wrong. | ||
| if (throwable == null) { | ||
| this.throwable = | ||
| new IllegalStateException("done txid=" + txid + ", my txid=" + this.txid); | ||
| } | ||
| } | ||
| // Mark done. | ||
| this.doneTxid = txid; | ||
| doneCondition.signalAll(); | ||
| return true; | ||
| } finally { | ||
| doneLock.unlock(); | ||
| } | ||
| // Mark done. | ||
| this.doneTxid = txid; | ||
| // Wake up waiting threads. | ||
| notify(); | ||
| return true; | ||
| } | ||
|
|
||
| boolean cancel(boolean mayInterruptIfRunning) { | ||
| throw new UnsupportedOperationException(); | ||
| } | ||
|
|
||
| synchronized long get(long timeoutNs) throws InterruptedException, | ||
| long get(long timeoutNs) throws InterruptedException, | ||
| ExecutionException, TimeoutIOException { | ||
| final long done = System.nanoTime() + timeoutNs; | ||
| while (!isDone()) { | ||
| wait(1000); | ||
| if (System.nanoTime() >= done) { | ||
| throw new TimeoutIOException( | ||
| "Failed to get sync result after " + TimeUnit.NANOSECONDS.toMillis(timeoutNs) | ||
| + " ms for txid=" + this.txid + ", WAL system stuck?"); | ||
| doneLock.lock(); | ||
| try { | ||
| while (doneTxid == NOT_DONE) { | ||
| if (!doneCondition.await(timeoutNs, TimeUnit.NANOSECONDS)) { | ||
| throw new TimeoutIOException("Failed to get sync result after " | ||
| + TimeUnit.NANOSECONDS.toMillis(timeoutNs) + " ms for txid=" + this.txid | ||
| + ", WAL system stuck?"); | ||
| } | ||
| } | ||
| if (this.throwable != null) { | ||
| throw new ExecutionException(this.throwable); | ||
| } | ||
| return this.doneTxid; | ||
| } finally { | ||
| doneLock.unlock(); | ||
| } | ||
| if (this.throwable != null) { | ||
| throw new ExecutionException(this.throwable); | ||
| } | ||
| return this.doneTxid; | ||
| } | ||
|
|
||
| synchronized boolean isDone() { | ||
| return this.doneTxid != NOT_DONE; | ||
| } | ||
|
|
||
| synchronized boolean isThrowable() { | ||
| return isDone() && getThrowable() != null; | ||
| boolean isDone() { | ||
| doneLock.lock(); | ||
| try { | ||
| return this.doneTxid != NOT_DONE; | ||
| } finally { | ||
| doneLock.unlock(); | ||
| } | ||
| } | ||
|
|
||
| synchronized Throwable getThrowable() { | ||
| return this.throwable; | ||
| Throwable getThrowable() { | ||
| doneLock.lock(); | ||
| try { | ||
| if (doneTxid == NOT_DONE) { | ||
| return null; | ||
| } | ||
| return this.throwable; | ||
| } finally { | ||
| doneLock.unlock(); | ||
| } | ||
| } | ||
| } | ||
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.
Just to note that above the methods are changed from getTxid to getTxId but then here we have beginTxid... super nit... but you might want them to be consistent? Just leave the getTxid?
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.
The whole Txid change mess was unintentional, undone.