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

Optimize checksum creation for remote cluster state #16046

Merged
merged 11 commits into from
Oct 1, 2024
Merged
Changes from 2 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
Expand Up @@ -27,6 +27,12 @@
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import java.util.function.Function;

import com.jcraft.jzlib.JZlib;

Expand All @@ -50,6 +56,7 @@ public class ClusterStateChecksum implements ToXContentFragment, Writeable {
static final String INDICES_CS = "indices_md";
private static final String CLUSTER_STATE_CS = "cluster_state";
private static final int CHECKSUM_SIZE = 8;
private static final int COMPONENT_SIZE = 11;
private static final Logger logger = LogManager.getLogger(ClusterStateChecksum.class);

long routingTableChecksum;
Expand All @@ -66,61 +73,143 @@ public class ClusterStateChecksum implements ToXContentFragment, Writeable {
long clusterStateChecksum;

public ClusterStateChecksum(ClusterState clusterState) {
long start = System.currentTimeMillis();
// keeping thread pool size to number of components.
ExecutorService executorService = Executors.newFixedThreadPool(COMPONENT_SIZE);
CountDownLatch latch = new CountDownLatch(COMPONENT_SIZE);

executeChecksumTask((stream) -> {
try {
clusterState.routingTable().writeVerifiableTo(stream);
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to create checksum for routing table", e);
}
return null;
}, checksum -> routingTableChecksum = checksum, executorService, latch);

executeChecksumTask((stream) -> {
try {
clusterState.nodes().writeVerifiableTo(stream);
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to create checksum for discovery nodes", e);
}
return null;
}, checksum -> nodesChecksum = checksum, executorService, latch);

himshikha marked this conversation as resolved.
Show resolved Hide resolved
executeChecksumTask((stream) -> {
try {
clusterState.coordinationMetadata().writeVerifiableTo(stream);
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to create checksum for coordination metadata", e);
}
return null;
}, checksum -> coordinationMetadataChecksum = checksum, executorService, latch);

executeChecksumTask((stream) -> {
try {
Settings.writeSettingsToStream(clusterState.metadata().persistentSettings(), stream);
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to create checksum for settings metadata", e);
}
return null;
}, checksum -> settingMetadataChecksum = checksum, executorService, latch);

executeChecksumTask((stream) -> {
try {
Settings.writeSettingsToStream(clusterState.metadata().transientSettings(), stream);
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to create checksum for transient settings metadata", e);
}
return null;
}, checksum -> transientSettingsMetadataChecksum = checksum, executorService, latch);

executeChecksumTask((stream) -> {
try {
clusterState.metadata().templatesMetadata().writeVerifiableTo(stream);
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to create checksum for templates metadata", e);
}
return null;
}, checksum -> templatesMetadataChecksum = checksum, executorService, latch);

executeChecksumTask((stream) -> {
try {
stream.writeStringCollection(clusterState.metadata().customs().keySet());
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to create checksum for customs metadata", e);
}
return null;
}, checksum -> customMetadataMapChecksum = checksum, executorService, latch);

executeChecksumTask((stream) -> {
try {
((DiffableStringMap) clusterState.metadata().hashesOfConsistentSettings()).writeTo(stream);
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to create checksum for hashesOfConsistentSettings", e);
}
return null;
}, checksum -> hashesOfConsistentSettingsChecksum = checksum, executorService, latch);

executeChecksumTask((stream) -> {
try {
stream.writeMapValues(
clusterState.metadata().indices(),
(checksumStream, value) -> value.writeVerifiableTo((BufferedChecksumStreamOutput) checksumStream)
);
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to create checksum for indices metadata", e);
}
return null;
}, checksum -> indicesChecksum = checksum, executorService, latch);

executeChecksumTask((stream) -> {
try {
clusterState.blocks().writeVerifiableTo(stream);
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to create checksum for cluster state blocks", e);
}
return null;
}, checksum -> blocksChecksum = checksum, executorService, latch);

executeChecksumTask((stream) -> {
try {
stream.writeStringCollection(clusterState.customs().keySet());
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to create checksum for cluster state customs", e);
}
return null;
}, checksum -> clusterStateCustomsChecksum = checksum, executorService, latch);

executorService.shutdown();
try {
latch.await();
} catch (InterruptedException e) {
throw new RemoteStateTransferException("Failed to create checksum for cluster state.", e);
}
createClusterStateChecksum();
logger.debug("Checksum execution time {}", System.currentTimeMillis() - start);
himshikha marked this conversation as resolved.
Show resolved Hide resolved
}

private void executeChecksumTask(Function<BufferedChecksumStreamOutput, Void> checksumTask, Consumer<Long> checksumConsumer, ExecutorService executorService, CountDownLatch latch) {
executorService.execute(() -> {
try {
long checksum = createChecksum(checksumTask);
checksumConsumer.accept(checksum);
latch.countDown();
} catch (IOException e) {
throw new RemoteStateTransferException("Failed to execute checksum task", e);
}
});
}

private long createChecksum(Function<BufferedChecksumStreamOutput, Void> o) throws IOException {
try (
BytesStreamOutput out = new BytesStreamOutput();
BufferedChecksumStreamOutput checksumOut = new BufferedChecksumStreamOutput(out)
) {
clusterState.routingTable().writeVerifiableTo(checksumOut);
routingTableChecksum = checksumOut.getChecksum();

checksumOut.reset();
clusterState.nodes().writeVerifiableTo(checksumOut);
nodesChecksum = checksumOut.getChecksum();

checksumOut.reset();
clusterState.coordinationMetadata().writeVerifiableTo(checksumOut);
coordinationMetadataChecksum = checksumOut.getChecksum();

// Settings create sortedMap by default, so no explicit sorting required here.
checksumOut.reset();
Settings.writeSettingsToStream(clusterState.metadata().persistentSettings(), checksumOut);
settingMetadataChecksum = checksumOut.getChecksum();

checksumOut.reset();
Settings.writeSettingsToStream(clusterState.metadata().transientSettings(), checksumOut);
transientSettingsMetadataChecksum = checksumOut.getChecksum();

checksumOut.reset();
clusterState.metadata().templatesMetadata().writeVerifiableTo(checksumOut);
templatesMetadataChecksum = checksumOut.getChecksum();

checksumOut.reset();
checksumOut.writeStringCollection(clusterState.metadata().customs().keySet());
customMetadataMapChecksum = checksumOut.getChecksum();

checksumOut.reset();
((DiffableStringMap) clusterState.metadata().hashesOfConsistentSettings()).writeTo(checksumOut);
hashesOfConsistentSettingsChecksum = checksumOut.getChecksum();

checksumOut.reset();
checksumOut.writeMapValues(
clusterState.metadata().indices(),
(stream, value) -> value.writeVerifiableTo((BufferedChecksumStreamOutput) stream)
);
indicesChecksum = checksumOut.getChecksum();

checksumOut.reset();
clusterState.blocks().writeVerifiableTo(checksumOut);
blocksChecksum = checksumOut.getChecksum();

checksumOut.reset();
checksumOut.writeStringCollection(clusterState.customs().keySet());
clusterStateCustomsChecksum = checksumOut.getChecksum();
} catch (IOException e) {
logger.error("Failed to create checksum for cluster state.", e);
throw new RemoteStateTransferException("Failed to create checksum for cluster state.", e);
o.apply(checksumOut);
return checksumOut.getChecksum();
}
createClusterStateChecksum();
}

private void createClusterStateChecksum() {
Expand Down
Loading