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

SOLR-17537: Manage ZK Compression through Curator #2849

Draft
wants to merge 3 commits into
base: main
Choose a base branch
from
Draft
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
Expand Up @@ -58,7 +58,6 @@
import org.apache.solr.common.util.ReflectMapWriter;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.ZLibCompressor;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
Expand Down Expand Up @@ -86,9 +85,6 @@ public class SolrZkClient implements Closeable {

private final ZkMetrics metrics = new ZkMetrics();

private Compressor compressor;
private int minStateByteLenForCompression;

public MapWriter getMetrics() {
return metrics::writeMap;
}
Expand Down Expand Up @@ -199,6 +195,7 @@ private SolrZkClient(
.authorization(zkCredentialsProvider.getCredentials())
.retryPolicy(retryPolicy)
.runSafeService(curatorSafeServiceExecutor)
.compressionProvider(new SolrZkCompressionProvider(compressor, minStateByteLenForCompression))
.build();
if (onReconnect != null) {
client
Expand Down Expand Up @@ -228,18 +225,6 @@ private SolrZkClient(
}

assert ObjectReleaseTracker.track(this);
if (aclProvider == null) {
this.aclProvider = useDefaultCredsAndACLs ? createACLProvider() : new DefaultZkACLProvider();
} else {
this.aclProvider = aclProvider;
}

if (compressor == null) {
this.compressor = new ZLibCompressor();
} else {
this.compressor = compressor;
}
this.minStateByteLenForCompression = minStateByteLenForCompression;
}

public CuratorFramework getCuratorFramework() {
Expand Down Expand Up @@ -444,18 +429,6 @@ public byte[] getData(
.storingStatIn(stat)
.usingWatcher(wrapWatcher(watcher))
.forPath(path));
if (compressor.isCompressedBytes(result)) {
log.debug("Zookeeper data at path {} is compressed", path);
try {
result = compressor.decompressBytes(result);
} catch (Exception e) {
throw new SolrException(
SolrException.ErrorCode.SERVER_ERROR,
String.format(
Locale.ROOT, "Unable to decompress data at path: %s from zookeeper", path),
e);
}
}
metrics.reads.increment();
if (result != null) {
metrics.bytesRead.add(result.length);
Expand All @@ -470,16 +443,11 @@ public NodeData getNode(final String path, Watcher watcher, boolean retryOnConnL
}

/** Returns node's state */
public Stat setData(final String path, byte data[], final int version, boolean retryOnConnLoss)
public Stat setData(final String path, final byte[] data, final int version, boolean retryOnConnLoss)
throws KeeperException, InterruptedException {
if (SolrZkClient.shouldCompressData(data, path, minStateByteLenForCompression)) {
// state.json should be compressed before being put to ZK
data = compressor.compressBytes(data);
}
final byte[] finalData = data;
Stat result =
runWithCorrectThrows(
"setting data", () -> client.setData().withVersion(version).forPath(path, finalData));
"setting data", () -> client.setData().withVersion(version).forPath(path, data));
metrics.writes.increment();
if (data != null) {
metrics.bytesWritten.add(data.length);
Expand Down Expand Up @@ -675,11 +643,6 @@ public void makePath(
createBuilder.orSetData();
}

if (SolrZkClient.shouldCompressData(data, path, minStateByteLenForCompression)) {
// state.json should be compressed before being put to ZK
data = compressor.compressBytes(data);
}

metrics.writes.increment();
if (data != null) {
metrics.bytesWritten.add(data.length);
Expand Down Expand Up @@ -1353,14 +1316,4 @@ public SolrZkClient build() {
return new SolrZkClient(this);
}
}

static boolean shouldCompressData(byte[] data, String path, int minStateByteLenForCompression) {
if (path.endsWith("state.json")
&& minStateByteLenForCompression > -1
&& data.length > minStateByteLenForCompression) {
// state.json should be compressed before being put to ZK
return true;
}
return false;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
package org.apache.solr.common.cloud;

import org.apache.curator.framework.api.CompressionProvider;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.Compressor;
import org.apache.solr.common.util.ZLibCompressor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.lang.invoke.MethodHandles;
import java.util.Locale;

public class SolrZkCompressionProvider implements CompressionProvider {

private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

private final Compressor compressor;
private final int minStateByteLenForCompression;

public SolrZkCompressionProvider(Compressor compressor, int minStateByteLenForCompression) {
this.compressor = compressor != null ? compressor : new ZLibCompressor();
this.minStateByteLenForCompression = minStateByteLenForCompression;
}

@Override
public byte[] compress(String path, byte[] data) throws Exception {
if (path.endsWith("state.json")
&& minStateByteLenForCompression > -1
&& data.length >= minStateByteLenForCompression) {
// state.json should be compressed before being put to ZK
return compressor.compressBytes(data);
} else {
return data;
}
}

@Override
public byte[] decompress(String path, byte[] compressedData) throws SolrException {
if (compressor.isCompressedBytes(compressedData)) {
log.info("Zookeeper data at path {} is compressed", path);
try {
return compressor.decompressBytes(compressedData);
} catch (Exception e) {
throw new SolrException(
SolrException.ErrorCode.SERVER_ERROR,
String.format(
Locale.ROOT, "Unable to decompress data at path: %s from zookeeper", path),
e);
}
} else {
return compressedData;
}
}
}
Loading