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
BP-14 force() API - client side implementation #1436
Conversation
6034b74
to
c738745
Compare
retest this please |
1 similar comment
retest this please |
@@ -1790,6 +1851,14 @@ void handleDelayedWriteBookieFailure() { | |||
} | |||
return; | |||
} | |||
if (writeFlags.contains(WriteFlag.DEFERRED_SYNC)) { | |||
if (LOG.isDebugEnabled()) { | |||
LOG.debug("Cannot perform ensemble changee with writeflags {}." |
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.
a typo changee
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.
done
@@ -41,6 +41,7 @@ | |||
|
|||
String ADD_OP = "ADD_ENTRY"; | |||
String ADD_OP_UR = "ADD_ENTRY_UR"; // Under Replicated during AddEntry. | |||
String FORCE_OP = "FORCE"; |
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.
how about also add some comments for force
here?
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.
done
long ledgerId = forceLedgerResponse.getLedgerId(); | ||
|
||
if (LOG.isDebugEnabled()) { | ||
logResponse(status, "ledger", ledgerId); |
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.
seems logResponse
already has an isDebugEnabled()
internal.
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 is consistent with all the rest of the file, I can change every where, but I think that it can be a slight save of resources.
I would prefer to keep the style consistent.
I will change if you feel strong about it
@jvrao @eolivelli : I am wondering how would be the relationship between If you guys agree with that, it is probably making more sense to reuse if no, we need to figure out what does following behavior mean:
|
@sijie explicitlac for deferred sync writers only sends the value of LastAddConfirmed and it is still a way to notify LAC changes without a real write. The difference is that LastAddConfirmed variable does not advance at every write. I think they are different operations, and need different semantics. Force() must be acknowledged by the full ensemble. I think it is better to keep force and explicitlac separated. I will add test cases to cover the usage of explicit LAC with deferred sync flag. I will check the implementation today, in case I am missing something. Cc @jvrao |
// remember that we are inside OrderedExecutor, this induces a strict ordering | ||
// on the sequence of events | ||
this.currentNonDurableLastAddConfirmed = lh.pendingAddsSequenceHead; | ||
LOG.debug("start force() on ledger {} capturing {} ", lh.ledgerId, currentNonDurableLastAddConfirmed); |
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.
instead of capturing {} ; just say clientNonDurableLac {}
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.
done
@@ -60,6 +60,11 @@ public WriteSet getWriteSetForLongPoll(long entryId) { | |||
return WriteSetImpl.create(ensembleSize, ensembleSize /* writeQuorumSize */, entryId); | |||
} | |||
|
|||
@Override | |||
public WriteSet getWriteSetForForceLedger() { | |||
return WriteSetImpl.create(ensembleSize, ensembleSize /* writeQuorumSize */, 0); |
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.
Have a comment similar to longPoll method above.
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.
done
bk.getMainWorkerPool().submit(new SafeRunnable() { | ||
@Override | ||
public void safeRun() { | ||
LOG.warn("Attempt to use a closed ledger: {}", ledgerId); |
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.
Nit: Force() attempted on a closed ledger {}
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.
done
return result; | ||
} | ||
|
||
if (pendingAddsSequenceHead == INVALID_ENTRY_ID) { |
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.
Please add comments on what it really means.
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.
done
LOG.debug("Cannot perform ensemble changee with writeflags {}." | ||
+ "Failed bookies {} for ledger {}.", | ||
writeFlags, delayedWriteFailedBookies, ledgerId); | ||
} |
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.
Maybe we should treat this also unrecoverable error, or even fail at the caller itself.
Because if AQ< WQ we can continue to write but force is going to fail anyway.
To be consistent, any write error (even if it is delayed write error it needs to fail under this flag)
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.
actually with writeflag = DEFERRED_SYNC this block of code will never run.
I have copied this block for consistency with other cases of "enabled" disableEnsembleChangeFeature.
I can drop it if you prefer, this code will never run
@@ -284,7 +284,9 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre | |||
// Got an error after satisfying AQ. This means we are under replicated at the create itself. | |||
// Update the stat to reflect it. | |||
addOpUrCounter.inc(); | |||
if (!lh.bk.getDisableEnsembleChangeFeature().isAvailable() && !lh.bk.delayEnsembleChange) { | |||
if (!lh.bk.getDisableEnsembleChangeFeature().isAvailable() | |||
&& !lh.writeFlags.contains(WriteFlag.DEFERRED_SYNC) |
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.
What is the use of adding this check here?
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.
We are acting as "enabled" disableEnsembleChangeFeature, so I kept the same exact beheaviour
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.
I don't think this check makes sense. This may not cause harm as failedBookies
set is best effort scenario.
@sijie do you think we need this check here?
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.
removed
I am not saying they are same operations. what I am questioning here is - is it better to extend the explicit lac request with a flag that force flushing the data? In that way the behavior would be better defined when people enable explicit lac on a deferred-sync ledger. Or we have to clarify following cases:
The case 4 is the most confused case. |
// Build the request | ||
BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() | ||
.setVersion(ProtocolVersion.VERSION_THREE) | ||
.setOperation(OperationType.FORCE_LEDGER) |
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.
Did we make protobuf changes for this?
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.
There is a new RPC for FORCE_LEDGER, nothing else
I don't understand where the explicitLAC is clashing with the deferred_sync. ExplicitLAC is completely independent operatin, where if the client thinks it's LAC is ahead of piggybacked LAC, it can send an update. and differedSync dictates how the client LAC gets updated. Maybe I am missing something @sijie |
@jvrao : explicit lac is advancing lac, while force is also advancing lac. my point is why can we consolidate these two behaviors as one. to me, 'explicit lac' can be a background 'force'; 'force' is an explicit application-defined 'explict lac' operation. |
@sijie I have added a test case about using ExplicitLAC and DEFERRED_SYNC. I think that current behaviour is what I am expecting:
It seems to me that this is consistent with the expectations of DEFERRED_SYNC writeflag.
I think that the client controls LAC using force() and ExplicitLAC is a background task which seems to me like an optimization. Maybe it is like the initial proposal of having a "piggybacked LastAddPersisted" on AddResponse which would have made LAC to advance even without an explicit "force()". We could add a flag which enables a timer to send explicit force() periodically, or that performs a force() during the execution of ExplicitLAC timer. cc @jvrao |
if we step back to think about the behavior here, both of these two features are kind of "flush/sync" behavior. forget about the implementation here, think from a storage system, image we have a "flush" primitive - it basically flushes unwritten data and make these data available. so, "explicit lac" - it is a background "flush", flushing the data between lac and lastAddPushed. it is running in background. those are exactly similar/same behavior from this sense. if you look at filesystem, filesystem has pdflush doing background flush and provides fsync systemcall for applications to do explicitly flush. but under the hood, they are doing the same thing - flushing the unflushed data. if you map the behavior of bookkeeper to filesystem, "explicit lac" is exactly the background flush "pdflush"; while "force" is the system call. my main point here is if these two behaviors are similar/same, we should consolidate the underlying mechanism rather than having two separate routes to do same operations. |
@sijie your point is very interesting. |
sorry your comment isn't clear to you. are you saying JV asked for failing pending adds when encountered issues on #force? or |
I meant @jvrao suggested to error our pending adds. I will remove that from the patch, but before spending time I would like to have acknowledgement feom JV |
@sijie @jvrao FYI I have added a commit to drop 'error out pending adds" in case of failure in force operation. So we are not erroring out pending adds, they will fail naturally in case of bookie failure. Hopefully @sijie and @jiazhai will be still okay with the patch |
@eolivelli thanks a lot for pushing this change forward. Great work.!! |
ff6b612
to
dd9bdf0
Compare
@jvrao @sijie @jiazhai I have rebased to latest master, I will wait for CI to pass and then merge. I think existing BC tests are enough to guarantee that we did not break compatibility. Now that BP-14 has got its final form I will start tests and benchmarks on downstream applications (once this change reaches Apache Snapshots Repository) |
rerun bookkeeper-server bookie tests |
4 similar comments
rerun bookkeeper-server bookie tests |
rerun bookkeeper-server bookie tests |
rerun bookkeeper-server bookie tests |
rerun bookkeeper-server bookie tests |
The force() API enables the client (usually with DEFERRED_SYNC write flags) to require a point of synchronization with all the bookies in the ensemble, to have guarantees about durability of previously written entries (and ackknowledgerd), this way LastAddConfirmed is able to advance.
For DEFERRED_SYNC writers LastAddConfirmed will advance only using this API