Skip to content

[client-v2] Compression Support #1761

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

Merged
merged 7 commits into from
Aug 6, 2024
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
5 changes: 5 additions & 0 deletions client-v2/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,11 @@
<artifactId>lz4-pure-java</artifactId>
<optional>true</optional>
</dependency>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>org.apache.commons.compress</artifactId>
<version>${repackaged.version}</version>
</dependency>
<!-- https://mvnrepository.com/artifact/com.fasterxml.jackson.core/jackson-core -->
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
Expand Down
39 changes: 37 additions & 2 deletions client-v2/src/main/java/com/clickhouse/client/api/Client.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import com.clickhouse.client.api.insert.InsertSettings;
import com.clickhouse.client.api.insert.POJOSerializer;
import com.clickhouse.client.api.insert.SerializerNotFoundException;
import com.clickhouse.client.api.internal.ClickHouseLZ4OutputStream;
import com.clickhouse.client.api.internal.ClientStatisticsHolder;
import com.clickhouse.client.api.internal.ClientV1AdaptorHelper;
import com.clickhouse.client.api.internal.HttpAPIClientHelper;
Expand All @@ -34,11 +35,14 @@
import com.clickhouse.client.config.ClickHouseClientOption;
import com.clickhouse.client.config.ClickHouseDefaults;
import com.clickhouse.client.http.ClickHouseHttpProto;
import com.clickhouse.client.http.config.ClickHouseHttpOption;
import com.clickhouse.data.ClickHouseColumn;
import com.clickhouse.data.ClickHouseDataStreamFactory;
import com.clickhouse.data.ClickHouseFormat;
import com.clickhouse.data.ClickHousePipedOutputStream;
import com.clickhouse.data.format.BinaryStreamUtils;
import org.apache.commons.compress.compressors.lz4.BlockLZ4CompressorOutputStream;
import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorOutputStream;
import org.apache.hc.core5.concurrent.DefaultThreadFactory;
import org.apache.hc.core5.http.ClassicHttpResponse;
import org.apache.hc.core5.http.HttpStatus;
Expand Down Expand Up @@ -413,6 +417,32 @@ public Builder compressClientRequest(boolean enabled) {
return this;
}

/**
* Configures the client to use HTTP compression. In this case compression is controlled by
* http headers. Client compression will set {@code Content-Encoding: lz4} header and server
* compression will set {@code Accept-Encoding: lz4} header.
*
* @param enabled - indicates if http compression is enabled
* @return
*/
public Builder useHttpCompression(boolean enabled) {
this.configuration.put("client.use_http_compression", String.valueOf(enabled));
return this;
}

/**
* Sets buffer size for uncompressed data in LZ4 compression.
* For outgoing data it is the size of a buffer that will be compressed.
* For incoming data it is the size of a buffer that will be decompressed.
*
* @param size - size of the buffer in bytes
* @return
*/
public Builder setLZ4UncompressedBufferSize(int size) {
this.configuration.put("compression.lz4.uncompressed_buffer_size", String.valueOf(size));
return this;
}

