|
@@ -13,39 +13,45 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
|
|
|
import org.elasticsearch.ElasticsearchException;
|
|
|
import org.elasticsearch.Version;
|
|
|
import org.elasticsearch.action.ActionListener;
|
|
|
-import org.elasticsearch.cluster.ClusterStatePublicationEvent;
|
|
|
+import org.elasticsearch.action.ActionListenerResponseHandler;
|
|
|
+import org.elasticsearch.action.support.ChannelActionListener;
|
|
|
import org.elasticsearch.cluster.ClusterState;
|
|
|
+import org.elasticsearch.cluster.ClusterStatePublicationEvent;
|
|
|
import org.elasticsearch.cluster.Diff;
|
|
|
import org.elasticsearch.cluster.IncompatibleClusterStateVersionException;
|
|
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
|
|
-import org.elasticsearch.common.bytes.BytesReference;
|
|
|
+import org.elasticsearch.common.bytes.ReleasableBytesReference;
|
|
|
import org.elasticsearch.common.compress.Compressor;
|
|
|
import org.elasticsearch.common.compress.CompressorFactory;
|
|
|
+import org.elasticsearch.common.io.Streams;
|
|
|
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
|
|
import org.elasticsearch.common.io.stream.InputStreamStreamInput;
|
|
|
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
|
|
|
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
|
|
import org.elasticsearch.common.io.stream.OutputStreamStreamOutput;
|
|
|
+import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
|
|
|
import org.elasticsearch.common.io.stream.StreamInput;
|
|
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
|
|
+import org.elasticsearch.common.util.BigArrays;
|
|
|
+import org.elasticsearch.common.util.LazyInitializable;
|
|
|
+import org.elasticsearch.core.AbstractRefCounted;
|
|
|
+import org.elasticsearch.core.Releasables;
|
|
|
import org.elasticsearch.core.internal.io.IOUtils;
|
|
|
import org.elasticsearch.threadpool.ThreadPool;
|
|
|
import org.elasticsearch.transport.BytesTransportRequest;
|
|
|
-import org.elasticsearch.transport.TransportChannel;
|
|
|
import org.elasticsearch.transport.TransportException;
|
|
|
import org.elasticsearch.transport.TransportRequestOptions;
|
|
|
import org.elasticsearch.transport.TransportResponse;
|
|
|
-import org.elasticsearch.transport.TransportResponseHandler;
|
|
|
import org.elasticsearch.transport.TransportService;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Map;
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
import java.util.function.BiConsumer;
|
|
|
-import java.util.function.Consumer;
|
|
|
import java.util.function.Function;
|
|
|
|
|
|
public class PublicationTransportHandler {
|
|
@@ -55,6 +61,7 @@ public class PublicationTransportHandler {
|
|
|
public static final String PUBLISH_STATE_ACTION_NAME = "internal:cluster/coordination/publish_state";
|
|
|
public static final String COMMIT_STATE_ACTION_NAME = "internal:cluster/coordination/commit_state";
|
|
|
|
|
|
+ private final BigArrays bigArrays;
|
|
|
private final TransportService transportService;
|
|
|
private final NamedWriteableRegistry namedWriteableRegistry;
|
|
|
private final Function<PublishRequest, PublishWithJoinResponse> handlePublishRequest;
|
|
@@ -76,9 +83,14 @@ public class PublicationTransportHandler {
|
|
|
private static final TransportRequestOptions STATE_REQUEST_OPTIONS =
|
|
|
TransportRequestOptions.of(null, TransportRequestOptions.Type.STATE);
|
|
|
|
|
|
- public PublicationTransportHandler(TransportService transportService, NamedWriteableRegistry namedWriteableRegistry,
|
|
|
- Function<PublishRequest, PublishWithJoinResponse> handlePublishRequest,
|
|
|
- BiConsumer<ApplyCommitRequest, ActionListener<Void>> handleApplyCommit) {
|
|
|
+ public PublicationTransportHandler(
|
|
|
+ BigArrays bigArrays,
|
|
|
+ TransportService transportService,
|
|
|
+ NamedWriteableRegistry namedWriteableRegistry,
|
|
|
+ Function<PublishRequest, PublishWithJoinResponse> handlePublishRequest,
|
|
|
+ BiConsumer<ApplyCommitRequest, ActionListener<Void>> handleApplyCommit
|
|
|
+ ) {
|
|
|
+ this.bigArrays = bigArrays;
|
|
|
this.transportService = transportService;
|
|
|
this.namedWriteableRegistry = namedWriteableRegistry;
|
|
|
this.handlePublishRequest = handlePublishRequest;
|
|
@@ -88,31 +100,9 @@ public class PublicationTransportHandler {
|
|
|
|
|
|
transportService.registerRequestHandler(COMMIT_STATE_ACTION_NAME, ThreadPool.Names.GENERIC, false, false,
|
|
|
ApplyCommitRequest::new,
|
|
|
- (request, channel, task) -> handleApplyCommit.accept(request, transportCommitCallback(channel)));
|
|
|
- }
|
|
|
-
|
|
|
- private ActionListener<Void> transportCommitCallback(TransportChannel channel) {
|
|
|
- return new ActionListener<Void>() {
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onResponse(Void aVoid) {
|
|
|
- try {
|
|
|
- channel.sendResponse(TransportResponse.Empty.INSTANCE);
|
|
|
- } catch (IOException e) {
|
|
|
- logger.debug("failed to send response on commit", e);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onFailure(Exception e) {
|
|
|
- try {
|
|
|
- channel.sendResponse(e);
|
|
|
- } catch (IOException ie) {
|
|
|
- e.addSuppressed(ie);
|
|
|
- logger.debug("failed to send response on commit", e);
|
|
|
- }
|
|
|
- }
|
|
|
- };
|
|
|
+ (request, channel, task) -> handleApplyCommit.accept(
|
|
|
+ request,
|
|
|
+ new ChannelActionListener<>(channel, COMMIT_STATE_ACTION_NAME, request).map(r -> TransportResponse.Empty.INSTANCE)));
|
|
|
}
|
|
|
|
|
|
public PublishClusterStateStats stats() {
|
|
@@ -197,50 +187,96 @@ public class PublicationTransportHandler {
|
|
|
|
|
|
public PublicationContext newPublicationContext(ClusterStatePublicationEvent clusterStatePublicationEvent) {
|
|
|
final PublicationContext publicationContext = new PublicationContext(clusterStatePublicationEvent);
|
|
|
-
|
|
|
- // Build the serializations we expect to need now, early in the process, so that an error during serialization fails the publication
|
|
|
- // straight away. This isn't watertight since we send diffs on a best-effort basis and may fall back to sending a full state (and
|
|
|
- // therefore serializing it) if the diff-based publication fails.
|
|
|
- publicationContext.buildDiffAndSerializeStates();
|
|
|
- return publicationContext;
|
|
|
+ boolean success = false;
|
|
|
+ try {
|
|
|
+ // Build the serializations we expect to need now, early in the process, so that an error during serialization fails the
|
|
|
+ // publication straight away. This isn't watertight since we send diffs on a best-effort basis and may fall back to sending a
|
|
|
+ // full state (and therefore serializing it) if the diff-based publication fails.
|
|
|
+ publicationContext.buildDiffAndSerializeStates();
|
|
|
+ success = true;
|
|
|
+ return publicationContext;
|
|
|
+ } finally {
|
|
|
+ if (success == false) {
|
|
|
+ publicationContext.decRef();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- private static BytesReference serializeFullClusterState(ClusterState clusterState, Version nodeVersion) throws IOException {
|
|
|
- final BytesStreamOutput bStream = new BytesStreamOutput();
|
|
|
- try (StreamOutput stream = new OutputStreamStreamOutput(CompressorFactory.COMPRESSOR.threadLocalOutputStream(bStream))) {
|
|
|
- stream.setVersion(nodeVersion);
|
|
|
- stream.writeBoolean(true);
|
|
|
- clusterState.writeTo(stream);
|
|
|
+ private ReleasableBytesReference serializeFullClusterState(ClusterState clusterState, DiscoveryNode node) {
|
|
|
+ final Version nodeVersion = node.getVersion();
|
|
|
+ final BytesStreamOutput bytesStream = new ReleasableBytesStreamOutput(bigArrays);
|
|
|
+ boolean success = false;
|
|
|
+ try {
|
|
|
+ try (StreamOutput stream = new OutputStreamStreamOutput(
|
|
|
+ CompressorFactory.COMPRESSOR.threadLocalOutputStream(Streams.flushOnCloseStream(bytesStream)))
|
|
|
+ ) {
|
|
|
+ stream.setVersion(nodeVersion);
|
|
|
+ stream.writeBoolean(true);
|
|
|
+ clusterState.writeTo(stream);
|
|
|
+ } catch (IOException e) {
|
|
|
+ throw new ElasticsearchException("failed to serialize cluster state for publishing to node {}", e, node);
|
|
|
+ }
|
|
|
+ final ReleasableBytesReference result = new ReleasableBytesReference(bytesStream.bytes(), bytesStream::close);
|
|
|
+ logger.trace(
|
|
|
+ "serialized full cluster state version [{}] for node version [{}] with size [{}]",
|
|
|
+ clusterState.version(),
|
|
|
+ nodeVersion,
|
|
|
+ result.length());
|
|
|
+ success = true;
|
|
|
+ return result;
|
|
|
+ } finally {
|
|
|
+ if (success == false) {
|
|
|
+ bytesStream.close();
|
|
|
+ }
|
|
|
}
|
|
|
- final BytesReference serializedState = bStream.bytes();
|
|
|
- logger.trace("serialized full cluster state version [{}] for node version [{}] with size [{}]",
|
|
|
- clusterState.version(), nodeVersion, serializedState.length());
|
|
|
- return serializedState;
|
|
|
}
|
|
|
|
|
|
- private static BytesReference serializeDiffClusterState(Diff<ClusterState> diff, Version nodeVersion) throws IOException {
|
|
|
- final BytesStreamOutput bStream = new BytesStreamOutput();
|
|
|
- try (StreamOutput stream = new OutputStreamStreamOutput(CompressorFactory.COMPRESSOR.threadLocalOutputStream(bStream))) {
|
|
|
- stream.setVersion(nodeVersion);
|
|
|
- stream.writeBoolean(false);
|
|
|
- diff.writeTo(stream);
|
|
|
+ private ReleasableBytesReference serializeDiffClusterState(long clusterStateVersion, Diff<ClusterState> diff, DiscoveryNode node) {
|
|
|
+ final Version nodeVersion = node.getVersion();
|
|
|
+ final BytesStreamOutput bytesStream = new ReleasableBytesStreamOutput(bigArrays);
|
|
|
+ boolean success = false;
|
|
|
+ try {
|
|
|
+ try (StreamOutput stream = new OutputStreamStreamOutput(
|
|
|
+ CompressorFactory.COMPRESSOR.threadLocalOutputStream(Streams.flushOnCloseStream(bytesStream)))
|
|
|
+ ) {
|
|
|
+ stream.setVersion(nodeVersion);
|
|
|
+ stream.writeBoolean(false);
|
|
|
+ diff.writeTo(stream);
|
|
|
+ } catch (IOException e) {
|
|
|
+ throw new ElasticsearchException("failed to serialize cluster state diff for publishing to node {}", e, node);
|
|
|
+ }
|
|
|
+ final ReleasableBytesReference result = new ReleasableBytesReference(bytesStream.bytes(), bytesStream::close);
|
|
|
+ logger.trace(
|
|
|
+ "serialized cluster state diff for version [{}] for node version [{}] with size [{}]",
|
|
|
+ clusterStateVersion,
|
|
|
+ nodeVersion,
|
|
|
+ result.length());
|
|
|
+ success = true;
|
|
|
+ return result;
|
|
|
+ } finally {
|
|
|
+ if (success == false) {
|
|
|
+ bytesStream.close();
|
|
|
+ }
|
|
|
}
|
|
|
- return bStream.bytes();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Publishing a cluster state typically involves sending the same cluster state (or diff) to every node, so the work of diffing,
|
|
|
* serializing, and compressing the state can be done once and the results shared across publish requests. The
|
|
|
- * {@code PublicationContext} implements this sharing.
|
|
|
+ * {@code PublicationContext} implements this sharing. It's ref-counted: the initial reference is released by the coordinator when
|
|
|
+ * a state (or diff) has been sent to every node, every transmitted diff also holds a reference in case it needs to retry with a full
|
|
|
+ * state.
|
|
|
*/
|
|
|
- public class PublicationContext {
|
|
|
+ public class PublicationContext extends AbstractRefCounted {
|
|
|
|
|
|
private final DiscoveryNodes discoveryNodes;
|
|
|
private final ClusterState newState;
|
|
|
private final ClusterState previousState;
|
|
|
private final boolean sendFullVersion;
|
|
|
- private final Map<Version, BytesReference> serializedStates = new HashMap<>();
|
|
|
- private final Map<Version, BytesReference> serializedDiffs = new HashMap<>();
|
|
|
+
|
|
|
+ // All the values of these maps have one ref for the context (while it's open) and one for each in-flight message.
|
|
|
+ private final Map<Version, ReleasableBytesReference> serializedStates = new ConcurrentHashMap<>();
|
|
|
+ private final Map<Version, ReleasableBytesReference> serializedDiffs = new HashMap<>();
|
|
|
|
|
|
PublicationContext(ClusterStatePublicationEvent clusterStatePublicationEvent) {
|
|
|
discoveryNodes = clusterStatePublicationEvent.getNewState().nodes();
|
|
@@ -250,33 +286,25 @@ public class PublicationTransportHandler {
|
|
|
}
|
|
|
|
|
|
void buildDiffAndSerializeStates() {
|
|
|
- Diff<ClusterState> diff = null;
|
|
|
+ assert refCount() > 0;
|
|
|
+ final LazyInitializable<Diff<ClusterState>, RuntimeException> diffSupplier
|
|
|
+ = new LazyInitializable<>(() -> newState.diff(previousState));
|
|
|
for (DiscoveryNode node : discoveryNodes) {
|
|
|
- try {
|
|
|
- if (sendFullVersion || previousState.nodes().nodeExists(node) == false) {
|
|
|
- if (serializedStates.containsKey(node.getVersion()) == false) {
|
|
|
- serializedStates.put(node.getVersion(), serializeFullClusterState(newState, node.getVersion()));
|
|
|
- }
|
|
|
- } else {
|
|
|
- // will send a diff
|
|
|
- if (diff == null) {
|
|
|
- diff = newState.diff(previousState);
|
|
|
- }
|
|
|
- if (serializedDiffs.containsKey(node.getVersion()) == false) {
|
|
|
- final BytesReference serializedDiff = serializeDiffClusterState(diff, node.getVersion());
|
|
|
- serializedDiffs.put(node.getVersion(), serializedDiff);
|
|
|
- logger.trace("serialized cluster state diff for version [{}] in for node version [{}] with size [{}]",
|
|
|
- newState.version(), node.getVersion(), serializedDiff.length());
|
|
|
- }
|
|
|
- }
|
|
|
- } catch (IOException e) {
|
|
|
- throw new ElasticsearchException("failed to serialize cluster state for publishing to node {}", e, node);
|
|
|
+ if (sendFullVersion || previousState.nodes().nodeExists(node) == false) {
|
|
|
+ serializedStates.computeIfAbsent(
|
|
|
+ node.getVersion(),
|
|
|
+ v -> serializeFullClusterState(newState, node));
|
|
|
+ } else {
|
|
|
+ serializedDiffs.computeIfAbsent(
|
|
|
+ node.getVersion(),
|
|
|
+ v -> serializeDiffClusterState(newState.version(), diffSupplier.getOrCompute(), node));
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
public void sendPublishRequest(DiscoveryNode destination, PublishRequest publishRequest,
|
|
|
ActionListener<PublishWithJoinResponse> listener) {
|
|
|
+ assert refCount() > 0;
|
|
|
assert publishRequest.getAcceptedState() == newState : "state got switched on us";
|
|
|
assert transportService.getThreadPool().getThreadContext().isSystemContext();
|
|
|
final ActionListener<PublishWithJoinResponse> responseActionListener;
|
|
@@ -314,37 +342,22 @@ public class PublicationTransportHandler {
|
|
|
public void sendApplyCommit(DiscoveryNode destination, ApplyCommitRequest applyCommitRequest,
|
|
|
ActionListener<TransportResponse.Empty> listener) {
|
|
|
assert transportService.getThreadPool().getThreadContext().isSystemContext();
|
|
|
- transportService.sendRequest(destination, COMMIT_STATE_ACTION_NAME, applyCommitRequest, STATE_REQUEST_OPTIONS,
|
|
|
- new TransportResponseHandler<TransportResponse.Empty>() {
|
|
|
-
|
|
|
- @Override
|
|
|
- public TransportResponse.Empty read(StreamInput in) {
|
|
|
- return TransportResponse.Empty.INSTANCE;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void handleResponse(TransportResponse.Empty response) {
|
|
|
- listener.onResponse(response);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void handleException(TransportException exp) {
|
|
|
- listener.onFailure(exp);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public String executor() {
|
|
|
- return ThreadPool.Names.GENERIC;
|
|
|
- }
|
|
|
- });
|
|
|
+ transportService.sendRequest(
|
|
|
+ destination,
|
|
|
+ COMMIT_STATE_ACTION_NAME,
|
|
|
+ applyCommitRequest,
|
|
|
+ STATE_REQUEST_OPTIONS,
|
|
|
+ new ActionListenerResponseHandler<>(listener, in -> TransportResponse.Empty.INSTANCE, ThreadPool.Names.GENERIC));
|
|
|
}
|
|
|
|
|
|
private void sendFullClusterState(DiscoveryNode destination, ActionListener<PublishWithJoinResponse> listener) {
|
|
|
- BytesReference bytes = serializedStates.get(destination.getVersion());
|
|
|
+ assert refCount() > 0;
|
|
|
+ ReleasableBytesReference bytes = serializedStates.get(destination.getVersion());
|
|
|
if (bytes == null) {
|
|
|
try {
|
|
|
- bytes = serializeFullClusterState(newState, destination.getVersion());
|
|
|
- serializedStates.put(destination.getVersion(), bytes);
|
|
|
+ bytes = serializedStates.computeIfAbsent(
|
|
|
+ destination.getVersion(),
|
|
|
+ v -> serializeFullClusterState(newState, destination));
|
|
|
} catch (Exception e) {
|
|
|
logger.warn(() -> new ParameterizedMessage(
|
|
|
"failed to serialize cluster state before publishing it to node {}", destination), e);
|
|
@@ -352,58 +365,71 @@ public class PublicationTransportHandler {
|
|
|
return;
|
|
|
}
|
|
|
}
|
|
|
- sendClusterState(destination, bytes, false, listener);
|
|
|
+ sendClusterState(destination, bytes, listener);
|
|
|
}
|
|
|
|
|
|
private void sendClusterStateDiff(DiscoveryNode destination, ActionListener<PublishWithJoinResponse> listener) {
|
|
|
- final BytesReference bytes = serializedDiffs.get(destination.getVersion());
|
|
|
+ final ReleasableBytesReference bytes = serializedDiffs.get(destination.getVersion());
|
|
|
assert bytes != null
|
|
|
: "failed to find serialized diff for node " + destination + " of version [" + destination.getVersion() + "]";
|
|
|
- sendClusterState(destination, bytes, true, listener);
|
|
|
- }
|
|
|
|
|
|
- private void sendClusterState(DiscoveryNode destination, BytesReference bytes, boolean retryWithFullClusterStateOnFailure,
|
|
|
- ActionListener<PublishWithJoinResponse> listener) {
|
|
|
- try {
|
|
|
- final BytesTransportRequest request = new BytesTransportRequest(bytes, destination.getVersion());
|
|
|
- final Consumer<TransportException> transportExceptionHandler = exp -> {
|
|
|
- if (retryWithFullClusterStateOnFailure && exp.unwrapCause() instanceof IncompatibleClusterStateVersionException) {
|
|
|
- logger.debug("resending full cluster state to node {} reason {}", destination, exp.getDetailedMessage());
|
|
|
- sendFullClusterState(destination, listener);
|
|
|
- } else {
|
|
|
- logger.debug(() -> new ParameterizedMessage("failed to send cluster state to {}", destination), exp);
|
|
|
- listener.onFailure(exp);
|
|
|
+ // acquire a ref to the context just in case we need to try again with the full cluster state
|
|
|
+ if (tryIncRef() == false) {
|
|
|
+ assert false;
|
|
|
+ listener.onFailure(new IllegalStateException("publication context released before transmission"));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ sendClusterState(destination, bytes, ActionListener.runAfter(listener.delegateResponse((delegate, e) -> {
|
|
|
+ if (e instanceof TransportException) {
|
|
|
+ final TransportException transportException = (TransportException) e;
|
|
|
+ if (transportException.unwrapCause() instanceof IncompatibleClusterStateVersionException) {
|
|
|
+ logger.debug(() -> new ParameterizedMessage(
|
|
|
+ "resending full cluster state to node {} reason {}",
|
|
|
+ destination,
|
|
|
+ transportException.getDetailedMessage()));
|
|
|
+ sendFullClusterState(destination, delegate);
|
|
|
+ return;
|
|
|
}
|
|
|
- };
|
|
|
- final TransportResponseHandler<PublishWithJoinResponse> responseHandler =
|
|
|
- new TransportResponseHandler<PublishWithJoinResponse>() {
|
|
|
-
|
|
|
- @Override
|
|
|
- public PublishWithJoinResponse read(StreamInput in) throws IOException {
|
|
|
- return new PublishWithJoinResponse(in);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void handleResponse(PublishWithJoinResponse response) {
|
|
|
- listener.onResponse(response);
|
|
|
- }
|
|
|
+ }
|
|
|
|
|
|
- @Override
|
|
|
- public void handleException(TransportException exp) {
|
|
|
- transportExceptionHandler.accept(exp);
|
|
|
- }
|
|
|
+ logger.debug(new ParameterizedMessage("failed to send cluster state to {}", destination), e);
|
|
|
+ delegate.onFailure(e);
|
|
|
+ }), this::decRef));
|
|
|
+ }
|
|
|
|
|
|
- @Override
|
|
|
- public String executor() {
|
|
|
- return ThreadPool.Names.GENERIC;
|
|
|
- }
|
|
|
- };
|
|
|
- transportService.sendRequest(destination, PUBLISH_STATE_ACTION_NAME, request, STATE_REQUEST_OPTIONS, responseHandler);
|
|
|
+ private void sendClusterState(
|
|
|
+ DiscoveryNode destination,
|
|
|
+ ReleasableBytesReference bytes,
|
|
|
+ ActionListener<PublishWithJoinResponse> listener
|
|
|
+ ) {
|
|
|
+ assert refCount() > 0;
|
|
|
+ if (bytes.tryIncRef() == false) {
|
|
|
+ assert false;
|
|
|
+ listener.onFailure(new IllegalStateException("serialized cluster state released before transmission"));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ transportService.sendRequest(
|
|
|
+ destination,
|
|
|
+ PUBLISH_STATE_ACTION_NAME,
|
|
|
+ new BytesTransportRequest(bytes, destination.getVersion()),
|
|
|
+ STATE_REQUEST_OPTIONS,
|
|
|
+ new ActionListenerResponseHandler<PublishWithJoinResponse>(
|
|
|
+ ActionListener.runAfter(listener, bytes::decRef),
|
|
|
+ PublishWithJoinResponse::new,
|
|
|
+ ThreadPool.Names.GENERIC));
|
|
|
} catch (Exception e) {
|
|
|
+ assert false : e;
|
|
|
logger.warn(() -> new ParameterizedMessage("error sending cluster state to {}", destination), e);
|
|
|
listener.onFailure(e);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void closeInternal() {
|
|
|
+ serializedDiffs.values().forEach(Releasables::closeExpectNoException);
|
|
|
+ serializedStates.values().forEach(Releasables::closeExpectNoException);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
}
|