Skip to content

Commit 262b3eb

Browse files
committed
Optimize LZ4 input and output stream
1 parent 1f6666a commit 262b3eb

9 files changed

Lines changed: 316 additions & 215 deletions

File tree

clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java

Lines changed: 146 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,8 @@
1010
import java.util.concurrent.BlockingQueue;
1111
import java.util.concurrent.TimeUnit;
1212

13+
import com.clickhouse.client.config.ClickHouseClientOption;
14+
1315
/**
1416
* Extended input stream for read optimization.
1517
*/
@@ -184,11 +186,17 @@ public long skip(long n) throws IOException {
184186

185187
static final class WrappedInputStream extends ClickHouseInputStream {
186188
private final InputStream in;
189+
private final byte[] buffer;
187190

191+
private int position;
192+
private int limit;
188193
private boolean closed;
189194

190-
WrappedInputStream(InputStream input) {
195+
WrappedInputStream(InputStream input, int bufferSize) {
191196
in = ClickHouseChecker.nonNull(input, "InputStream");
197+
buffer = new byte[bufferSize];
198+
position = 0;
199+
limit = 0;
192200
closed = false;
193201
}
194202

@@ -198,26 +206,34 @@ private void ensureOpen() throws IOException {
198206
}
199207
}
200208

209+
private int updateBuffer() throws IOException {
210+
if (closed) {
211+
return -1;
212+
}
213+
214+
position = 0;
215+
int count = in.read(buffer);
216+
limit = count > 0 ? count : 0;
217+
return count;
218+
}
219+
201220
@Override
202221
public int available() throws IOException {
203-
return !closed ? in.available() : 0;
222+
return !closed && (position < limit || updateBuffer() > 0) ? limit - position : 0;
204223
}
205224

206225
@Override
207226
public byte readByte() throws IOException {
208-
ensureOpen();
209-
210-
int v = in.read();
211-
if (v != -1) {
212-
return (byte) v;
227+
if (position >= limit && updateBuffer() < 0) {
228+
try {
229+
close();
230+
} catch (IOException e) {
231+
// ignore
232+
}
233+
throw new EOFException();
213234
}
214235

215-
try {
216-
close();
217-
} catch (IOException e) {
218-
// ignore
219-
}
220-
throw new EOFException();
236+
return buffer[position++];
221237
}
222238

223239
@Override
@@ -232,26 +248,125 @@ public void close() throws IOException {
232248
in.close();
233249
} finally {
234250
closed = true;
251+
position = 0;
252+
limit = 0;
235253
}
236254
}
237255
}
238256

239257
@Override
240258
public int read() throws IOException {
241259
ensureOpen();
242-
return in.read();
260+
261+
int value = -1;
262+
if (position < limit || updateBuffer() > 0) {
263+
value = 0xFF & buffer[position++];
264+
}
265+
return value;
243266
}
244267

245268
@Override
246269
public int read(byte[] b, int off, int len) throws IOException {
270+
if (position >= limit && updateBuffer() < 0) {
271+
return -1;
272+
}
273+
247274
ensureOpen();
248-
return in.read(b, off, len);
275+
276+
int counter = 0;
277+
while (counter < len) {
278+
int size = Math.min(limit - position, len - counter);
279+
System.arraycopy(buffer, position, b, off, size);
280+
position += size;
281+
off += size;
282+
counter += size;
283+
284+
if (position >= limit && updateBuffer() < 0) {
285+
break;
286+
}
287+
}
288+
289+
return counter;
290+
}
291+
292+
@Override
293+
public byte[] readBytes(int length) throws IOException {
294+
if (length <= 0) {
295+
return EMPTY_BYTES;
296+
}
297+
298+
ensureOpen();
299+
300+
byte[] bytes = new byte[length];
301+
int offset = 0;
302+
int counter = 0;
303+
while (counter < length) {
304+
if (position >= limit && updateBuffer() < 0) {
305+
try {
306+
close();
307+
} catch (IOException e) {
308+
// ignore
309+
}
310+
throw counter == 0 ? new EOFException()
311+
: new IOException(ClickHouseUtils
312+
.format("Reached end of input stream after reading %d of %d bytes", counter,
313+
bytes.length));
314+
}
315+
316+
int size = Math.min(limit - position, length - counter);
317+
System.arraycopy(buffer, position, bytes, offset, size);
318+
position += size;
319+
offset += size;
320+
counter += size;
321+
}
322+
323+
return bytes;
324+
}
325+
326+
@Override
327+
public String readString(int byteLength, Charset charset) throws IOException {
328+
ensureOpen();
329+
330+
if (byteLength < 1) {
331+
return "";
332+
}
333+
334+
if (charset == null) {
335+
charset = StandardCharsets.UTF_8;
336+
}
337+
338+
if (limit - position > byteLength) {
339+
int offset = position;
340+
position += byteLength;
341+
return new String(buffer, offset, byteLength, charset);
342+
}
343+
344+
return new String(readBytes(byteLength), charset);
249345
}
250346

251347
@Override
252348
public long skip(long n) throws IOException {
253349
ensureOpen();
254-
return in.skip(n);
350+
351+
long counter = 0L;
352+
while (n > 0L) {
353+
if (position >= limit && updateBuffer() < 0) {
354+
break;
355+
} else {
356+
int remain = limit - position;
357+
if (n > remain) {
358+
n -= remain;
359+
counter += remain;
360+
position = limit;
361+
} else {
362+
counter += n;
363+
position += n;
364+
n = 0L;
365+
}
366+
}
367+
}
368+
369+
return counter;
255370
}
256371
}
257372