/**
* Sets the default database name that will be used by operations if not specified.
* @param database - actual default database name.
Expand Down Expand Up @@ -571,6 +601,10 @@ private Map<String, String> setDefaults(Map<String, String> userConfig) {
String.valueOf(ClickHouseClientOption.MAX_THREADS_PER_CLIENT.getDefaultValue()));
}

if (!userConfig.containsKey("compression.lz4.uncompressed_buffer_size")) {
userConfig.put("compression.lz4.uncompressed_buffer_size",
String.valueOf(ClickHouseLZ4OutputStream.UNCOMPRESSED_BUFF_SIZE));
}
return userConfig;
}
}
Expand Down Expand Up @@ -764,6 +798,7 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
}
}
}
out.close();
})) {


Expand Down Expand Up @@ -882,7 +917,7 @@ public CompletableFuture<InsertResponse> insert(String tableName,
while ((bytesRead = data.read(buffer)) != -1) {
out.write(buffer, 0, bytesRead);
}
out.flush();
out.close();
})) {


Expand Down Expand Up @@ -1042,7 +1077,7 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
ClassicHttpResponse httpResponse =
httpClientHelper.executeRequest(selectedNode, finalSettings.getAllSettings(), output -> {
output.write(sqlQuery.getBytes(StandardCharsets.UTF_8));
output.flush();
output.close();
});

// Check response
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,174 @@
package com.clickhouse.client.api.internal;

import com.clickhouse.client.api.ClientException;
import com.clickhouse.data.ClickHouseByteUtils;
import com.clickhouse.data.ClickHouseCityHash;
import com.clickhouse.data.ClickHouseUtils;
import net.jpountz.lz4.LZ4FastDecompressor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;

public class ClickHouseLZ4InputStream extends InputStream {

private static Logger LOG = LoggerFactory.getLogger(ClickHouseLZ4InputStream.class);
private final LZ4FastDecompressor decompressor;

private final InputStream in;

private ByteBuffer buffer;

private byte[] tmpBuffer = new byte[1];


public ClickHouseLZ4InputStream(InputStream in, LZ4FastDecompressor decompressor, int bufferSize) {
super();
this.decompressor = decompressor;
this.in = in;
this.buffer = ByteBuffer.allocate(bufferSize);
this.buffer.limit(0);
}

@Override
public int read() throws IOException {
int n = read(tmpBuffer, 0, 1);
return n == -1 ? -1 : tmpBuffer[0] & 0xFF;
}

@Override
public int read(byte[] b, int off, int len) throws IOException {
if (b == null) {
throw new NullPointerException("b is null");
} else if (off < 0) {
throw new IndexOutOfBoundsException("off is negative");
} else if (len < 0) {
throw new IndexOutOfBoundsException("len is negative");
} else if (off + len > b.length) {
throw new IndexOutOfBoundsException("off + len is greater than b.length");
} else if (len == 0) {
return 0;
}

int readBytes = 0;
do {
int remaining = Math.min(len - readBytes, buffer.remaining());
buffer.get(b, off + readBytes, remaining);
readBytes += remaining;
} while (readBytes < len && refill() != -1);

return readBytes == 0 ? -1 : readBytes;
}


static final byte MAGIC = (byte) 0x82;
static final int HEADER_LENGTH = 25;

static final byte[] headerBuff = new byte[HEADER_LENGTH];

/**
* Method ensures to read all bytes from the input stream.
* In case of network connection it may be a case when not all bytes are read at once.
* @throws IOException
*/
private boolean readFully(byte[] b, int off, int len) throws IOException {
int n = 0;
while (n < len) {
int count = in.read(b, off + n, len - n);
if (count < 0) {
if (n == 0) {
return false;
}
throw new IOException(ClickHouseUtils.format("Incomplete read: {0} of {1}", n, len));
}
n += count;
}

return true;
}

private int refill() throws IOException {

// read header
boolean readFully = readFully(headerBuff, 0, HEADER_LENGTH);
if (!readFully) {
return -1;
}

if (headerBuff[16] != MAGIC) {
// 1 byte - 0x82 (shows this is LZ4)
throw new ClientException("Invalid LZ4 magic byte: '" + headerBuff[16] + "'");
}

// 4 bytes - size of the compressed data including 9 bytes of the header
int compressedSizeWithHeader = getInt32(headerBuff, 17);
// 4 bytes - size of uncompressed data
int uncompressedSize = getInt32(headerBuff, 21);

int offset = 9;
final byte[] block = new byte[compressedSizeWithHeader];
block[0] = MAGIC;
setInt32(block, 1, compressedSizeWithHeader);
setInt32(block, 5, uncompressedSize);
// compressed data: compressed_size - 9 bytes
int remaining = compressedSizeWithHeader - offset;

readFully = readFully(block, offset, remaining);
if (!readFully) {
throw new EOFException("Unexpected end of stream");
}

long[] real = ClickHouseCityHash.cityHash128(block, 0, compressedSizeWithHeader);
if (real[0] != getInt64(headerBuff, 0) || real[1] != ClickHouseByteUtils.getInt64(headerBuff, 8)) {
throw new ClientException("Corrupted stream: checksum mismatch");
}

if (buffer.capacity() < uncompressedSize) {
buffer = ByteBuffer.allocate(uncompressedSize);
LOG.warn("Buffer size is too small, reallocate buffer with size: " + uncompressedSize);
}
decompressor.decompress(ByteBuffer.wrap(block), offset, buffer, 0, uncompressedSize);
buffer.position(0);
buffer.limit(uncompressedSize);
return uncompressedSize;
}

/**
* Read int32 Little Endian
* @param bytes
* @param offset
* @return
*/
static int getInt32(byte[] bytes, int offset) {
return (0xFF & bytes[offset]) | ((0xFF & bytes[offset + 1]) << 8) | ((0xFF & bytes[offset + 2]) << 16)
| ((0xFF & bytes[offset + 3]) << 24);
}

