Skip to content

Conversation

@ivankelly
Copy link
Contributor

Which means that for the two LedgerHandle operations that mutate the
metadata, ensemble change and closing, ensure that metadata is written
to the metadata store before the client ever uses it.

Master issue: #281

Which means that for the two LedgerHandle operations that mutate the
metadata, ensemble change and closing, ensure that metadata is written
to the metadata store before the client ever uses it.

Master issue: apache#281
@ivankelly ivankelly self-assigned this Sep 4, 2018
@sijie
Copy link
Member

sijie commented Sep 4, 2018

@jvrao @dlg99 @athanatos please spend some time on reviewing this PR.

@sijie
Copy link
Member

sijie commented Sep 7, 2018

Ping @jvrao @athanatos @dlg99

@ivankelly
Copy link
Contributor Author

rerun integration tests

1 similar comment
@ivankelly
Copy link
Contributor Author

rerun integration tests

int idx = entry.getKey();
BookieSocketAddress addr = entry.getValue();
if (LOG.isDebugEnabled()) {
LOG.debug("[EnsembleChange-L{}] replacing bookie: {} index: {}", ledgerId, addr, idx);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There used to have an ensembleChangeIdx in the logging message for debugging purpose. That was very useful on debugging ensemble change issues. It would be good if we can keep it.

Copy link

@athanatos athanatos Sep 11, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that's idx here if I'm interpreting it correctly. This variant allows you to specify more than one.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, the ensembleChangeIdx was something different. i'll readd it

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

suffix 'idx' confuses people. :)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ya, i'll change the name in any case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added a logContext variable, that's used in the calling method and this method to tie the logs of the operation together.

break;
}
}
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

there is a logging message dropped. it would be great not to drop log messages related to ensemble changes. they exist for a reason.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed, the final summary message is pretty handy.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will readd. It's not a final summary however, it's the change that we wish to make. It's not final until the zookeeper write completes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed, but let us not drop any debug/log messages.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Readded, but in the calling method, as this method doesn't have all the context.

// the ledger isn't closed between checking and
// updating lastAddPushed
if (getLedgerMetadata().isClosed()) {
if (getLedgerMetadata().isClosed() || closing) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

getLedgerMetadata().isClosed() || closing is used over multiple places, it would be good to have a function for it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will do

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

See #isHandleWrittable()

}