@@ -274,7 +389,21 @@ public static ClickHouseInputStream of(BlockingQueue<ByteBuffer> queue, int time
274389
* {@link ClickHouseInputStream}
275390
*/
276391
public static ClickHouseInputStream of(InputStream input) {
277-
return input instanceof ClickHouseInputStream ? (ClickHouseInputStream) input : new WrappedInputStream(input);
392+
return of(input, (int) ClickHouseClientOption.MAX_BUFFER_SIZE.getDefaultValue());
393+
}
394+
395+
/**
396+
* Wraps the given input stream.
397+
*
398+
* @param input non-null input stream
399+
* @param bufferSize buffer size which is always greater than zero(usually 4096
400+
* or larger)
401+
* @return wrapped input, or the same input if it's instance of
402+
* {@link ClickHouseInputStream}
403+
*/
404+
public static ClickHouseInputStream of(InputStream input, int bufferSize) {
405+
return input instanceof ClickHouseInputStream ? (ClickHouseInputStream) input
406+
: new WrappedInputStream(input, bufferSize);
278407
}
279408

280409
/**

clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java

Lines changed: 29 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -91,15 +91,33 @@ private static <T extends Enum<T>> T toEnum(int value, Class<T> enumType) {
9191
}
9292

9393
public static int toInt32(byte[] bytes, int offset) {
94-
return (0xFF & bytes[offset]) | ((0xFF & bytes[offset + 1]) << 8) | ((0xFF & bytes[offset + 2]) << 16)
95-
| ((0xFF & bytes[offset + 3]) << 24);
94+
return (0xFF & bytes[offset++]) | ((0xFF & bytes[offset++]) << 8) | ((0xFF & bytes[offset++]) << 16)
95+
| ((0xFF & bytes[offset]) << 24);
9696
}
9797

9898
public static long toInt64(byte[] bytes, int offset) {
99-
return (0xFFL & bytes[offset]) | ((0xFFL & bytes[offset + 1]) << 8) | ((0xFFL & bytes[offset + 2]) << 16)
100-
| ((0xFFL & bytes[offset + 3]) << 24) | ((0xFFL & bytes[offset + 4]) << 32)
101-
| ((0xFFL & bytes[offset + 5]) << 40) | ((0xFFL & bytes[offset + 6]) << 48)
102-
| ((0xFFL & bytes[offset + 7]) << 56);
99+
return (0xFFL & bytes[offset++]) | ((0xFFL & bytes[offset++]) << 8) | ((0xFFL & bytes[offset++]) << 16)
100+
| ((0xFFL & bytes[offset++]) << 24) | ((0xFFL & bytes[offset++]) << 32)
101+
| ((0xFFL & bytes[offset++]) << 40) | ((0xFFL & bytes[offset++]) << 48)
102+
| ((0xFFL & bytes[offset]) << 56);
103+
}
104+
105+
public static void setInt32(byte[] bytes, int offset, int value) {
106+
bytes[offset++] = (byte) (0xFF & value);
107+
bytes[offset++] = (byte) (0xFF & (value >> 8));
108+
bytes[offset++] = (byte) (0xFF & (value >> 16));
109+
bytes[offset] = (byte) (0xFF & (value >> 24));
110+
}
111+
112+
public static void setInt64(byte[] bytes, int offset, long value) {
113+
bytes[offset++] = (byte) (0xFF & value);
114+
bytes[offset++] = (byte) (0xFF & (value >> 8));
115+
bytes[offset++] = (byte) (0xFF & (value >> 16));
116+
bytes[offset++] = (byte) (0xFF & (value >> 24));
117+
bytes[offset++] = (byte) (0xFF & (value >> 32));
118+
bytes[offset++] = (byte) (0xFF & (value >> 40));
119+
bytes[offset++] = (byte) (0xFF & (value >> 48));
120+
bytes[offset] = (byte) (0xFF & (value >> 56));
103121
}
104122

105123
/**
@@ -662,7 +680,7 @@ public static short readUnsignedInt8(ClickHouseInputStream input) throws IOExcep
662680
* end of the stream
663681
*/
664682
public static void writeUnsignedInt8(OutputStream output, int value) throws IOException {
665-
output.write((byte) (ClickHouseChecker.between(value, ClickHouseValues.TYPE_INT, 0, U_INT8_MAX) & 0xFFL));
683+
output.write((byte) (0xFF & ClickHouseChecker.between(value, ClickHouseValues.TYPE_INT, 0, U_INT8_MAX)));
666684
}
667685

668686
/**
@@ -686,7 +704,7 @@ public static short readInt16(ClickHouseInputStream input) throws IOException {
686704
* end of the stream
687705
*/
688706
public static void writeInt16(OutputStream output, short value) throws IOException {
689-
output.write(new byte[] { (byte) (0xFFL & value), (byte) (0xFFL & (value >> 8)) });
707+
output.write(new byte[] { (byte) (0xFF & value), (byte) (0xFF & (value >> 8)) });
690708
}
691709

692710
/**
@@ -749,8 +767,8 @@ public static int readInt32(ClickHouseInputStream input) throws IOException {
749767
* end of the stream
750768
*/
751769
public static void writeInt32(OutputStream output, int value) throws IOException {
752-
output.write(new byte[] { (byte) (0xFFL & value), (byte) (0xFFL & (value >> 8)), (byte) (0xFFL & (value >> 16)),
753-
(byte) (0xFFL & (value >> 24)) });
770+
output.write(new byte[] { (byte) (0xFF & value), (byte) (0xFF & (value >> 8)), (byte) (0xFF & (value >> 16)),
771+
(byte) (0xFF & (value >> 24)) });
754772
}
755773

756774
/**
@@ -799,14 +817,8 @@ public static long readInt64(ClickHouseInputStream input) throws IOException {
799817
* end of the stream
800818
*/
801819
public static void writeInt64(OutputStream output, long value) throws IOException {
802-
value = Long.reverseBytes(value);
803-
804820
byte[] bytes = new byte[8];
805-
for (int i = 7; i >= 0; i--) {
806-
bytes[i] = (byte) (value & 0xFFL);
807-
value >>= 8;
808-
}
809-
821+
setInt64(bytes, 0, value);
810822
output.write(bytes);
811823
}
812824

clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBlockChecksum.java

Lines changed: 0 additions & 66 deletions
This file was deleted.

0 commit comments

Comments
 (0)