Skip to content
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

Zen2: Cluster state publication pipeline #32584

Merged
merged 20 commits into from
Aug 7, 2018
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,348 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.cluster.coordination;

import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.Discovery.AckListener;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportResponse;

import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.function.LongSupplier;

public abstract class Publication extends AbstractComponent {

private final List<PublicationTarget> publicationTargets;
private final PublishRequest publishRequest;
private final AckListener ackListener;
private final LongSupplier currentTimeSupplier;
private final long startTime;

private Optional<ApplyCommitRequest> applyCommitRequest; // set when state is committed
private boolean isCompleted; // set when publication is completed
private boolean timedOut; // set when publication timed out

public Publication(Settings settings, PublishRequest publishRequest, AckListener ackListener, LongSupplier currentTimeSupplier) {
super(settings);
this.publishRequest = publishRequest;
this.ackListener = ackListener;
this.currentTimeSupplier = currentTimeSupplier;
startTime = currentTimeSupplier.getAsLong();
applyCommitRequest = Optional.empty();
publicationTargets = new ArrayList<>(publishRequest.getAcceptedState().getNodes().getNodes().size());
publishRequest.getAcceptedState().getNodes().iterator().forEachRemaining(n -> publicationTargets.add(new PublicationTarget(n)));
}

public void start(Set<DiscoveryNode> faultyNodes) {
logger.trace("publishing {} to {}", publishRequest, publicationTargets);

for (final DiscoveryNode faultyNode : faultyNodes) {
onFaultyNode(faultyNode);
}
onPossibleCommitFailure();
publicationTargets.forEach(PublicationTarget::sendPublishRequest);
}

public void onTimeout() {
assert timedOut == false;
timedOut = true;
if (applyCommitRequest.isPresent() == false) {
logger.debug("onTimeout: [{}] timed out before committing", this);
// fail all current publications
final Exception e = new ElasticsearchException("publication timed out before committing");
publicationTargets.stream().filter(PublicationTarget::isActive).forEach(pt -> pt.setFailed(e));
}
onPossibleCompletion();
}

public void onFaultyNode(DiscoveryNode faultyNode) {
publicationTargets.forEach(t -> t.onFaultyNode(faultyNode));
onPossibleCompletion();
}

private void onPossibleCompletion() {
if (isCompleted) {
return;
}

if (timedOut == false) {
for (final PublicationTarget target : publicationTargets) {
if (target.isActive()) {
return;
}
}
}

if (applyCommitRequest.isPresent() == false) {
logger.debug("onPossibleCompletion: [{}] commit failed", this);
assert isCompleted == false;
isCompleted = true;
onCompletion(false);
return;
}

assert isCompleted == false;
isCompleted = true;
onCompletion(true);
assert applyCommitRequest.isPresent();
logger.trace("onPossibleCompletion: [{}] was successful", this);
}

// For assertions only: verify that this invariant holds
private boolean publicationCompletedIffAllTargetsInactiveOrTimedOut() {
if (timedOut == false) {
for (final PublicationTarget target : publicationTargets) {
if (target.isActive()) {
return isCompleted == false;
}
}
}
return isCompleted;
}

private void onPossibleCommitFailure() {
if (applyCommitRequest.isPresent()) {
onPossibleCompletion();
return;
}

final CoordinationState.VoteCollection possiblySuccessfulNodes = new CoordinationState.VoteCollection();
for (PublicationTarget publicationTarget : publicationTargets) {
if (publicationTarget.mayCommitInFuture()) {
possiblySuccessfulNodes.addVote(publicationTarget.discoveryNode);
} else {
assert publicationTarget.isFailed() : publicationTarget;
}
}

if (isPublishQuorum(possiblySuccessfulNodes) == false) {
logger.debug("onPossibleCommitFailure: non-failed nodes {} do not form a quorum, so {} cannot succeed",
possiblySuccessfulNodes, this);
Exception e = new Discovery.FailedToCommitClusterStateException("non-failed nodes do not form a quorum");
publicationTargets.stream().filter(PublicationTarget::isActive).forEach(pt -> pt.setFailed(e));
onPossibleCompletion();
}
}

protected abstract void onCompletion(boolean committed);

protected abstract boolean isPublishQuorum(CoordinationState.VoteCollection votes);

protected abstract Optional<ApplyCommitRequest> handlePublishResponse(DiscoveryNode sourceNode, PublishResponse publishResponse);

protected abstract void onPossibleJoin(DiscoveryNode sourceNode, PublishWithJoinResponse response);
Copy link
Contributor

Choose a reason for hiding this comment

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

Does this need the whole PublishWithJoinResponse or could it just take the Optional<Join>? Or maybe even the Join only if present?

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 don't know yet. I've added a TODO in 43e3399


protected abstract void sendPublishRequest(DiscoveryNode destination, PublishRequest publishRequest,
ActionListener<PublishWithJoinResponse> responseActionListener);

protected abstract void sendApplyCommit(DiscoveryNode destination, ApplyCommitRequest applyCommit,
ActionListener<TransportResponse.Empty> responseActionListener);

@Override
public String toString() {
return "Publication{term=" + publishRequest.getAcceptedState().term() +
", version=" + publishRequest.getAcceptedState().version() + '}';
}

enum PublicationTargetState {
NOT_STARTED,
FAILED,
SENT_PUBLISH_REQUEST,
WAITING_FOR_QUORUM,
SENT_APPLY_COMMIT,
APPLIED_COMMIT,
}

class PublicationTarget {
private final DiscoveryNode discoveryNode;
private boolean ackIsPending = true;
private PublicationTargetState state = PublicationTargetState.NOT_STARTED;

PublicationTarget(DiscoveryNode discoveryNode) {
this.discoveryNode = discoveryNode;
}

@Override
public String toString() {
return "PublicationTarget{" +
"discoveryNode=" + discoveryNode +
", state=" + state +
", ackIsPending=" + ackIsPending +
'}';
}

void sendPublishRequest() {
if (isFailed()) {
return;
}
assert state == PublicationTargetState.NOT_STARTED : state + " -> " + PublicationTargetState.SENT_PUBLISH_REQUEST;
state = PublicationTargetState.SENT_PUBLISH_REQUEST;
Publication.this.sendPublishRequest(discoveryNode, publishRequest, new PublishResponseHandler());
// TODO Can this ^ fail with an exception? Target should be failed if so.
assert publicationCompletedIffAllTargetsInactiveOrTimedOut();
}

void handlePublishResponse(PublishResponse publishResponse) {
assert isWaitingForQuorum() : this;
logger.trace("handlePublishResponse: handling [{}] from [{}])", publishResponse, discoveryNode);
if (applyCommitRequest.isPresent()) {
sendApplyCommit();
} else {
Publication.this.handlePublishResponse(discoveryNode, publishResponse).ifPresent(applyCommit -> {
assert applyCommitRequest.isPresent() == false;
applyCommitRequest = Optional.of(applyCommit);
ackListener.onCommit(TimeValue.timeValueMillis(currentTimeSupplier.getAsLong() - startTime));
publicationTargets.stream().filter(PublicationTarget::isWaitingForQuorum).forEach(PublicationTarget::sendApplyCommit);
});
}
}

void sendApplyCommit() {
assert state == PublicationTargetState.WAITING_FOR_QUORUM : state + " -> " + PublicationTargetState.SENT_APPLY_COMMIT;
state = PublicationTargetState.SENT_APPLY_COMMIT;
assert applyCommitRequest.isPresent();
Publication.this.sendApplyCommit(discoveryNode, applyCommitRequest.get(), new ApplyCommitResponseHandler());
assert publicationCompletedIffAllTargetsInactiveOrTimedOut();
}

void setAppliedCommit() {
assert state == PublicationTargetState.SENT_APPLY_COMMIT : state + " -> " + PublicationTargetState.APPLIED_COMMIT;
state = PublicationTargetState.APPLIED_COMMIT;
ackOnce(null);
}

void setFailed(Exception e) {
assert state != PublicationTargetState.APPLIED_COMMIT : state + " -> " + PublicationTargetState.FAILED;
state = PublicationTargetState.FAILED;
ackOnce(e);
}

void onFaultyNode(DiscoveryNode faultyNode) {
if (isActive() && discoveryNode.equals(faultyNode)) {
logger.debug("onFaultyNode: [{}] is faulty, failing target in publication {}", faultyNode, Publication.this);
setFailed(new ElasticsearchException("faulty node"));
onPossibleCommitFailure();
}
}

private void ackOnce(Exception e) {
if (ackIsPending) {
ackIsPending = false;
ackListener.onNodeAck(discoveryNode, e);
}
}

boolean isActive() {
return state != PublicationTargetState.FAILED
&& state != PublicationTargetState.APPLIED_COMMIT;
}

boolean isWaitingForQuorum() {
return state == PublicationTargetState.WAITING_FOR_QUORUM;
}

boolean mayCommitInFuture() {
return (state == PublicationTargetState.NOT_STARTED
|| state == PublicationTargetState.SENT_PUBLISH_REQUEST
|| state == PublicationTargetState.WAITING_FOR_QUORUM);
}

boolean isFailed() {
return state == PublicationTargetState.FAILED;
}

private class PublishResponseHandler implements ActionListener<PublishWithJoinResponse> {

@Override
public void onResponse(PublishWithJoinResponse response) {
if (isFailed()) {
logger.debug("PublishResponseHandler.handleResponse: already failed, ignoring response from [{}]", discoveryNode);
assert publicationCompletedIffAllTargetsInactiveOrTimedOut();
return;
}

// TODO: check if we need to pass the full response here or if it's sufficient to just pass the optional join.
onPossibleJoin(discoveryNode, response);

assert state == PublicationTargetState.SENT_PUBLISH_REQUEST : state + " -> " + PublicationTargetState.WAITING_FOR_QUORUM;
state = PublicationTargetState.WAITING_FOR_QUORUM;
handlePublishResponse(response.getPublishResponse());

assert publicationCompletedIffAllTargetsInactiveOrTimedOut();
}

@Override
public void onFailure(Exception e) {
assert e instanceof TransportException;
final TransportException exp = (TransportException) e;
if (exp.getRootCause() instanceof CoordinationStateRejectedException) {
logger.debug("PublishResponseHandler: [{}] failed: {}", discoveryNode, exp.getRootCause().getMessage());
} else {
logger.debug(() -> new ParameterizedMessage("PublishResponseHandler: [{}] failed", discoveryNode), exp);
}
assert ((TransportException) e).getRootCause() instanceof Exception;
setFailed((Exception) exp.getRootCause());
onPossibleCommitFailure();
assert publicationCompletedIffAllTargetsInactiveOrTimedOut();
}

}

private class ApplyCommitResponseHandler implements ActionListener<TransportResponse.Empty> {

@Override
public void onResponse(TransportResponse.Empty ignored) {
if (isFailed()) {
logger.debug("ApplyCommitResponseHandler.handleResponse: already failed, ignoring response from [{}]",
discoveryNode);
return;
}
setAppliedCommit();
onPossibleCompletion();
assert publicationCompletedIffAllTargetsInactiveOrTimedOut();
}

@Override
public void onFailure(Exception e) {
assert e instanceof TransportException;
final TransportException exp = (TransportException) e;
if (exp.getRootCause() instanceof CoordinationStateRejectedException) {
logger.debug("ApplyCommitResponseHandler: [{}] failed: {}", discoveryNode, exp.getRootCause().getMessage());
} else {
logger.debug(() -> new ParameterizedMessage("ApplyCommitResponseHandler: [{}] failed", discoveryNode), exp);
}
assert ((TransportException) e).getRootCause() instanceof Exception;
setFailed((Exception) exp.getRootCause());
onPossibleCompletion();
assert publicationCompletedIffAllTargetsInactiveOrTimedOut();
}
}
}
}
Loading