LedgerMetadataBuilder withWriteQuorumSize(int writeQuorumSize) {
checkArgument(ensembleSize >= writeQuorumSize, "Write quorum must be less or equal to ensemble size");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

check writeQuorumSize >= ackQuorumSize as well.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added


// should still be able to close as long as recovery closed the ledger
// with the same last entryId and length as in the write handle.
writeLh.close();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm this changes the closing behavior. I am not sure how it would impact the applications. so I would suggest keep the original behavior if closing a ledger hit metadata version exception don't attempt. If you want to change the behavior, do a separate PR for it.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This seems like a desirable change to me. Moreover, it protects against the case where the racing update came from the same client due to ZooKeeperClient resending the update.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@sijie the previous behaviour is not documented, nor is it well defined. In some cases a metadata version exception allowed a close to succeed, and in others it did not. I would not expect any application is relying on this behaviour, and if they are, they are probably broken in many other ways.
I can revert this my putting throwing an exception in the Predicate part of the loop if the metadata is closed. There'd be no guarantee that behaviour is still exactly matching though, because it isn't well defined currently.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I understood it is not documented. but if we are changing any existing behavior, I would suggest either doing it in a separate PR, or if it is difficult to do it in a separate PR, then update the javadoc of this method to make things clear "this method will not throw any exceptions anymore if hitting metadata version exceptions"

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It may still throw an exception on metadata version exception. However, it will only throw the exception if the length or last entry id in the conflicting write is different to what the caller of #close believed it to be. I strongly prefer making this change as part of this patch, as to do otherwise would be to insert arbitrary strange behaviour into the new implementation. I'll add a javadoc for this.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added javadoc


while ((pendingAddOp = pendingAddOps.peek()) != null
&& blockAddCompletions.get() == 0) {
&& !changingEnsemble) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

don't we need to make changingEnsemble volatile? or who is guaranteed this value is synchronized correctly ?

Copy link

@athanatos athanatos Sep 11, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, sendAddSuccessCallbacks reads it without the lock and handleBookieFailure appears to rely on readers seeing it synchronously for correctness, so either you need the lock there or changingEnsemble would have to be volatile.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It doesn't need to be volatile. it is only ever accessed in the ordered executor thread for this ledgerhandle. Rather than throwing synchronized around everything, we should start asserting in methods that we are in fact running in the ordered executor thread.

The previous blockAddCompletion stuff would not have been safe if these weren't in the same thread.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Such an assert would certainly clarify matters.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Adding these asserts is out of scope for this change.

if (delayedWriteFailedBookies.isEmpty()) {
return;
}
Map<Integer, BookieSocketAddress> toReplace = new HashMap<>();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the existing code to construct the hashmap from delayedWriteFailedBookies. it is one line code. any reason why do you split into 2 lines?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will change it back. not sure why i changed it in first place,

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

changed.

handleUnrecoverableErrorDuringAdd(rc);
synchronized (metadataLock) {
if (changingEnsemble) {
delayedWriteFailedBookies.putAll(failedBookies);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this change potentially has a side effect causing a longer pause during ensemble changes. I don't think it is a good idea to completely block on waiting previous ensemble to be done. The only matter is we need to ensure we only update local copy of metadata until all ensemble changes are completed, no? Can you explain the performance difference between current approach and previous approach?

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I could be misunderstanding, but I think the main difference in behavior is that between when changingEnsemble gets set to true and when the ensemble change completes, new writes continue to use the old ensemble whereas with the old machinery they would have optimistically used the new one. In both cases, we have to delay acks until the ensemble change is complete, but with this variant we may have to wait to resend writes for entries written since the ensemble change began which didn't get aQ responses from unchanged indexes. Am I understanding that correctly @ivankelly ?

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jvrao Something to note here is that unsetSuccessAndSendWriteRequest will resend the write request to those bookies regardless of whether the entry already has aQ copies, so it shouldn't generate additional under replicated ledgers in the common case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@sijie I think superficially it can look like it takes longer, but in practice it should only ever take less time.

The dominating latency in this operation is the write to zookeeper(LatWrite). In the case where we do not block other changes to the ensemble, and there are two failures, one of the updates will fail, have to reread (LatRead) and write again.

So the latency is (LatWrite + LatRead + LatWrite).

With the new code, it's just LatWrite + LatWrite. It gets better if there's more than 2 failures.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@athanatos your understanding is correct. We could modify this to unsetSuccessAndSendWriteRequest after each sucessful zk write. It would at least spread the outbound load a little.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not really worried about that, and we can always add machinery for projecting the post-update ensemble for writes later.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

make senses to me

.lastEntry().getValue().get(replacedBookieIdx);
replaced &= !Objects.equal(replacedBookieAddr, failedBookieAddr);
List<BookieSocketAddress> origEnsemble = getCurrentEnsemble();
ensembleChangeLoop(origEnsemble, toReplace);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ensembleChangeLoop potentially will trigger callbacks. can we not call this method under metadataLock?

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, both the initial error check branch and the unsetSuccessAndSendWriteRequest calls can call callbacks.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, I can move it out. There shouldn't be a problem with callbacks under metadatalock though, as it's only protected a few members. Maybe deadlocks could be an issue down the line.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes move the callbacks out of locks would be better in general.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Moved out of synchronized block.

List<BookieSocketAddress> lastEnsemble = metadata.getLastEnsembleValue();
boolean failedBookieInEnsemble = failedBookies.entrySet().stream()
.anyMatch((e) -> lastEnsemble.get(e.getKey()).equals(e.getValue()));
return !metadata.isClosed() && !metadata.isInRecovery() && failedBookieInEnsemble;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

!metadata.isClosed() &&
!metadata.isInRecovery() &&
failedBookies.entrySet().stream()
                        .anyMatch((e) -> lastEnsemble.get(e.getKey()).equals(e.getValue()))

so we only compute failedBookieInEnsmeble when metadata is open.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

changed

// f) writing entry E+1 encountered LedgerFencedException which will enter ledger close procedure
// g) it would find that ledger metadata is closed, then it callbacks immediately without erroring
// out any pendings
synchronized (LedgerHandle.this) {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

drainPendingAddsToErrorOut is already synchronized.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the synchronization is around the other fields also, like the length.

final State prevState;
List<PendingAddOp> pendingAdds;

if (isClosed()) {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it possible for isClosed() but not closing here? If closing, drainPendingAddsToErrorOut must have already happened (and no new ones can be added due to the check in doAsyncAddEntry), so this must necessarily be a noop. I think we should be able to check closing instead and assert that pendingAddOps is already empty.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@athanatos if we're listening for metadata updates, it's possible someone else updated the metadata and closed it just before we close it. However, i think isClosed is wrong here as it confuses two things. What we care about in this check is whether the handle is closed or not, so using an enum for the handle state would work better.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, but I think what I'm getting at is that whatever discovered that the ledger is actually closed or closed it should really already have called drainPendingAddsToErrorOut -- this code shouldn't be discovering that for the first time.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It looks like the listener stuff isn't in yet (#1580). In any case, this close is only on the writing handle. So if the state of the ledger changes, that state change will be accompanied by the any new writes being fenced (the protocol is that if the state is changed from open by anyone put the writer, all bookies in last ensemble must be told to fence).

So the only way for the handle to get to closed is for a call to this 'close' method. So we should check that the handle is in the 'open' state and only then drain the pending adds. (i.e. the state of the pending adds and the handle open state are tied).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This discussion becomes moot with how I've changed it now. We do call drain() in all cases, but I've done it this way to only have one synchronized block. In the case that the ledger is already closed, the drain should just return an empty list.

// Original intent of this change is to do a best-effort ensemble change.
// But this is not possible until the local metadata is completely immutable.
// Until the feature "Make LedgerMetadata Immutable #610" Is complete we will use
// handleBookieFailure() to handle delayed writes as regular bookie failures.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This comment probably needs to be rewritten to explain precisely why we need to block acks and what would need to change.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The whole deferred handle failure needs to change, but I can add a comment to clarify why this was previously an issue.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yep. That comment seems to imply that this refactor would fix it, so at least that bit should be updated.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The whole deferred handle failure needs to change,

@ivankelly can you flush out your thoughts on this?
I agree with modifying comments as per the discussion above.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I haven't fully fleshed it out in my head yet, but if we are to update the metadata we need to block completions while we do it, but if it turns out we can't, we shouldn't consider it an unrecoverable failure, unless we cannot get an ack quorum.

public static final long INVALID_LEDGER_ID = -0xABCDABCDL;

final AtomicInteger blockAddCompletions = new AtomicInteger(0);
final Object metadataLock = new Object();

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm guessing that the reason for this lock is to prevent write completion events from contending on the LedgerHandle object with write initiation? Is that contention really enough of a problem for this to be a measurable win? If so, I think that the state protected by this lock (changingEnsemble, delayedWriteFailedBookies) should be moved into an actual object with descriptive methods. As it is, it's a bit tough to infer the update rules.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it's not actually needed because everything it protects is run on the orderedexecutor thread for this ledger. A lock was requested in a previous patch though until the threading model is clarified.
#1621 (comment)

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Well, for that purpose, couldn't we just use the existing LedgerHandle object until we're sure about the single thread/ledger implementation (which I'd assume would mean pervasive assertions with testing)?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd prefer not to use the handle itself, as that's an object that's exposed to clients, so clients could try to lock on it in a callback.

The single thread/ledger thing would take a couple of forms, but the biggest pieces would be 1) making sure a ledger handle instance is only given one executor 2) making sure all callbacks from bookie client are run on that one executor & 3) asserting that Thread.currentThread().getId() matches the ID of the executor given to the handle.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Well, that ship would seem to have sailed since we lock it elsewhere, right?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right now, all the writes, write responses and metadata changes on write are done through orderedexecutor (lid) right? Where is the need for lock serialization?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Well, that ship would seem to have sailed since we lock it elsewhere, right?

That's true, but I don't want to throw fuel on that fire.

Right now, all the writes, write responses and metadata changes on write are done through orderedexecutor (lid) right? Where is the need for lock serialization?

There isn't (I don't think), but I would like another mechanism in place to ensure safety before removing it.

List<BookieSocketAddress> newEnsemble = getCurrentEnsemble();
Set<Integer> replaced = EnsembleUtils.diffEnsemble(origEnsemble, newEnsemble);
unsetSuccessAndSendWriteRequest(newEnsemble, replaced);
changingEnsemble = false;

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we have to call sendAddSuccessCallbacks after calling unsetSucccessAndSendWriteRequest. It's possible that every pendingAddOp had a bookie in the write set swapped, but nevertheless all have ackQuorum satisfied. In that case, PendingAddOp.unsetSuccessAndSendWriteRequest will not call sendAddSuccessCallbacks(), but we will still have writes ready to go. In fact, I think the call in PendingAddOp.unsetSuccessAndSendWriteRequest is entirely unnecessary and should be removed in favor of a single call here. As a side effect, you can call it outside of the metadataLock avoiding that problem as well.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree. Will change this.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree. Will change this.

PendingAddOp.unsetSuccessAndSendWriteRequest() will do a sendWriteRequest() even if the AQ satisfied. When that write request response comes back irrespective of a pass/fail it will do sendAddSuccessCallbacks().

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jvrao you're right. This stuff jumps far too much between LedgerHandle and PendingAddOp. I'll leave this for now.

@athanatos
Copy link

@sijie @ivankelly Ok, I'm done reviewing for now, I left some comments/questions.

@sijie
Copy link
Member

sijie commented Sep 11, 2018

thank you @athanatos

Copy link
Contributor Author

@ivankelly ivankelly left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@sijie @athanatos Thanks for the reviews.

There's a few things that need to be settled before i push a new patch (see my comments)

break;
}
}
}
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will readd. It's not a final summary however, it's the change that we wish to make. It's not final until the zookeeper write completes.