/**
* Read int64 Little Endian
* @param bytes
* @param offset
* @return
*/
static long getInt64(byte[] bytes, int offset) {
return (0xFFL & bytes[offset]) | ((0xFFL & bytes[offset + 1]) << 8) | ((0xFFL & bytes[offset + 2]) << 16)
| ((0xFFL & bytes[offset + 3]) << 24) | ((0xFFL & bytes[offset + 4]) << 32)
| ((0xFFL & bytes[offset + 5]) << 40) | ((0xFFL & bytes[offset + 6]) << 48)
| ((0xFFL & bytes[offset + 7]) << 56);
}

static void setInt32(byte[] bytes, int offset, int value) {
bytes[offset] = (byte) (0xFF & value);
bytes[offset + 1] = (byte) (0xFF & (value >> 8));
bytes[offset + 2] = (byte) (0xFF & (value >> 16));
bytes[offset + 3] = (byte) (0xFF & (value >> 24));
}

@Override
public void close() throws IOException {
super.close();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,101 @@
package com.clickhouse.client.api.internal;

import com.clickhouse.data.ClickHouseCityHash;
import net.jpountz.lz4.LZ4Compressor;

import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;

public class ClickHouseLZ4OutputStream extends OutputStream {

public static final int UNCOMPRESSED_BUFF_SIZE = 8192;

private final ByteBuffer buffer;

private final OutputStream out;

private final LZ4Compressor compressor;

private byte tmpBuffer[] = new byte[1];

private final ByteBuffer compressedBuffer;

private static int HEADER_LEN = 15; // 9 bytes for header, 6 bytes for checksum


public ClickHouseLZ4OutputStream(OutputStream out, LZ4Compressor compressor, int bufferSize) {
super();
this.buffer = ByteBuffer.allocate(bufferSize);
this.out = out;
this.compressor = compressor;
this.compressedBuffer = ByteBuffer.allocate(compressor.maxCompressedLength(buffer.capacity()) + HEADER_LEN);
}

@Override
public void write(int b) throws IOException {
tmpBuffer[0] = (byte) b;
write(tmpBuffer, 0, 1);
}

@Override
public void write( byte[] b, int off, int len) throws IOException {
if (b == null) {
throw new NullPointerException("b is null");
} else if (off < 0) {
throw new IndexOutOfBoundsException("off is negative");
} else if (len < 0) {
throw new IndexOutOfBoundsException("len is negative");
} else if (off + len > b.length) {
throw new IndexOutOfBoundsException("off + len is greater than b.length");
} else if (len == 0) {
return;
}

int writtenBytes = 0;
do {
int remaining = Math.min(len - writtenBytes, buffer.remaining());
buffer.put(b, off + writtenBytes, remaining);
writtenBytes += remaining;
if (buffer.remaining() == 0) {
flush();
}
} while (writtenBytes < len);
}

@Override
public void flush() throws IOException {
compressedBuffer.clear();
compressedBuffer.put(16, ClickHouseLZ4InputStream.MAGIC);
int uncompressedLen = buffer.position();
buffer.flip();
int compressed = compressor.compress(buffer, 0, uncompressedLen, compressedBuffer, 25,
compressedBuffer.remaining() - 25);
int compressedSizeWithHeader = compressed + 9;
ClickHouseLZ4InputStream.setInt32(compressedBuffer.array(), 17, compressedSizeWithHeader); // compressed size with header
ClickHouseLZ4InputStream.setInt32(compressedBuffer.array(), 21, uncompressedLen); // uncompressed size
long[] hash = ClickHouseCityHash.cityHash128(compressedBuffer.array(), 16, compressedSizeWithHeader);
setInt64(compressedBuffer.array(), 0, hash[0]);
setInt64(compressedBuffer.array(), 8, hash[1]);
compressedBuffer.flip();
out.write(compressedBuffer.array(), 0, compressed + 25);
buffer.clear();
}


static void setInt64(byte[] bytes, int offset, long value) {
bytes[offset] = (byte) (0xFF & value);
bytes[offset + 1] = (byte) (0xFF & (value >> 8));
bytes[offset + 2] = (byte) (0xFF & (value >> 16));
bytes[offset + 3] = (byte) (0xFF & (value >> 24));
bytes[offset + 4] = (byte) (0xFF & (value >> 32));
bytes[offset + 5] = (byte) (0xFF & (value >> 40));
bytes[offset + 6] = (byte) (0xFF & (value >> 48));
bytes[offset + 7] = (byte) (0xFF & (value >> 56));
}
@Override
public void close() throws IOException {
flush();
out.close();
}
}
Loading