List<BookieSocketAddress> lastEnsemble = metadata.getLastEnsembleValue();
boolean failedBookieInEnsemble = failedBookies.entrySet().stream()
.anyMatch((e) -> lastEnsemble.get(e.getKey()).equals(e.getValue()));
return !metadata.isClosed() && !metadata.isInRecovery() && failedBookieInEnsemble;
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure

// the ledger isn't closed between checking and
// updating lastAddPushed
if (getLedgerMetadata().isClosed()) {
if (getLedgerMetadata().isClosed() || closing) {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will do

}

LedgerMetadataBuilder withWriteQuorumSize(int writeQuorumSize) {
checkArgument(ensembleSize >= writeQuorumSize, "Write quorum must be less or equal to ensemble size");
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok


// should still be able to close as long as recovery closed the ledger
// with the same last entryId and length as in the write handle.
writeLh.close();
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@sijie the previous behaviour is not documented, nor is it well defined. In some cases a metadata version exception allowed a close to succeed, and in others it did not. I would not expect any application is relying on this behaviour, and if they are, they are probably broken in many other ways.
I can revert this my putting throwing an exception in the Predicate part of the loop if the metadata is closed. There'd be no guarantee that behaviour is still exactly matching though, because it isn't well defined currently.

public static final long INVALID_LEDGER_ID = -0xABCDABCDL;

final AtomicInteger blockAddCompletions = new AtomicInteger(0);
final Object metadataLock = new Object();
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it's not actually needed because everything it protects is run on the orderedexecutor thread for this ledger. A lock was requested in a previous patch though until the threading model is clarified.
#1621 (comment)

private LedgerMetadata metadata;
final long ledgerId;
long lastAddPushed;
boolean closing;
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@sijie with mutable metadata, when close happens, we updated the state in the local metadata immediately. So any call to isClosed would be true if the client was currently closing.

@athanatos yes, I was thinking of adding an explicit handleState enum. Can do now.

final State prevState;
List<PendingAddOp> pendingAdds;

if (isClosed()) {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@athanatos if we're listening for metadata updates, it's possible someone else updated the metadata and closed it just before we close it. However, i think isClosed is wrong here as it confuses two things. What we care about in this check is whether the handle is closed or not, so using an enum for the handle state would work better.

.lastEntry().getValue().get(replacedBookieIdx);
replaced &= !Objects.equal(replacedBookieAddr, failedBookieAddr);
List<BookieSocketAddress> origEnsemble = getCurrentEnsemble();
ensembleChangeLoop(origEnsemble, toReplace);
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, I can move it out. There shouldn't be a problem with callbacks under metadatalock though, as it's only protected a few members. Maybe deadlocks could be an issue down the line.

List<BookieSocketAddress> newEnsemble = getCurrentEnsemble();
Set<Integer> replaced = EnsembleUtils.diffEnsemble(origEnsemble, newEnsemble);
unsetSuccessAndSendWriteRequest(newEnsemble, replaced);
changingEnsemble = false;
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree. Will change this.


static Set<Integer> diffEnsemble(List<BookieSocketAddress> e1,
List<BookieSocketAddress> e2) {
checkArgument(e1.size() == e2.size(), "Ensembles must be of same size");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How is this better than Assert?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

asserts get turned off by default at runtime.

}
metadata.addEnsemble(newEnsembleStartEntry, newEnsemble);
void notifyWriteFailed(int index, BookieSocketAddress addr) {
synchronized (metadataLock) {
Copy link
Contributor

@jvrao jvrao Sep 15, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What are we protecting it from? from ReadOnlyLedgerHandle??? In the current code, even that will go through the same orderedexecutor right?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

From our own past and future misdeeds :)

We should replace this with checkState(Thread.currentThread().getId() == myThread), but that's a separate change. synchronization that doesn't hit contention is cheap anyhow, it only touches a thread local afair.

LOG.info("[EnsembleChange-L{}-{}] : resolved ledger metadata conflict and writing to zookeeper,"
+ " local meta data is \n {} \n, zk meta data is \n {}.",
ledgerId, ensembleChangeIdx, metadata, newMeta);
LOG.debug("Ledger {} reaches max allowed ensemble change number {}",
Copy link
Contributor

@jvrao jvrao Sep 15, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It will be nice to have this at INFO level as it gives good idea on why we are failing the write.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata);
long newEnsembleStartEntry = getLastAddConfirmed() + 1;
checkState(lastEnsembleKey <= newEnsembleStartEntry,
"New ensemble must either replace the last ensemble, or add a new one");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please print the lastEnsembleKey and newEnsembleStartEntry here;
With this check state what happens? Does the write fail?
Also, when can this happen? If we are here we don't have any outstanding metadata update in the flight.
For the ledger L1 with Ensemble (B1, B2, B3) when LAC is at X; X+1 write failed on Bookie B1. The ensemble is changed to (B4,B2,B3) and then we noticed that B2 also failed. In that case, do we end up in this situation?
But that may be OK state right? why are you doing checkState?

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe that this cannot be violated if the protocol is operating correctly (the only other way for an ensemble to be a fence, but that's rules out by the previous argument). The checkState call would appear to be a statement that this case is impossible? The == case happens when we got a new failure since we sent the ensemble change (which worked), so we have to replace it. @ivankelly Is that right?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jvrao @athanatos's understanding is correct. The checkState is an assertion.

checkState(lastEnsembleKey <= newEnsembleStartEntry,
"New ensemble must either replace the last ensemble, or add a new one");
if (lastEnsembleKey.equals(newEnsembleStartEntry)) {
return builder.replaceEnsembleEntry(newEnsembleStartEntry, newEnsemble).build();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm; so the above checkSate should be < instead of <=??

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, <, if we fail to write at all to an ensemble, it can be replaced.

@ivankelly
Copy link
Contributor Author

rerun bookkeeper-server client tests

@ivankelly
Copy link
Contributor Author

@dlg99 this is still failing on MDC stuff. The problem is that if the runnable is submitted from a thread without MDC, such as the zookeeper callback threads, they won't get the context. Not sure of how to solve this without a huge amount of plumbing :/

Though now it only prints when the new ensemble has been persisted.
Also disabled the MDC test check for this log because waiting for
persistence breaks the MDC in a way that's non-trivial to fix.
@ivankelly
Copy link
Contributor Author

@dlg99 actually, think about this more, it's a general problem with the MDC approach. It worked before because "New Ensemble" was being printed before actually trying to write to zookeeper, do the request hadn't lefts the ordered executor thread pools. Once the request goes to zookeeper, the MDC context is lost. For now, I've commented out that check.

@ivankelly
Copy link
Contributor Author

rerun bookkeeper-server bookie tests

@sijie
Copy link
Member

sijie commented Oct 4, 2018

@athanatos @jvrao can you review @ivankelly 's latest change?

@ivankelly
Copy link
Contributor Author

@sijie @athanatos @jvrao could you all take another look at this? I'm eager to get this changeset out of my queue

@ivankelly
Copy link
Contributor Author

@sijie @athanatos @jvrao pinging again

@ivankelly
Copy link
Contributor Author

rerun java8 tests
rerun integration tests

@ivankelly
Copy link
Contributor Author

rerun integration tests

@ivankelly
Copy link
Contributor Author

rerun java8 tests

@ivankelly
Copy link
Contributor Author

rebuild java8

2 similar comments
@ivankelly
Copy link
Contributor Author

rebuild java8

@ivankelly
Copy link
Contributor Author

rebuild java8

@ivankelly
Copy link
Contributor Author

@sijie @jvrao @athanatos weekly reminder to please take another look at this

@athanatos
Copy link

@ivankelly Looks like you've addressed my comments, LGTM

@ivankelly
Copy link
Contributor Author

@athanatos thanks. if you're happy with the change could you mark it as approved?

Copy link
Contributor

@jvrao jvrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great work. LGTM

@sijie sijie added this to the 4.9.0 milestone Oct 25, 2018
@sijie sijie merged commit 6bf6971 into apache:master Oct 25, 2018
@ivankelly
Copy link
Contributor Author

@sijie @jvrao @athanatos thanks for the reviews guys. Now that this is in there's a bunch of small cleanup patches that also need to go in to enforce the immutability. They'll be much much smaller though.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants