diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index c0151718d..79cab26bd 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -42,6 +42,7 @@ jobs: - clickhouse: "21.8" protocol: grpc fail-fast: false + timeout-minutes: 45 name: Build against ClickHouse ${{ matrix.clickhouse }} (${{ matrix.protocol }}) steps: - name: Check out Git repository @@ -90,3 +91,11 @@ jobs: - name: Build run: | mvn --batch-mode --update-snapshots -Drelease -DclickhouseVersion=${{ matrix.clickhouse }} -Dprotocol=${{ matrix.protocol }} verify + - name: Upload test results + uses: actions/upload-artifact@v2 + if: failure() + with: + name: result ${{ github.job }} + path: | + **/target/failsafe-reports + **/target/surefire-reports diff --git a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/client/ClientState.java b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/client/ClientState.java index 517d3f815..e33545b0f 100644 --- a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/client/ClientState.java +++ b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/client/ClientState.java @@ -15,6 +15,7 @@ import com.clickhouse.client.ClickHouseClient; import com.clickhouse.client.ClickHouseClientBuilder; import com.clickhouse.client.ClickHouseCredentials; +import com.clickhouse.client.ClickHouseException; import com.clickhouse.client.ClickHouseCompression; import com.clickhouse.client.ClickHouseFormat; import com.clickhouse.client.ClickHouseNode; @@ -79,7 +80,7 @@ private ClickHouseClient createClient() { } @Setup(Level.Trial) - public void doSetup(ServerState serverState) throws Exception { + public void doSetup(ServerState serverState) throws ClickHouseException { server = ClickHouseNode.builder().host(serverState.getHost()).port(ClickHouseProtocol.valueOf(protocol)) .database(serverState.getDatabase()) .credentials( @@ -91,18 +92,18 @@ public void doSetup(ServerState serverState) throws Exception { "create table if not exists system.test_insert(id String, i Nullable(UInt64), s Nullable(String), t Nullable(DateTime))engine=Memory" }; for (String sql : sqls) { - try (ClickHouseResponse resp = client.connect(server).query(sql).execute().get()) { + try (ClickHouseResponse resp = client.connect(server).query(sql).executeAndWait()) { } } } @TearDown(Level.Trial) - public void doTearDown(ServerState serverState) throws Exception { + public void doTearDown(ServerState serverState) throws ClickHouseException { dispose(); - try (ClickHouseResponse resp = client.connect(server).query("truncate table system.test_insert").execute() - .get()) { + try (ClickHouseResponse resp = client.connect(server).query("truncate table system.test_insert") + .executeAndWait()) { } finally { try { diff --git a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/misc/StreamBenchmark.java b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/misc/StreamBenchmark.java new file mode 100644 index 000000000..2a4e5e856 --- /dev/null +++ b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/misc/StreamBenchmark.java @@ -0,0 +1,167 @@ +package com.clickhouse.benchmark.misc; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +import com.clickhouse.benchmark.BaseState; +import com.clickhouse.client.ClickHouseClient; +import com.clickhouse.client.ClickHouseConfig; +import com.clickhouse.client.ClickHouseInputStream; +import com.clickhouse.client.ClickHouseOutputStream; +import com.clickhouse.client.config.ClickHouseBufferingMode; +import com.clickhouse.client.config.ClickHouseClientOption; +import com.clickhouse.client.config.ClickHouseOption; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * Blocking: + * Benchmark Mode Cnt Score Error Units + * StreamBenchmark.async thrpt 20 1.574 ? 0.039 ops/s + * StreamBenchmark.jdk thrpt 20 4281.206 ? 91.983 ops/s + * StreamBenchmark.piped thrpt 20 3913.994 ? 142.566 ops/s + * StreamBenchmark.wrapped thrpt 20 3939.248 ? 54.868 ops/s + * + * Non-blocking: + * + */ +@State(Scope.Benchmark) +@Warmup(iterations = 10, timeUnit = TimeUnit.SECONDS, time = 1) +@Measurement(iterations = 10, timeUnit = TimeUnit.SECONDS, time = 1) +@Fork(value = 2) +@Threads(value = -1) +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.SECONDS) +public class StreamBenchmark { + @State(Scope.Thread) + public static class StreamState extends BaseState { + public int bufferSize; + public int samples; + + public byte[] bytes; + public ClickHouseConfig config; + + @Setup(Level.Trial) + public void setupSamples() { + bufferSize = Integer.getInteger("buffer", + (int) ClickHouseClientOption.WRITE_BUFFER_SIZE.getDefaultValue()); + samples = Integer.getInteger("samples", 500000); + + bytes = new byte[samples]; + + Map options = new HashMap<>(); + options.put(ClickHouseClientOption.ASYNC, Boolean.parseBoolean(System.getProperty("async", "true"))); + options.put(ClickHouseClientOption.REQUEST_BUFFERING, ClickHouseBufferingMode.valueOf( + System.getProperty("mode", ClickHouseClientOption.REQUEST_BUFFERING.getDefaultValue().toString()) + .toUpperCase())); + options.put(ClickHouseClientOption.WRITE_BUFFER_SIZE, bufferSize); + options.put(ClickHouseClientOption.MAX_QUEUED_BUFFERS, + Integer.getInteger("queue", (int) ClickHouseClientOption.MAX_QUEUED_BUFFERS.getDefaultValue())); + options.put(ClickHouseClientOption.COMPRESS, Boolean.parseBoolean(System.getProperty("compress", "false"))); + options.put(ClickHouseClientOption.DECOMPRESS, + Boolean.parseBoolean(System.getProperty("compress", "false"))); + options.put(ClickHouseClientOption.USE_BLOCKING_QUEUE, + Boolean.parseBoolean(System.getProperty("blocking", "true"))); + config = new ClickHouseConfig(options, null, null, null); + } + + @Setup(Level.Iteration) + public void initStream() { + new Random().nextBytes(bytes); + } + } + + @Benchmark + public void classic(StreamState state, Blackhole consumer) throws IOException { + int size = state.bufferSize; + byte[] buffer = new byte[size]; + int count = 0; + ByteArrayOutputStream out = new ByteArrayOutputStream(state.samples); + try (InputStream in = new ByteArrayInputStream(state.bytes)) { + int read = 0; + while ((read = in.read(buffer, 0, size)) > 0) { + out.write(buffer, 0, read); + count += read; + } + if (count != state.samples) { + throw new IllegalStateException(String.format("Expect %d bytes but got %d", size, count)); + } + out.flush(); + out.close(); + } + if (!Arrays.equals(state.bytes, out.toByteArray())) { + throw new IllegalStateException("Incorrect result"); + } + } + + @Benchmark + public void piped(StreamState state, Blackhole consumer) throws IOException { + int size = state.bufferSize; + long count = 0; + ByteArrayOutputStream out = new ByteArrayOutputStream(state.samples); + try (InputStream in = new ByteArrayInputStream(state.bytes)) { + if ((count = ClickHouseInputStream.pipe(in, out, size)) != state.samples) { + throw new IllegalStateException(String.format("Expect %d bytes but got %d", size, count)); + } + out.flush(); + out.close(); + } + if (!Arrays.equals(state.bytes, out.toByteArray())) { + throw new IllegalStateException("Incorrect result"); + } + } + + @Benchmark + public void wrapped(StreamState state, Blackhole consumer) throws IOException { + int size = state.bufferSize; + long count = 0; + ByteArrayOutputStream bao = new ByteArrayOutputStream(state.samples); + try (ClickHouseInputStream in = ClickHouseInputStream.of(new ByteArrayInputStream(state.bytes), size); + ClickHouseOutputStream out = ClickHouseOutputStream.of(bao, size)) { + if ((count = in.pipe(out)) != state.samples) { + throw new IllegalStateException(String.format("Expect %d bytes but got %d", size, count)); + } + out.flush(); + } + if (!Arrays.equals(state.bytes, bao.toByteArray())) { + throw new IllegalStateException("Incorrect result"); + } + } + + @Benchmark + public void async(StreamState state, Blackhole consumer) throws IOException { + int size = state.bufferSize; + long count = 0; + ByteArrayOutputStream bao = new ByteArrayOutputStream(state.samples); + try (ClickHouseInputStream in = ClickHouseInputStream.of(new ByteArrayInputStream(state.bytes), size); + ClickHouseOutputStream out = ClickHouseClient.getAsyncRequestOutputStream(state.config, bao, null)) { + if ((count = in.pipe(out)) != state.samples) { + throw new IllegalStateException(String.format("Expect %d bytes but got %d", size, count)); + } + out.flush(); + } + if (!Arrays.equals(state.bytes, bao.toByteArray())) { + throw new IllegalStateException("Incorrect result"); + } + } +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseByteBuffer.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseByteBuffer.java index dab766d5f..8def416b0 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseByteBuffer.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseByteBuffer.java @@ -108,92 +108,359 @@ protected ClickHouseByteBuffer(byte[] bytes, int offset, int length) { this.length = length; } - // public byte[] toInt8Array() - /** - * Converts 4 bytes to Int32. Same as {@code getInt32(0)}. + * Converts 4 bytes(little-endian) to Int32. Same as {@code getInteger(0)}. * * @return Int32 value + * @deprecated will be removed in v0.3.3, please use {@link #asInteger()} + * instead */ + @Deprecated public int asInt32() { - return getInt32(0); + return getInteger(0); } + /** + * Converts 4 bytes(little-endian) to UInt32. Same as + * {@code getUnsignedInteger(0)}. + * + * @return UInt32 value + * @deprecated will be removed in v0.3.3, please use + * {@link #asUnsignedInteger()} instead + */ + @Deprecated public long asUnsignedInt32() { - return getUnsignedInt32(0); + return getUnsignedInteger(0); } /** - * Converts 4 bytes starting from {@code position() + offset} to Int32. + * Converts 4 bytes(little-endian) starting from {@code position() + offset} to + * Int32. * * @param offset zero-based relative offset, 1 means the second byte starting * from {@link #position()} * @return Int32 value + * @deprecated will be removed in v0.3.3, please use {@link #getInteger(int)} + * instead */ + @Deprecated public int getInt32(int offset) { - offset += position; - return (0xFF & array[offset++]) | ((0xFF & array[offset++]) << 8) | ((0xFF & array[offset++]) << 16) - | ((0xFF & array[offset]) << 24); + return getInteger(offset); } + /** + * Converts 4 bytes(little-endian) starting from {@code position() + offset} to + * UInt32. + * + * @param offset zero-based relative offset, 1 means the second byte starting + * from {@link #position()} + * @return UInt32 value + * @deprecated will be removed in v0.3.3, please use + * {@link #getUnsignedInteger(int)} instead + */ + @Deprecated public long getUnsignedInt32(int offset) { - return 0xFFFFFFFFL & getInt32(offset); + return getUnsignedInteger(offset); } /** - * Converts 8 bytes to Int64. Same as {@code getInt64(0)}. + * Converts 8 bytes(little-endian) to Int64. Same as {@code getLong(0)}. * * @return Int64 value + * @deprecated will be removed in v0.3.3, please use {@link #asLong()} instead */ + @Deprecated public long asInt64() { - return getInt64(0); + return asLong(); } + /** + * Converts 8 bytes(little-endian) to unsigned Int64. Same as + * {@code getBigInteger(0, 8, true)}. + * + * @return unsigned Int64 value + * @deprecated will be removed in v0.3.3, please use {@link #asUnsignedLong()} + * instead + */ + @Deprecated public BigInteger asUnsignedInt64() { return getBigInteger(0, 8, true); } /** - * Converts 8 bytes starting from {@code position() + offset} to Int64. + * Converts 8 bytes(little-endian) starting from {@code position() + offset} to + * Int64. * * @param offset zero-based relative offset, 1 means the second byte starting * from {@link #position()} * @return Int64 value + * @deprecated will be removed in v0.3.3, please use {@link #getLong(int)} + * instead */ + @Deprecated public long getInt64(int offset) { - offset += position; - return (0xFFL & array[offset++]) | ((0xFFL & array[offset++]) << 8) | ((0xFFL & array[offset++]) << 16) - | ((0xFFL & array[offset++]) << 24) | ((0xFFL & array[offset++]) << 32) - | ((0xFFL & array[offset++]) << 40) | ((0xFFL & array[offset++]) << 48) - | ((0xFFL & array[offset]) << 56); + return getLong(offset); } + /** + * Converts all bytes(little-endian) to signed big integer. Same as + * {@code getBigInteger(0, length(), false)}. + * + * @return non-null signed big integer + */ public BigInteger asBigInteger() { return getBigInteger(0, length, false); } + public double asDouble() { + return getDouble(0); + } + + public double[] asDoubleArray() { + int step = 8; + int len = length / step; + double[] values = new double[len]; + for (int i = 0, offset = 0; i < len; i++, offset += step) { + values[i] = getDouble(offset); + } + return values; + } + + public float asFloat() { + return getFloat(0); + } + + public float[] asFloatArray() { + int step = 4; + int len = length / step; + float[] values = new float[len]; + for (int i = 0, offset = 0; i < len; i++, offset += step) { + values[i] = getFloat(offset); + } + return values; + } + + /** + * Converts all bytes(little-endian) to unsigned big integer. Same as + * {@code getBigInteger(0, length(), true)}. + * + * @return non-null unsigned big integer + */ public BigInteger asUnsignedBigInteger() { return getBigInteger(0, length, true); } + /** + * Converts {@code byteLength} bytes(little-endian) starting from + * {@code position() + offset} to big integer. + * + * @param offset zero-based relative offset, 1 means the second byte + * starting from {@link #position()} + * @param byteLength bytes to convert + * @param unsigned true if it's unsigned big integer; false otherwise + * @return non-null big integer + */ public BigInteger getBigInteger(int offset, int byteLength, boolean unsigned) { - reverse(offset, byteLength); - // return new BigInteger(1, array, offset + position, byteLength); - // just for supporting JDK 8 :< - byte[] bytes = array; - if (offset != 0 || array.length != byteLength) { - bytes = new byte[byteLength]; - System.arraycopy(array, offset + position, bytes, 0, byteLength); + byte[] bytes = new byte[byteLength]; + int startIndex = offset + position; + for (int i = startIndex + byteLength - 1, j = 0; i >= startIndex; i--, j++) { + bytes[j] = array[i]; } return unsigned ? new BigInteger(1, bytes) : new BigInteger(bytes); } + /** + * Converts 8 bytes(little-endian) starting from {@code position() + offset} to + * double. + * + * @param offset zero-based relative offset, 1 means the second byte + * starting from {@link #position()} + * @return double + */ + public double getDouble(int offset) { + return Double.longBitsToDouble(getLong(offset)); + } + + /** + * Converts 4 bytes(little-endian) starting from {@code position() + offset} to + * float. + * + * @param offset zero-based relative offset, 1 means the second byte + * starting from {@link #position()} + * @return float + */ + public float getFloat(int offset) { + return Float.intBitsToFloat(getInteger(offset)); + } + + /** + * Converts 4 bytes(little-endian) starting from {@code position() + offset} to + * signed integer. + * + * @param offset zero-based relative offset, 1 means the second byte + * starting from {@link #position()} + * @return signed integer + */ + public int getInteger(int offset) { + int i = offset + position; + return (0xFF & array[i]) | ((0xFF & array[i + 1]) << 8) | ((0xFF & array[i + 2]) << 16) + | ((0xFF & array[i + 3]) << 24); + } + + /** + * Converts 8 bytes(little-endian) starting from {@code position() + offset} to + * signed long. + * + * @param offset zero-based relative offset, 1 means the second byte + * starting from {@link #position()} + * @return signed long + */ + public long getLong(int offset) { + int i = offset + position; + return (0xFFL & array[i]) | ((0xFFL & array[i + 1]) << 8) | ((0xFFL & array[i + 2]) << 16) + | ((0xFFL & array[i + 3]) << 24) | ((0xFFL & array[i + 4]) << 32) + | ((0xFFL & array[i + 5]) << 40) | ((0xFFL & array[i + 6]) << 48) + | ((0xFFL & array[i + 7]) << 56); + } + + /** + * Converts 2 bytes(little-endian) starting from {@code position() + offset} to + * signed short. + * + * @param offset zero-based relative offset, 1 means the second byte + * starting from {@link #position()} + * @return signed short + */ + public short getShort(int offset) { + int i = offset + position; + return (short) ((0xFF & array[i]) | (array[i + 1] << 8)); + } + + /** + * Converts 4 bytes(little-endian) starting from {@code position() + offset} to + * unsigned integer. + * + * @param offset zero-based relative offset, 1 means the second byte + * starting from {@link #position()} + * @return unsigned integer + */ + public long getUnsignedInteger(int offset) { + return 0xFFFFFFFFL & getInteger(offset); + } + + /** + * Converts 8 bytes(little-endian) starting from {@code position() + offset} to + * unsigned long. + * + * @param offset zero-based relative offset, 1 means the second byte + * starting from {@link #position()} + * @return non-null unsigned long + */ + public BigInteger getUnsignedLong(int offset) { + return getBigInteger(0, 8, true); + } + + /** + * Converts 2 bytes(little-endian) starting from {@code position() + offset} to + * unsigned short. + * + * @param offset zero-based relative offset, 1 means the second byte + * starting from {@link #position()} + * @return unsigned short + */ + public int getUnsignedShort(int offset) { + return 0xFFFF & getShort(offset); + } + + public BigInteger[] asBigIntegerArray(int byteLength, boolean unsigned) { + int len = length / byteLength; + BigInteger[] values = new BigInteger[len]; + for (int i = 0, offset = 0; i < len; i++, offset += byteLength) { + values[i] = getBigInteger(offset, byteLength, unsigned); + } + return values; + } + + public int asInteger() { + return getInteger(0); + } + + public long asLong() { + return getLong(0); + } + + public short asShort() { + return getShort(0); + } + + public int[] asIntegerArray() { + int step = 4; + int len = length / step; + int[] values = new int[len]; + for (int i = 0, offset = 0; i < len; i++, offset += step) { + values[i] = getInteger(offset); + } + return values; + } + + public long[] asLongArray() { + int step = 8; + int len = length / step; + long[] values = new long[len]; + for (int i = 0, offset = 0; i < len; i++, offset += step) { + values[i] = getLong(offset); + } + return values; + } + + public short[] asShortArray() { + int step = 2; + int len = length / step; + short[] values = new short[len]; + for (int i = 0, offset = 0; i < len; i++, offset += step) { + values[i] = getShort(offset); + } + return values; + } + + public long asUnsignedInteger() { + return getUnsignedInteger(0); + } + + public BigInteger asUnsignedLong() { + return getUnsignedLong(0); + } + + public int asUnsignedShort() { + return getUnsignedShort(0); + } + + public long[] asUnsignedIntegerArray() { + int step = 4; + int len = length / step; + long[] values = new long[len]; + for (int i = 0, offset = 0; i < len; i++, offset += step) { + values[i] = getUnsignedInteger(offset); + } + return values; + } + + public int[] asUnsignedShortArray() { + int step = 2; + int len = length / step; + int[] values = new int[len]; + for (int i = 0, offset = 0; i < len; i++, offset += step) { + values[i] = getUnsignedShort(offset); + } + return values; + } + public UUID asUuid() { return getUuid(0); } public UUID getUuid(int offset) { - return new UUID(getInt64(offset), getInt64(offset + 8)); + return new UUID(getLong(offset), getLong(offset + 8)); } /** @@ -213,6 +480,9 @@ public String asAsciiString() { * @return non-null string */ public String asString(Charset charset) { + if (length < 1) { + return ""; + } return new String(array, position, length, charset != null ? charset : StandardCharsets.UTF_8); } @@ -243,6 +513,17 @@ public ClickHouseByteBuffer compact() { return this; } + /** + * Checks whether the buffer is compact or not. A buffer is compact when + * {@code position} points to zero and {@code length} equals to + * {@code array().length}. + * + * @return true if the buffer is compact; false otherwise + */ + public boolean isCompact() { + return position == 0 && array.length == length; + } + /** * Checks whether the buffer is empty or not. Please pay attention that this * will return {@code true} when {@link #length()} is zero, even @@ -323,9 +604,8 @@ public ClickHouseByteBuffer update(byte[] bytes) { if (bytes == null || bytes.length == 0) { reset(); } else { - array = bytes; position = 0; - length = bytes.length; + length = (array = bytes).length; } return this; @@ -346,7 +626,9 @@ public ClickHouseByteBuffer update(byte[] bytes, int offset, int length) { validate(bytes, offset, length); } - this.array = bytes; + if (bytes != this.array) { + this.array = bytes; + } this.position = offset; this.length = length; return this; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClient.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClient.java index faf188d99..f88ed7ee0 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClient.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClient.java @@ -14,14 +14,16 @@ import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.Function; + +import com.clickhouse.client.config.ClickHouseBufferingMode; import com.clickhouse.client.config.ClickHouseClientOption; import com.clickhouse.client.config.ClickHouseDefaults; import com.clickhouse.client.config.ClickHouseOption; -import com.clickhouse.client.data.ClickHousePipedStream; /** * A unified interface defines Java client for ClickHouse. A client can only @@ -78,13 +80,62 @@ static ClickHouseOutputStream getRequestOutputStream(ClickHouseConfig config, Ou ClickHouseCompression.NONE, postCloseAction); } - int bufferSize = config.getWriteBufferSize(); - ClickHouseCompression compression = ClickHouseCompression.NONE; - if (config.isDecompressClientRequet()) { - bufferSize = (int) config.getOption(ClickHouseClientOption.MAX_COMPRESS_BLOCK_SIZE); - compression = config.getDecompressAlgorithmForClientRequest(); + return ClickHouseOutputStream.of(output, config.getWriteBufferSize(), config.getRequestCompressAlgorithm(), + postCloseAction); + } + + /** + * Gets piped output stream for writing data into request asynchronously. When + * {@code config} is null or {@code config.isAsync()} is false, this method is + * same as + * {@link #getRequestOutputStream(ClickHouseConfig, OutputStream, Runnable)}. + * + * @param config optional configuration + * @param output non-null output stream + * @param postCloseAction custom action will be performed right after closing + * the output stream + * @return wrapped output, or the same output if it's instance of + * {@link ClickHouseOutputStream} + */ + static ClickHouseOutputStream getAsyncRequestOutputStream(ClickHouseConfig config, OutputStream output, + Runnable postCloseAction) { + if (config == null || !config.isAsync() + || config.getRequestBuffering() == ClickHouseBufferingMode.RESOURCE_EFFICIENT) { + return getRequestOutputStream(config, output, postCloseAction); } - return ClickHouseOutputStream.of(output, bufferSize, compression, postCloseAction); + + final CountDownLatch latch = new CountDownLatch(1); + final ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance() + .createPipedOutputStream(config, () -> { + long timeout = config.getSocketTimeout(); + try { + if (timeout > 0L) { + if (!latch.await(timeout, TimeUnit.MILLISECONDS)) { + throw new IllegalStateException( + ClickHouseUtils.format("Async write timed out after %d ms", timeout)); + } + } else { + latch.await(); + } + + if (postCloseAction != null) { + postCloseAction.run(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException("Stopped waiting for writes", e); + } + }); + submit(() -> { + try (ClickHouseInputStream in = stream.getInputStream(); + ClickHouseOutputStream out = getRequestOutputStream(config, output, postCloseAction)) { + in.pipe(out); + } finally { + latch.countDown(); + } + return null; + }); + return stream; } /** @@ -104,13 +155,57 @@ static ClickHouseInputStream getResponseInputStream(ClickHouseConfig config, Inp ClickHouseCompression.NONE, postCloseAction); } - int bufferSize = config.getReadBufferSize(); - ClickHouseCompression compression = ClickHouseCompression.NONE; - if (config.isCompressServerResponse()) { - bufferSize = (int) config.getOption(ClickHouseClientOption.MAX_COMPRESS_BLOCK_SIZE); - compression = config.getCompressAlgorithmForServerResponse(); + return ClickHouseInputStream.of(input, config.getReadBufferSize(), config.getResponseCompressAlgorithm(), + postCloseAction); + } + + /** + * Gets piped input stream for reading data from response asynchronously. When + * {@code config} is null or {@code config.isAsync()} is faluse, this method is + * same as + * {@link #getResponseInputStream(ClickHouseConfig, InputStream, Runnable)}. + * + * @param config optional configuration + * @param input non-null input stream + * @param postCloseAction custom action will be performed right after closing + * the input stream + * @return wrapped input, or the same input if it's instance of + * {@link ClickHouseInputStream} + */ + @SuppressWarnings("squid:S2095") + static ClickHouseInputStream getAsyncResponseInputStream(ClickHouseConfig config, InputStream input, + Runnable postCloseAction) { + if (config == null || !config.isAsync() + || config.getResponseBuffering() == ClickHouseBufferingMode.RESOURCE_EFFICIENT) { + return getResponseInputStream(config, input, postCloseAction); + } + + // raw response -> input + final ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance() + .createPipedOutputStream(config, null); + final ClickHouseInputStream wrappedInput; + // raw response -> decompressed response -> input + if (config.isResponseCompressed()) { // one more thread for decompression? + final ClickHousePipedOutputStream decompressedStream = ClickHouseDataStreamFactory.getInstance() + .createPipedOutputStream(config, null); + wrappedInput = getResponseInputStream(config, decompressedStream.getInputStream(), postCloseAction); + submit(() -> { + try (ClickHouseInputStream in = ClickHouseInputStream.of(input, config.getReadBufferSize()); + ClickHouseOutputStream out = decompressedStream) { + in.pipe(out); + } + return null; + }); + } else { + wrappedInput = getResponseInputStream(config, input, postCloseAction); } - return ClickHouseInputStream.of(input, bufferSize, compression, postCloseAction); + submit(() -> { + try (ClickHouseInputStream in = wrappedInput; ClickHouseOutputStream out = stream) { + in.pipe(out); + } + return null; + }); + return stream.getInputStream(); } /** @@ -216,17 +311,10 @@ static CompletableFuture dump(ClickHouseNode server, request.query(theQuery); } - try (ClickHouseResponse response = request.execute().get()) { + try (ClickHouseResponse response = request.executeAndWait()) { response.pipe(output, request.getConfig().getWriteBufferSize()); return response.getSummary(); } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw ClickHouseException.forCancellation(e, theServer); - } catch (CancellationException e) { - throw ClickHouseException.forCancellation(e, theServer); - } catch (ExecutionException e) { - throw ClickHouseException.of(e, theServer); } finally { try { output.close(); @@ -284,13 +372,13 @@ static CompletableFuture load(ClickHouseNode server, try (ClickHouseClient client = ClickHouseClient.builder() .nodeSelector(ClickHouseNodeSelector.of(theServer.getProtocol())) .option(ClickHouseClientOption.ASYNC, true).build()) { - ClickHousePipedStream stream = ClickHouseDataStreamFactory.getInstance() - .createPipedStream(client.getConfig()); + ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance() + .createPipedOutputStream(client.getConfig(), null); // execute query in a separate thread(because async is explicitly set to true) CompletableFuture future = client.connect(theServer).write().table(table) .decompressClientRequest(compression != null && compression != ClickHouseCompression.NONE, compression) - .format(format).data(input = stream.getInput()).execute(); + .format(format).data(input = stream.getInputStream()).execute(); try { // write data into stream in current thread writer.write(stream); @@ -348,15 +436,8 @@ static CompletableFuture load(ClickHouseNode server, ClickHouseResponse response = client.connect(theServer).write().table(table) .decompressClientRequest(compression != null && compression != ClickHouseCompression.NONE, compression) - .format(format).data(input).execute().get()) { + .format(format).data(input).executeAndWait()) { return response.getSummary(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw ClickHouseException.forCancellation(e, theServer); - } catch (CancellationException e) { - throw ClickHouseException.forCancellation(e, theServer); - } catch (ExecutionException e) { - throw ClickHouseException.of(e, theServer); } finally { try { input.close(); @@ -418,17 +499,10 @@ static CompletableFuture> send(ClickHouseNode se request.session(UUID.randomUUID().toString(), false); } for (String query : queries) { - try (ClickHouseResponse resp = request.query(query).execute().get()) { + try (ClickHouseResponse resp = request.query(query).executeAndWait()) { list.add(resp.getSummary()); } } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw ClickHouseException.forCancellation(e, theServer); - } catch (CancellationException e) { - throw ClickHouseException.forCancellation(e, theServer); - } catch (ExecutionException e) { - throw ClickHouseException.of(e, theServer); } return list; @@ -460,15 +534,8 @@ static CompletableFuture send(ClickHouseNode server, .nodeSelector(ClickHouseNodeSelector.of(theServer.getProtocol())) .option(ClickHouseClientOption.ASYNC, false).build(); ClickHouseResponse resp = client.connect(theServer).format(ClickHouseFormat.RowBinary).query(sql) - .params(params).execute().get()) { + .params(params).executeAndWait()) { return resp.getSummary(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw ClickHouseException.forCancellation(e, theServer); - } catch (CancellationException e) { - throw ClickHouseException.forCancellation(e, theServer); - } catch (ExecutionException e) { - throw ClickHouseException.of(e, theServer); } }); } @@ -549,17 +616,10 @@ static CompletableFuture> send(ClickHouseNode se arr[j] = v != null ? v.resetToNullOrEmpty().toSqlExpression() : ClickHouseValues.NULL_EXPR; } } - try (ClickHouseResponse resp = request.params(arr).execute().get()) { + try (ClickHouseResponse resp = request.params(arr).executeAndWait()) { list.add(resp.getSummary()); } } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw ClickHouseException.forCancellation(e, theServer); - } catch (CancellationException e) { - throw ClickHouseException.forCancellation(e, theServer); - } catch (ExecutionException e) { - throw ClickHouseException.of(e, theServer); } return list; @@ -602,17 +662,10 @@ static CompletableFuture> send(ClickHouseNode se for (String[] p : params) { builder.setLength(0); query.apply(builder, p); - try (ClickHouseResponse resp = request.query(builder.toString()).execute().get()) { + try (ClickHouseResponse resp = request.query(builder.toString()).executeAndWait()) { list.add(resp.getSummary()); } } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw ClickHouseException.forCancellation(e, theServer); - } catch (CancellationException e) { - throw ClickHouseException.forCancellation(e, theServer); - } catch (ExecutionException e) { - throw ClickHouseException.of(e, theServer); } return list; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseConfig.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseConfig.java index c69fd5502..1a3b1eb67 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseConfig.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseConfig.java @@ -11,6 +11,7 @@ import java.util.Set; import java.util.TimeZone; +import com.clickhouse.client.config.ClickHouseBufferingMode; import com.clickhouse.client.config.ClickHouseClientOption; import com.clickhouse.client.config.ClickHouseOption; import com.clickhouse.client.config.ClickHouseDefaults; @@ -102,10 +103,10 @@ protected static final Object mergeMetricRegistry(List list) { // common options optimized for read private final boolean async; private final String clientName; - private final boolean compressServerResponse; + private final boolean compressRequest; private final ClickHouseCompression compressAlgorithm; private final int compressLevel; - private final boolean decompressClientRequest; + private final boolean decompressResponse; private final ClickHouseCompression decompressAlgorithm; private final int decompressLevel; private final int connectionTimeout; @@ -115,6 +116,8 @@ protected static final Object mergeMetricRegistry(List list) { private final int bufferSize; private final int readBufferSize; private final int writeBufferSize; + private final ClickHouseBufferingMode requestBuffering; + private final ClickHouseBufferingMode responseBuffering; private final int maxExecutionTime; private final int maxQueuedBuffers; private final int maxQueuedRequests; @@ -133,6 +136,7 @@ protected static final Object mergeMetricRegistry(List list) { private final String sslRootCert; private final String sslCert; private final String sslKey; + private final boolean useBlockingQueue; private final boolean useObjectsInArray; private final boolean useServerTimeZone; private final boolean useServerTimeZoneForDates; @@ -183,12 +187,12 @@ public ClickHouseConfig(Map options, ClickHouseC this.async = (boolean) getOption(ClickHouseClientOption.ASYNC, ClickHouseDefaults.ASYNC); this.clientName = (String) getOption(ClickHouseClientOption.CLIENT_NAME); - this.compressServerResponse = (boolean) getOption(ClickHouseClientOption.COMPRESS); - this.compressAlgorithm = (ClickHouseCompression) getOption(ClickHouseClientOption.COMPRESS_ALGORITHM); - this.compressLevel = (int) getOption(ClickHouseClientOption.COMPRESS_LEVEL); - this.decompressClientRequest = (boolean) getOption(ClickHouseClientOption.DECOMPRESS); - this.decompressAlgorithm = (ClickHouseCompression) getOption(ClickHouseClientOption.DECOMPRESS_ALGORITHM); - this.decompressLevel = (int) getOption(ClickHouseClientOption.DECOMPRESS_LEVEL); + this.compressRequest = (boolean) getOption(ClickHouseClientOption.DECOMPRESS); + this.compressAlgorithm = (ClickHouseCompression) getOption(ClickHouseClientOption.DECOMPRESS_ALGORITHM); + this.compressLevel = (int) getOption(ClickHouseClientOption.DECOMPRESS_LEVEL); + this.decompressResponse = (boolean) getOption(ClickHouseClientOption.COMPRESS); + this.decompressAlgorithm = (ClickHouseCompression) getOption(ClickHouseClientOption.COMPRESS_ALGORITHM); + this.decompressLevel = (int) getOption(ClickHouseClientOption.COMPRESS_LEVEL); this.connectionTimeout = (int) getOption(ClickHouseClientOption.CONNECTION_TIMEOUT); this.database = (String) getOption(ClickHouseClientOption.DATABASE, ClickHouseDefaults.DATABASE); this.format = (ClickHouseFormat) getOption(ClickHouseClientOption.FORMAT, ClickHouseDefaults.FORMAT); @@ -196,10 +200,14 @@ public ClickHouseConfig(Map options, ClickHouseC -1); this.bufferSize = ClickHouseUtils.getBufferSize((int) getOption(ClickHouseClientOption.BUFFER_SIZE), -1, this.maxBufferSize); - this.readBufferSize = ClickHouseUtils.getBufferSize((int) getOption(ClickHouseClientOption.READ_BUFFER_SIZE), - this.bufferSize, this.maxBufferSize); - this.writeBufferSize = ClickHouseUtils.getBufferSize((int) getOption(ClickHouseClientOption.WRITE_BUFFER_SIZE), - this.bufferSize, this.maxBufferSize); + this.readBufferSize = ClickHouseUtils.getBufferSize( + (int) getOption(ClickHouseClientOption.READ_BUFFER_SIZE), this.bufferSize, this.maxBufferSize); + this.writeBufferSize = ClickHouseUtils.getBufferSize( + (int) getOption(ClickHouseClientOption.WRITE_BUFFER_SIZE), this.bufferSize, this.maxBufferSize); + this.requestBuffering = (ClickHouseBufferingMode) getOption(ClickHouseClientOption.REQUEST_BUFFERING, + ClickHouseDefaults.BUFFERING); + this.responseBuffering = (ClickHouseBufferingMode) getOption(ClickHouseClientOption.RESPONSE_BUFFERING, + ClickHouseDefaults.BUFFERING); this.maxExecutionTime = (int) getOption(ClickHouseClientOption.MAX_EXECUTION_TIME); this.maxQueuedBuffers = (int) getOption(ClickHouseClientOption.MAX_QUEUED_BUFFERS); this.maxQueuedRequests = (int) getOption(ClickHouseClientOption.MAX_QUEUED_REQUESTS); @@ -221,6 +229,7 @@ public ClickHouseConfig(Map options, ClickHouseC this.sslRootCert = (String) getOption(ClickHouseClientOption.SSL_ROOT_CERTIFICATE); this.sslCert = (String) getOption(ClickHouseClientOption.SSL_CERTIFICATE); this.sslKey = (String) getOption(ClickHouseClientOption.SSL_KEY); + this.useBlockingQueue = (boolean) getOption(ClickHouseClientOption.USE_BLOCKING_QUEUE); this.useObjectsInArray = (boolean) getOption(ClickHouseClientOption.USE_OBJECTS_IN_ARRAYS); this.useServerTimeZone = (boolean) getOption(ClickHouseClientOption.USE_SERVER_TIME_ZONE); this.useServerTimeZoneForDates = (boolean) getOption(ClickHouseClientOption.USE_SERVER_TIME_ZONE_FOR_DATES); @@ -249,26 +258,132 @@ public String getClientName() { return clientName; } + /** + * Checks if server response is compressed or not. + * + * @return true if server response is compressed; false otherwise + */ + public boolean isResponseCompressed() { + return decompressResponse; + } + + /** + * Gets server response compress algorithm. When {@link #isResponseCompressed()} + * is {@code false}, this will return {@link ClickHouseCompression#NONE}. + * + * @return non-null compress algorithm + */ + public ClickHouseCompression getResponseCompressAlgorithm() { + return decompressResponse ? decompressAlgorithm : ClickHouseCompression.NONE; + } + + /** + * Gets input compress level. When {@link #isResponseCompressed()} is + * {@code false}, this will return {@code 0}. + * + * @return compress level + */ + public int getResponseCompressLevel() { + return decompressResponse ? decompressLevel : 0; + } + + /** + * Checks if server response should be compressed or not. + * + * @return + * @deprecated will be removed in v0.3.3, please use + * {@link #isResponseCompressed()} instead + */ + @Deprecated public boolean isCompressServerResponse() { - return compressServerResponse; + return decompressResponse; } + /** + * Gets compress algorithm for server response. + * + * @return compress algorithm for server response + * @deprecated will be removed in v0.3.3, please use + * {@link #getResponseCompressAlgorithm()} instead + */ + @Deprecated public ClickHouseCompression getCompressAlgorithmForServerResponse() { - return compressAlgorithm; + return decompressAlgorithm; } + /** + * Gets compress level for server response. + * + * @return compress level + * @deprecated will be removed in v0.3.3, please use + * {@link #getResponseCompressLevel()} instead + */ + @Deprecated public int getCompressLevelForServerResponse() { - return compressLevel; + return decompressLevel; + } + + /** + * Checks if client's output, aka. client request, should be compressed or not. + * + * @return true if client request should be compressed; false otherwise + */ + public boolean isRequestCompressed() { + return compressRequest; } + /** + * Gets input compress algorithm. When {@link #isRequestCompressed()} is + * {@code false}, this will return {@link ClickHouseCompression#NONE}. + * + * @return non-null compress algorithm + */ + public ClickHouseCompression getRequestCompressAlgorithm() { + return compressRequest ? compressAlgorithm : ClickHouseCompression.NONE; + } + + /** + * Gets input compress level. When {@link #isRequestCompressed()} is + * {@code false}, this will return {@code 0}. + * + * @return compress level + */ + public int getRequestCompressLevel() { + return compressRequest ? compressLevel : 0; + } + + /** + * Checks if client request should be compressed or not. + * + * @return + * @deprecated will be removed in v0.3.3, please use + * {@link #isRequestCompressed()} instead + */ + @Deprecated public boolean isDecompressClientRequet() { - return decompressClientRequest; + return compressRequest; } + /** + * Gets compress algorithm for client request. + * + * @return compress algorithm for client request + * @deprecated will be removed in v0.3.3, please use + * {@link #getRequestCompressAlgorithm()} instead + */ + @Deprecated public ClickHouseCompression getDecompressAlgorithmForClientRequest() { return decompressAlgorithm; } + /** + * Gets compress level for client request. + * + * @return compress level + * @deprecated will be removed in v0.3.3, please use + * {@link #getRequestCompressLevel()} instead + */ + @Deprecated public int getDecompressLevelForClientRequest() { return decompressLevel; } @@ -321,6 +436,24 @@ public int getWriteBufferSize() { return writeBufferSize; } + /** + * Gets request buffering mode. + * + * @return request buffering mode + */ + public ClickHouseBufferingMode getRequestBuffering() { + return requestBuffering; + } + + /** + * Gets response buffering mode. + * + * @return response buffering mode + */ + public ClickHouseBufferingMode getResponseBuffering() { + return responseBuffering; + } + public int getMaxExecutionTime() { return maxExecutionTime; } @@ -398,6 +531,10 @@ public String getSslKey() { return sslKey; } + public boolean isUseBlockingQueue() { + return useBlockingQueue; + } + public boolean isUseObjectsInArray() { return useObjectsInArray; } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java index 35459b175..6ddc78806 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java @@ -3,7 +3,6 @@ import java.io.EOFException; import java.io.IOException; import java.io.UncheckedIOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -11,9 +10,6 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; -import java.util.function.Supplier; - -import com.clickhouse.client.data.ClickHouseSimpleRecord; /** * This defines a data processor for dealing with serialization and @@ -23,6 +19,42 @@ * can handle more scenarios like separator between columns and rows. */ public abstract class ClickHouseDataProcessor { + static final class RecordsIterator implements Iterator { + private final ClickHouseDataProcessor processor; + + RecordsIterator(ClickHouseDataProcessor processor) { + this.processor = processor; + } + + @Override + public boolean hasNext() { + return processor.hasNext(); + } + + @Override + public ClickHouseRecord next() { + return processor.nextRecord(); + } + } + + static final class ValuesIterator implements Iterator { + private final ClickHouseDataProcessor processor; + + ValuesIterator(ClickHouseDataProcessor processor) { + this.processor = processor; + } + + @Override + public boolean hasNext() { + return processor.hasNext(); + } + + @Override + public ClickHouseValue next() { + return processor.nextValue(); + } + } + public static final List DEFAULT_COLUMNS = Collections .singletonList(ClickHouseColumn.of("results", "Nullable(String)")); @@ -65,6 +97,7 @@ protected static , T extends ClickHouseValue> void buildMappin protected final ClickHouseInputStream input; protected final ClickHouseOutputStream output; protected final ClickHouseColumn[] columns; + protected final ClickHouseRecord currentRecord; protected final ClickHouseValue[] templates; protected final Map settings; @@ -82,14 +115,89 @@ protected static , T extends ClickHouseValue> void buildMappin */ // protected int writePosition; + /** + * Checks whether there's more to read from input stream. + * + * @return true if there's more; false otherwise + * @throws UncheckedIOException when failed to read columns from input stream + */ + private boolean hasNext() throws UncheckedIOException { + try { + return input.available() > 0; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + /** + * This method calls {@link #readAndFill(ClickHouseRecord)} and resets + * {@code readPosition} to zero(first column). + * + * @return non-null record + * @throws NoSuchElementException when no more record to read + * @throws UncheckedIOException when failed to read columns from input stream + */ + private ClickHouseRecord nextRecord() throws NoSuchElementException, UncheckedIOException { + final ClickHouseRecord r = config.isReuseValueWrapper() ? currentRecord : currentRecord.copy(); + try { + readAndFill(r); + readPosition = 0; + } catch (EOFException e) { + if (readPosition == 0) { // end of the stream, which is fine + throw new NoSuchElementException("No more record"); + } else { + throw new UncheckedIOException(ClickHouseUtils.format(ERROR_REACHED_END_OF_STREAM, + readPosition + 1, columns.length, columns[readPosition]), e); + } + } catch (IOException e) { + throw new UncheckedIOException( + ClickHouseUtils.format(ERROR_FAILED_TO_READ, readPosition + 1, columns.length, + columns[readPosition]), + e); + } + return r; + } + + /** + * This method calls {@link #readAndFill(ClickHouseValue, ClickHouseColumn)} and + * updates {@code readPosition} to point to next column. + * + * @return non-null value + * @throws NoSuchElementException when no more value to read + * @throws UncheckedIOException when failed to read columns from input stream + */ + private ClickHouseValue nextValue() throws NoSuchElementException, UncheckedIOException { + ClickHouseColumn column = columns[readPosition]; + ClickHouseValue value = templates[readPosition]; + if (value == null || config.isReuseValueWrapper()) { + value = ClickHouseValues.newValue(config, column); + } + try { + readAndFill(value, column); + if (++readPosition >= columns.length) { + readPosition = 0; + } + } catch (EOFException e) { + if (readPosition == 0) { // end of the stream, which is fine + throw new NoSuchElementException("No more value"); + } else { + throw new UncheckedIOException(ClickHouseUtils.format(ERROR_REACHED_END_OF_STREAM, + readPosition + 1, columns.length, column), e); + } + } catch (IOException e) { + throw new UncheckedIOException( + ClickHouseUtils.format(ERROR_FAILED_TO_READ, readPosition + 1, columns.length, column), e); + } + + return value; + } + /** * Factory method to create a record. * * @return new record */ - protected ClickHouseRecord createRecord() { - return ClickHouseSimpleRecord.of(getColumns(), templates); - } + protected abstract ClickHouseRecord createRecord(); /** * Initializes iterator of {@link ClickHouseRecord} for reading values record by @@ -98,46 +206,10 @@ protected ClickHouseRecord createRecord() { * @return non-null iterator of {@link ClickHouseRecord} */ protected Iterator initRecords() { - final ClickHouseRecord reusableRecord = createRecord(); - final Supplier factory; - if (config.isReuseValueWrapper()) { - factory = () -> reusableRecord; - } else { - factory = reusableRecord::copy; + if (readPosition != 0) { + throw new IllegalStateException("initRecords() is supposed to be called once during instantiation"); } - return new Iterator() { - @Override - public boolean hasNext() { - try { - return input.available() > 0; - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public ClickHouseRecord next() { - ClickHouseRecord currentRecord = factory.get(); - try { - readAndFill(currentRecord); - readPosition = 0; - } catch (EOFException e) { - if (readPosition == 0) { // end of the stream, which is fine - throw new NoSuchElementException("No more record"); - } else { - throw new UncheckedIOException(ClickHouseUtils.format(ERROR_REACHED_END_OF_STREAM, - readPosition + 1, columns.length, columns[readPosition]), e); - } - } catch (IOException e) { - throw new UncheckedIOException( - ClickHouseUtils.format(ERROR_FAILED_TO_READ, readPosition + 1, columns.length, - columns[readPosition]), - e); - } - - return currentRecord; - } - }; + return new RecordsIterator(this); } /** @@ -148,44 +220,9 @@ public ClickHouseRecord next() { */ protected Iterator initValues() { if (readPosition != 0) { - throw new IllegalStateException("This method is supposed to be called once during instantiation"); + throw new IllegalStateException("initValues() is supposed to be called once during instantiation"); } - - return new Iterator() { - @Override - public boolean hasNext() { - try { - return input.available() > 0; - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public ClickHouseValue next() { - ClickHouseColumn column = columns[readPosition]; - ClickHouseValue value = config.isReuseValueWrapper() ? templates[readPosition] - : ClickHouseValues.newValue(config, column); - try { - templates[readPosition] = read(value, column); - if (++readPosition >= columns.length) { - readPosition = 0; - } - } catch (EOFException e) { - if (readPosition == 0) { // end of the stream, which is fine - throw new NoSuchElementException("No more value"); - } else { - throw new UncheckedIOException(ClickHouseUtils.format(ERROR_REACHED_END_OF_STREAM, - readPosition + 1, columns.length, column), e); - } - } catch (IOException e) { - throw new UncheckedIOException( - ClickHouseUtils.format(ERROR_FAILED_TO_READ, readPosition + 1, columns.length, column), e); - } - - return value; - } - }; + return new ValuesIterator(this); } // protected abstract Object initWriter(); @@ -250,9 +287,7 @@ protected ClickHouseDataProcessor(ClickHouseConfig config, ClickHouseInputStream if (settings == null || settings.isEmpty()) { this.settings = Collections.emptyMap(); } else { - Map map = new HashMap<>(); - map.putAll(settings); - this.settings = Collections.unmodifiableMap(map); + this.settings = Collections.unmodifiableMap(new HashMap<>(settings)); } if (columns == null && input != null) { @@ -265,23 +300,24 @@ protected ClickHouseDataProcessor(ClickHouseConfig config, ClickHouseInputStream } else { int len = columns.size(); int idx = 0; - List list = new ArrayList<>(len); + this.columns = new ClickHouseColumn[len]; this.templates = new ClickHouseValue[len]; for (ClickHouseColumn column : columns) { column.setColumnIndex(idx, len); - list.add(column); + this.columns[idx] = column; if (config.isReuseValueWrapper()) { - templates[idx] = ClickHouseValues.newValue(config, column); + this.templates[idx] = ClickHouseValues.newValue(config, column); } idx++; } - this.columns = list.toArray(ClickHouseColumn.EMPTY_ARRAY); } if (this.columns.length == 0 || input == null) { + this.currentRecord = ClickHouseRecord.EMPTY; this.records = Collections.emptyIterator(); this.values = Collections.emptyIterator(); } else { + this.currentRecord = createRecord(); this.records = ClickHouseChecker.nonNull(initRecords(), "Records"); this.values = ClickHouseChecker.nonNull(initValues(), "Values"); } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataStreamFactory.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataStreamFactory.java index 8bfea1847..db09ed0cb 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataStreamFactory.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataStreamFactory.java @@ -4,10 +4,14 @@ import java.util.List; import java.util.Map; +import com.clickhouse.client.config.ClickHouseBufferingMode; +import com.clickhouse.client.config.ClickHouseClientOption; import com.clickhouse.client.config.ClickHouseDefaults; import com.clickhouse.client.data.ClickHousePipedStream; import com.clickhouse.client.data.ClickHouseRowBinaryProcessor; import com.clickhouse.client.data.ClickHouseTabSeparatedProcessor; +import com.clickhouse.client.stream.BlockingPipedOutputStream; +import com.clickhouse.client.stream.NonBlockingPipedOutputStream; /** * Factory class for creating objects to handle data stream. @@ -121,11 +125,52 @@ public ClickHouseSerializer getSerializer(ClickHouseFormat form * * @param config non-null configuration * @return piped stream + * @deprecated will be removed in v0.3.3, please use + * {@link #createPipedOutputStream(ClickHouseConfig, Runnable)} + * instead */ + @Deprecated public ClickHousePipedStream createPipedStream(ClickHouseConfig config) { - ClickHouseChecker.nonNull(config, "config"); + return config != null + ? new ClickHousePipedStream(config.getWriteBufferSize(), config.getMaxQueuedBuffers(), + config.getSocketTimeout()) + : new ClickHousePipedStream((int) ClickHouseClientOption.WRITE_BUFFER_SIZE.getDefaultValue(), + (int) ClickHouseClientOption.MAX_QUEUED_BUFFERS.getDefaultValue(), + (int) ClickHouseClientOption.SOCKET_TIMEOUT.getDefaultValue()); + } + + /** + * Creates a piped output stream. + * + * @param config non-null configuration + * @return piped output stream + */ + public ClickHousePipedOutputStream createPipedOutputStream(ClickHouseConfig config, Runnable postCloseAction) { + final int bufferSize = ClickHouseChecker.nonNull(config, "config").getWriteBufferSize(); + final boolean blocking; + final int queue; + final int timeout; + + if (config.getResponseBuffering() == ClickHouseBufferingMode.PERFORMANCE) { + blocking = false; + queue = 0; + timeout = 0; // questionable + } else { + blocking = config.isUseBlockingQueue(); + queue = config.getMaxQueuedBuffers(); + timeout = config.getSocketTimeout(); + } + return blocking + ? new BlockingPipedOutputStream(bufferSize, queue, timeout, postCloseAction) + : new NonBlockingPipedOutputStream(bufferSize, queue, timeout, null, postCloseAction); + } - return new ClickHousePipedStream(config.getWriteBufferSize(), config.getMaxQueuedBuffers(), - config.getSocketTimeout()); + public ClickHousePipedOutputStream createPipedOutputStream(int writeBufferSize, int queueSize, int timeout, + Runnable postCloseAction) { + return new BlockingPipedOutputStream( + ClickHouseUtils.getBufferSize(writeBufferSize, + (int) ClickHouseClientOption.WRITE_BUFFER_SIZE.getDefaultValue(), + (int) ClickHouseClientOption.MAX_BUFFER_SIZE.getDefaultValue()), + queueSize, timeout, postCloseAction); } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataUpdater.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataUpdater.java index 9cf174521..2551b67eb 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataUpdater.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataUpdater.java @@ -1,5 +1,7 @@ package com.clickhouse.client; +import java.io.IOException; + /** * This class defines custom reading/writing logic, which can be used in * {@link ClickHouseInputStream#readCustom(ClickHouseDataUpdater)} and @@ -17,6 +19,7 @@ public interface ClickHouseDataUpdater { * it should always greater than or equal to {@code position} * @return negative number, usually -1, indicates to more to update, or other * number for bytes being updated + * @throws IOException when it failed to update */ - int update(byte[] bytes, int position, int limit); + int update(byte[] bytes, int position, int limit) throws IOException; } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java index 34b3aa21b..7285cf483 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java @@ -94,7 +94,7 @@ public static ClickHouseInputStream of(BlockingQueue queue, int time * * @param deferredInput non-null deferred input stream * @param bufferSize buffer size which is always greater than zero(usually - * 4096 or larger) + * 8192 or larger) * @param postCloseAction custom action will be performed right after closing * the input stream * @return wrapped input @@ -108,7 +108,7 @@ public static ClickHouseInputStream of(ClickHouseDeferredValue defe * Wraps the given input stream. * * @param input input stream - * @param bufferSize buffer size which is always greater than zero(usually 4096 + * @param bufferSize buffer size which is always greater than zero(usually 8192 * or larger) * @return wrapped input, or the same input if it's instance of * {@link ClickHouseInputStream} @@ -122,7 +122,7 @@ public static ClickHouseInputStream of(InputStream input, int bufferSize) { * * @param input input stream * @param bufferSize buffer size which is always greater than zero(usually - * 4096 or larger) + * 8192 or larger) * @param postCloseAction custom action will be performed right after closing * the input stream * @return wrapped input, or the same input if it's instance of @@ -136,7 +136,7 @@ public static ClickHouseInputStream of(InputStream input, int bufferSize, Runnab * Wraps the given input stream. * * @param input input stream - * @param bufferSize buffer size which is always greater than zero(usually 4096 + * @param bufferSize buffer size which is always greater than zero(usually 8192 * or larger) * @param compression compression algorithm, null or * {@link ClickHouseCompression#NONE} means no compression @@ -152,7 +152,7 @@ public static ClickHouseInputStream of(InputStream input, int bufferSize, ClickH * * @param input input stream * @param bufferSize buffer size which is always greater than zero(usually - * 4096 or larger) + * 8192 or larger) * @param compression compression algorithm, null or * {@link ClickHouseCompression#NONE} means no * compression @@ -360,8 +360,9 @@ public static ClickHouseInputStream of(Iterable source, Class clazz, F } /** - * Pipes data from input stream to output stream. Input stream will be closed - * but output stream will remain open. + * Transfers data from input stream to output stream. Input stream will be + * closed but output stream will remain open. Please pay attention that you need + * to explictly call {@code output.flush()} before closing output stream. * * @param input non-null input stream, which will be closed * @param output non-null output stream, which will remain open @@ -372,6 +373,10 @@ public static ClickHouseInputStream of(Iterable source, Class clazz, F * data to output stream */ public static long pipe(InputStream input, OutputStream output, int bufferSize) throws IOException { + if (input instanceof ClickHouseInputStream && output instanceof ClickHouseOutputStream) { + return ((ClickHouseInputStream) input).pipe((ClickHouseOutputStream) output); + } + bufferSize = ClickHouseUtils.getBufferSize(bufferSize, (int) ClickHouseClientOption.WRITE_BUFFER_SIZE.getDefaultValue(), (int) ClickHouseClientOption.MAX_BUFFER_SIZE.getDefaultValue()); @@ -379,8 +384,9 @@ public static long pipe(InputStream input, OutputStream output, int bufferSize) } /** - * Pipes data from input stream to output stream. Input stream will be closed - * but output stream will remain open. + * Transfers data from input stream to output stream. Input stream will be + * closed but output stream will remain open. Please pay attention that you need + * to explictly call {@code output.flush()} before closing output stream. * * @param input non-null input stream, which will be closed * @param output non-null output stream, which will remain open @@ -390,19 +396,20 @@ public static long pipe(InputStream input, OutputStream output, int bufferSize) * data to output stream */ public static long pipe(InputStream input, OutputStream output, byte[] buffer) throws IOException { - if (input == null || output == null || buffer == null || buffer.length < 1) { + if (buffer == null && input instanceof ClickHouseInputStream && output instanceof ClickHouseOutputStream) { + return ((ClickHouseInputStream) input).pipe((ClickHouseOutputStream) output); + } else if (input == null || output == null || buffer == null || buffer.length < 1) { throw new IllegalArgumentException("Non-null input, output, and write buffer are required"); } int size = buffer.length; - long count = 0; + long count = 0L; int written = 0; try { while ((written = input.read(buffer, 0, size)) >= 0) { output.write(buffer, 0, written); count += written; } - output.flush(); input.close(); input = null; } finally { @@ -427,12 +434,25 @@ public static long pipe(InputStream input, OutputStream output, byte[] buffer) t protected final Runnable postCloseAction; protected boolean closed; + protected OutputStream copyTo; - protected ClickHouseInputStream(Runnable postCloseAction) { + protected ClickHouseInputStream(OutputStream copyTo, Runnable postCloseAction) { this.byteBuffer = ClickHouseByteBuffer.newInstance(); this.postCloseAction = postCloseAction; this.closed = false; + this.copyTo = copyTo; + } + + /** + * Closes the input stream quietly. + */ + protected void closeQuietly() { + try { + close(); + } catch (IOException e) { + // ignore + } } /** @@ -447,17 +467,6 @@ protected void ensureOpen() throws IOException { } } - /** - * Closes the input stream quietly. - */ - protected void closeQuietly() { - try { - close(); - } catch (IOException e) { - // ignore - } - } - /** * Peeks one byte. It's similar as {@link #read()} except it never changes * cursor. @@ -645,7 +654,7 @@ public String readUnicodeString(int byteLength) throws IOException { } /** - * Read varint from input stream. + * Reads a varint from input stream. * * @return varint * @throws IOException when failed to read value from input stream or reached @@ -653,19 +662,77 @@ public String readUnicodeString(int byteLength) throws IOException { */ public int readVarInt() throws IOException { // https://github.com/ClickHouse/ClickHouse/blob/abe314feecd1647d7c2b952a25da7abf5c19f352/src/IO/VarInt.h#L126 - long result = 0L; - int shift = 0; - for (int i = 0; i < 9; i++) { - // gets 7 bits from next byte - byte b = readByte(); - result |= (b & 0x7F) << shift; - if ((b & 0x80) == 0) { + int b = readByte(); + if (b >= 0) { + return b; + } + + int result = b & 0x7F; + for (int shift = 7; shift <= 28; shift += 7) { + if ((b = readByte()) >= 0) { + result |= b << shift; break; + } else { + result |= (b & 0x7F) << shift; } - shift += 7; } + // consume a few more bytes - readVarLong() should be called instead + if (b < 0) { + for (int shift = 35; shift <= 63; shift += 7) { + if (peek() < 0 || readByte() >= 0) { + break; + } + } + } + return result; + } - return (int) result; + /** + * Reads 64-bit varint as long from input stream. + * + * @return 64-bit varint + * @throws IOException when failed to read value from input stream or reached + * end of the stream + */ + public long readVarLong() throws IOException { + long b = readByte(); + if (b >= 0L) { + return b; + } + + long result = b & 0x7F; + for (int shift = 7; shift <= 63; shift += 7) { + if ((b = readByte()) >= 0) { + result |= b << shift; + break; + } else { + result |= (b & 0x7F) << shift; + } + } + return result; + } + + /** + * Sets target output stream to copy bytes to. This is mainly used for + * testing, for example: dump input into a file while reading. + * + * @param out the output stream to write bytes to + * @throws IOException when failed to flush previous target or not able to write + * remaining bytes in buffer to the given output stream + */ + public final void setCopyToTarget(OutputStream out) throws IOException { + if (this.copyTo != null) { + this.copyTo.flush(); + } else if (out != null) { + // process remaining bytes in current buffer + readCustom((b, p, l) -> { + if (p < l) { + out.write(b, p, l - p); + } + return 0; + }); + } + this.copyTo = out; } /** diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseOutputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseOutputStream.java index 10b6aa203..288598ee6 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseOutputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseOutputStream.java @@ -36,7 +36,7 @@ public static ClickHouseOutputStream of(OutputStream output) { * Wraps the given output stream. * * @param output non-null output stream - * @param bufferSize buffer size which is always greater than zero(usually 4096 + * @param bufferSize buffer size which is always greater than zero(usually 8192 * or larger) * @return wrapped output, or the same output if it's instance of * {@link ClickHouseOutputStream} @@ -50,7 +50,7 @@ public static ClickHouseOutputStream of(OutputStream output, int bufferSize) { * * @param output non-null output stream * @param bufferSize buffer size which is always greater than zero(usually - * 4096 or larger) + * 8192 or larger) * @param compression compression algorithm, null or * {@link ClickHouseCompression#NONE} means no * compression @@ -100,6 +100,55 @@ protected void ensureOpen() throws IOException { } } + /** + * Transfers bytes into output stream without creating a copy. + * + * @param bytes non-null byte array + * @return current output stream + * @throws IOException when failed to write value into output stream, not able + * to sent all bytes, or opereate on a closed stream + */ + public final ClickHouseOutputStream transferBytes(byte[] bytes) throws IOException { + return transferBytes(bytes, 0, bytes.length); + } + + /** + * Transfer bytes into output stream without creating a copy. + * + * @param buffer non-null byte buffer + * @param length bytes to write + * @return current output stream + * @throws IOException when failed to write value into output stream, not able + * to sent all bytes, or opereate on a closed stream + */ + public ClickHouseOutputStream transferBytes(ByteBuffer buffer, int length) throws IOException { + if (buffer == null || length < 0) { + throw new IllegalArgumentException("Non-null ByteBuffer and positive length are required"); + } + + byte[] bytes; + if (buffer.hasArray()) { + bytes = buffer.array(); + } else { + bytes = new byte[length]; + buffer.get(bytes); + } + return transferBytes(bytes, 0, length); + } + + /** + * Transfers the given bytes into output stream, without creating a copy as in + * {@link #writeBytes(byte[], int, int)}. + * + * @param bytes non-null byte array + * @param offset offset of the byte array + * @param length bytes to write + * @return current output stream + * @throws IOException when failed to write value into output stream, not able + * to sent all bytes, or opereate on a closed stream + */ + public abstract ClickHouseOutputStream transferBytes(byte[] bytes, int offset, int length) throws IOException; + @Override public final void write(int b) throws IOException { writeByte((byte) (0xFF & b)); @@ -187,7 +236,7 @@ public final ClickHouseOutputStream writeBytes(byte[] bytes) throws IOException } /** - * Writes bytes into output stream. + * Writes copy of given bytes into output stream. * * @param bytes non-null byte array * @param offset offset of the byte array diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHousePipedOutputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHousePipedOutputStream.java new file mode 100644 index 000000000..e3ba22362 --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHousePipedOutputStream.java @@ -0,0 +1,40 @@ +package com.clickhouse.client; + +/** + * SPSC(Single-producer single-consumer) channel for streaming. + */ +public abstract class ClickHousePipedOutputStream extends ClickHouseOutputStream { + protected ClickHousePipedOutputStream(Runnable postCloseAction) { + super(postCloseAction); + } + + /** + * Gets input stream to reada data being written into the output stream. + * + * @return non-null input stream + * @deprecated will be removed in v0.3.3, please use {@link #getInputStream()} + * instead + */ + @Deprecated + public final ClickHouseInputStream getInput() { + return getInputStream(null); + } + + /** + * Gets input stream to reada data being written into the output stream. + * + * @return non-null input stream + */ + public final ClickHouseInputStream getInputStream() { + return getInputStream(null); + } + + /** + * Gets input stream to reada data being written into the output stream. + * + * @param postCloseAction custom action will be performed right after closing + * the input stream + * @return non-null input stream + */ + public abstract ClickHouseInputStream getInputStream(Runnable postCloseAction); +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java index 058df11e8..6ba29be93 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java @@ -90,7 +90,8 @@ public Mutation data(String file, ClickHouseCompression compression) { final String fileName = ClickHouseChecker.nonEmpty(file, "File"); this.input = changeProperty(PROP_DATA, this.input, ClickHouseDeferredValue.of(() -> { try { - return ClickHouseInputStream.of(new FileInputStream(fileName), 123, compression); + return ClickHouseInputStream.of(new FileInputStream(fileName), + self.getConfig().getReadBufferSize(), compression); } catch (FileNotFoundException e) { throw new IllegalArgumentException(e); } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponse.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponse.java index 0143fdcd6..ff0488824 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponse.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponse.java @@ -114,7 +114,8 @@ default ClickHouseRecord firstRecord() { Iterable records(); /** - * Pipes the contents of this response into the given output stream. + * Pipes the contents of this response into the given output stream. Keep in + * mind that it's caller's responsibility to flush and close the output stream. * * @param output non-null output stream, which will remain open * @param bufferSize buffer size, 0 or negative value will be treated as @@ -122,17 +123,10 @@ default ClickHouseRecord firstRecord() { * @throws IOException when error occurred reading or writing data */ default void pipe(OutputStream output, int bufferSize) throws IOException { - ClickHouseChecker.nonNull(output, "output"); - - byte[] buffer = new byte[ClickHouseUtils.getBufferSize(bufferSize, - (int) ClickHouseClientOption.WRITE_BUFFER_SIZE.getDefaultValue(), - (int) ClickHouseClientOption.MAX_BUFFER_SIZE.getDefaultValue())]; - int counter = 0; - while ((counter = getInputStream().read(buffer, 0, bufferSize)) >= 0) { - output.write(buffer, 0, counter); - } - - // caller's responsibility to call output.flush() as needed + ClickHouseInputStream.pipe(getInputStream(), ClickHouseChecker.nonNull(output, "output"), + ClickHouseUtils.getBufferSize(bufferSize, + (int) ClickHouseClientOption.WRITE_BUFFER_SIZE.getDefaultValue(), + (int) ClickHouseClientOption.MAX_BUFFER_SIZE.getDefaultValue())); } /** diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java index a4cd2ffe7..a99f927fb 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java @@ -1,9 +1,18 @@ package com.clickhouse.client; +import java.lang.Thread.UncaughtExceptionHandler; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; +import com.clickhouse.client.logging.Logger; +import com.clickhouse.client.logging.LoggerFactory; + public class ClickHouseThreadFactory implements ThreadFactory { + private static final Logger log = LoggerFactory.getLogger(ClickHouseThreadFactory.class); + + private static final UncaughtExceptionHandler hanlder = (t, e) -> log.warn("Uncaught exception from thread: " + t, + e); + private final boolean daemon; private final int priority; @@ -43,7 +52,7 @@ public Thread newThread(Runnable r) { if (priority != t.getPriority()) { t.setPriority(priority); } - // t.setUncaughtExceptionHandler(null); + t.setUncaughtExceptionHandler(hanlder); return t; } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java index 8186aa4af..8f8a2d264 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java @@ -43,7 +43,14 @@ public final class ClickHouseUtils { : Paths.get(System.getProperty("user.home"), ".clickhouse").toFile().getAbsolutePath(); } - public static final int DEFAULT_BUFFER_SIZE = 4096; + /** + * Default buffer size. + */ + public static final int DEFAULT_BUFFER_SIZE = 8192; + + /** + * Maximum buffer size. + */ public static final int MAX_BUFFER_SIZE = Integer.MAX_VALUE - 8; public static final String VARIABLE_PREFIX = "{{"; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java index 89c44b4be..fcb20f761 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java @@ -1155,7 +1155,7 @@ private static ClickHouseValue newValue(ClickHouseConfig config, ClickHouseDataT } break; case Array: - if (column == null) { + if (column == null || column.getArrayBaseColumn().isNullable()) { value = ClickHouseArrayValue.ofEmpty(); } else if (column.getArrayNestedLevel() > 1) { value = ClickHouseArrayValue.of( diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseBufferingMode.java b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseBufferingMode.java new file mode 100644 index 000000000..d765ebfe9 --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseBufferingMode.java @@ -0,0 +1,25 @@ +package com.clickhouse.client.config; + +/** + * Supported buffering mode for dealing with request and response. + */ +public enum ClickHouseBufferingMode { + // TODO Adaptive / Dynamic + + /** + * Resource-efficient mode provides reasonable performance with least CPU and + * memory usage, which makes it ideal as default mode. Only buffer size is + * considered in this mode, no queue is used for buffering. + */ + RESOURCE_EFFICIENT, + /** + * Custom mode allows you to customize buffer size and queue to balance resource + * utilization and desired performance. + */ + CUSTOM, + /** + * Performance mode provides best performance at the cost of more CPU and much + * much more memory usage - almost everything is loaded into working memory. + */ + PERFORMANCE +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java index fb5add0f9..1cc9e7c86 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java @@ -20,20 +20,30 @@ public enum ClickHouseClientOption implements ClickHouseOption { */ ASYNC("async", true, "Whether the client should run in async mode."), /** - * Default buffer size in byte for both read and write. It will be reset to - * {@link #MAX_BUFFER_SIZE} if it's too large. + * Default buffer size in byte for both request and response. It will be reset + * to {@link #MAX_BUFFER_SIZE} if it's too large. */ - BUFFER_SIZE("buffer_size", 4096, "Default buffer size in byte for both read and write."), + BUFFER_SIZE("buffer_size", 8192, "Default buffer size in byte for both request and response."), /** - * Read buffer size in byte. It defaults to {@link #BUFFER_SIZE}, and it will be + * Read buffer size in byte. It's mainly for input stream(e.g. reading data from + * server response). Its value defaults to {@link #BUFFER_SIZE}, and it will be * reset to {@link #MAX_BUFFER_SIZE} when it's too large. */ READ_BUFFER_SIZE("read_buffer_size", BUFFER_SIZE.getDefaultValue(), "Read buffer size in byte"), /** - * Write buffer size in byte. It defaults to {@link #BUFFER_SIZE}, and it will + * Write buffer size in byte. It's mainly for output stream(e.g. writing data + * into request). Its value defaults to {@link #BUFFER_SIZE}, and it will * be reset to {@link #MAX_BUFFER_SIZE} when it's too large. */ - WRITE_BUFFER_SIZE("write_buffer_size", BUFFER_SIZE.getDefaultValue(), "Read buffer size in byte"), + WRITE_BUFFER_SIZE("write_buffer_size", BUFFER_SIZE.getDefaultValue(), "Write buffer size in byte"), + /** + * Request buffering mode. + */ + REQUEST_BUFFERING("request_buffering", ClickHouseDefaults.BUFFERING, "Request buffering mode"), + /** + * Response buffering mode. + */ + RESPONSE_BUFFERING("response_buffering", ClickHouseDefaults.BUFFERING, "Response buffering mode."), /** * Client name. */ @@ -96,7 +106,10 @@ public enum ClickHouseClientOption implements ClickHouseOption { /** * Maximum comression block size in byte, only useful when {@link #DECOMPRESS} * is {@code true}. + * + * @deprecated will be removed in v0.3.3 */ + @Deprecated MAX_COMPRESS_BLOCK_SIZE("max_compress_block_size", 1024 * 1024, "Maximum comression block size in byte."), /** * Maximum query execution time in seconds. @@ -105,7 +118,7 @@ public enum ClickHouseClientOption implements ClickHouseOption { /** * Maximum queued in-memory buffers. */ - MAX_QUEUED_BUFFERS("max_queued_buffers", 0, + MAX_QUEUED_BUFFERS("max_queued_buffers", 512, "Maximum queued in-memory buffers, 0 or negative number means no limit."), /** * Maxium queued requests. When {@link #MAX_THREADS_PER_CLIENT} is greater than @@ -184,6 +197,10 @@ public enum ClickHouseClientOption implements ClickHouseOption { * SSL key. */ SSL_KEY("sslkey", "", "RSA key in PKCS#8 format."), + /** + * Whether to use blocking queue for buffering. + */ + USE_BLOCKING_QUEUE("use_blocking_queue", true, "Whether to use blocking queue for buffering"), /** * Whether to use objects in array or not. */ diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseDefaults.java b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseDefaults.java index c9d114918..0ea76936a 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseDefaults.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseDefaults.java @@ -26,6 +26,10 @@ public enum ClickHouseDefaults implements ClickHouseOption { * Whether to create session automatically when there are multiple queries. */ AUTO_SESSION("auto_session", true, "Whether to create session automatically when there are multiple queries."), + /** + * Default buffering mode. + */ + BUFFERING("buffering", ClickHouseBufferingMode.RESOURCE_EFFICIENT, "Buffering mode."), /** * Default cluster. */ diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseOption.java b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseOption.java index 9e5dddcd5..5230446d4 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseOption.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseOption.java @@ -113,7 +113,6 @@ default Optional getDefaultValueFromSysProp() { * * @return effective default value */ - @SuppressWarnings("unchecked") default Serializable getEffectiveDefaultValue() { Optional value = getDefaultValueFromEnvVar(); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java index dbf5bcd91..07ced4001 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java @@ -702,7 +702,7 @@ public static void writeUnsignedInt8(OutputStream output, int value) throws IOEx * end of the stream */ public static short readInt16(ClickHouseInputStream input) throws IOException { - return (short) (input.readUnsignedByte() | (input.readByte() << 8)); + return input.readBuffer(2).asShort(); } /** @@ -739,7 +739,7 @@ public static void writeInt16(OutputStream output, int value) throws IOException * end of the stream */ public static int readUnsignedInt16(ClickHouseInputStream input) throws IOException { - return (int) (readInt16(input) & 0xFFFF); + return input.readBuffer(2).asUnsignedShort(); } /** @@ -764,7 +764,7 @@ public static void writeUnsignedInt16(OutputStream output, int value) throws IOE * end of the stream */ public static int readInt32(ClickHouseInputStream input) throws IOException { - return input.readBuffer(4).asInt32(); + return input.readBuffer(4).asInteger(); } /** @@ -789,7 +789,7 @@ public static void writeInt32(OutputStream output, int value) throws IOException * end of the stream */ public static long readUnsignedInt32(ClickHouseInputStream input) throws IOException { - return 0xFFFFFFFFL & readInt32(input); + return input.readBuffer(4).asUnsignedInteger(); } /** @@ -814,7 +814,7 @@ public static void writeUnsignedInt32(OutputStream output, long value) throws IO * end of the stream */ public static long readInt64(ClickHouseInputStream input) throws IOException { - return input.readBuffer(8).asInt64(); + return input.readBuffer(8).asLong(); } /** @@ -840,7 +840,7 @@ public static void writeInt64(OutputStream output, long value) throws IOExceptio * end of the stream */ public static BigInteger readUnsignedInt64(ClickHouseInputStream input) throws IOException { - return input.readBuffer(8).asUnsignedInt64(); + return input.readBuffer(8).asUnsignedLong(); } /** diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java index b18a2b146..2d609af54 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java @@ -5,13 +5,14 @@ import java.nio.ByteBuffer; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import com.clickhouse.client.ClickHouseByteBuffer; import com.clickhouse.client.ClickHouseDataUpdater; import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseOutputStream; +import com.clickhouse.client.ClickHousePipedOutputStream; import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.config.ClickHouseClientOption; @@ -20,8 +21,13 @@ * {@link java.io.PipedInputStream} for streaming data between server and * client. To avoid dead lock and high memory usage, please make sure writer and * reader are on two separate threads. + * + * @deprecated will be removed in v0.3.3, please use + * {@code ClickHouseDataStreamFactory.getInstance().createPipedStream()} + * instead */ -public class ClickHousePipedStream extends ClickHouseOutputStream { +@Deprecated +public class ClickHousePipedStream extends ClickHousePipedOutputStream { protected final BlockingQueue queue; private final int bufferSize; @@ -33,7 +39,7 @@ public ClickHousePipedStream(int bufferSize, int queueLength, int timeout) { super(null); // DisruptorBlockingQueue? Did not see much difference here... - this.queue = queueLength <= 0 ? new LinkedBlockingDeque<>() : new ArrayBlockingQueue<>(queueLength); + this.queue = queueLength <= 0 ? new LinkedBlockingQueue<>() : new ArrayBlockingQueue<>(queueLength); // may need an initialBufferSize and a monitor to update bufferSize in runtime this.bufferSize = ClickHouseUtils.getBufferSize(bufferSize, @@ -44,39 +50,45 @@ public ClickHousePipedStream(int bufferSize, int queueLength, int timeout) { this.buffer = ByteBuffer.allocate(this.bufferSize); } - private void updateBuffer() throws IOException { - if (buffer.hasRemaining()) { - ((Buffer) buffer).limit(buffer.position()); + private void updateBuffer(boolean allocateNewBuffer) throws IOException { + ByteBuffer b = buffer; + if (b.hasRemaining()) { + ((Buffer) b).limit(b.position()); } - ((Buffer) buffer).rewind(); + ((Buffer) b).rewind(); try { if (timeout > 0) { - if (!queue.offer(buffer, timeout, TimeUnit.MILLISECONDS)) { + if (!queue.offer(b, timeout, TimeUnit.MILLISECONDS)) { throw new IOException(ClickHouseUtils.format("Write timed out after %d ms", timeout)); } } else { - queue.put(buffer); + queue.put(b); } } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException("Thread was interrupted when putting buffer into queue", e); } - buffer = ByteBuffer.allocate(bufferSize); + if (allocateNewBuffer) { + buffer = ByteBuffer.allocate(bufferSize); + } } - public ClickHouseInputStream getInput() { - return ClickHouseInputStream.of(queue, timeout); + @Override + public ClickHouseInputStream getInputStream(Runnable postCloseAction) { + return ClickHouseInputStream.of(queue, timeout, postCloseAction); } @Override public void close() throws IOException { - if (this.closed) { + if (closed) { return; } - flush(); + if (buffer.position() > 0) { + updateBuffer(false); + } buffer = ClickHouseByteBuffer.EMPTY_BUFFER; try { @@ -90,8 +102,9 @@ public void close() throws IOException { } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException("Thread was interrupted when putting EMPTY buffer into queue", e); + } finally { + super.close(); } - this.closed = true; } @Override @@ -99,17 +112,22 @@ public void flush() throws IOException { ensureOpen(); if (buffer.position() > 0) { - updateBuffer(); + updateBuffer(true); } } + @Override + public ClickHouseOutputStream transferBytes(byte[] bytes, int offset, int length) throws IOException { + return writeBytes(bytes, offset, length); + } + @Override public ClickHouseOutputStream writeByte(byte b) throws IOException { ensureOpen(); buffer.put(b); if (!buffer.hasRemaining()) { - updateBuffer(); + updateBuffer(true); } return this; } @@ -125,16 +143,25 @@ public ClickHouseOutputStream writeBytes(byte[] bytes, int offset, int length) t } ensureOpen(); + ByteBuffer b = buffer; while (length > 0) { - int remain = buffer.remaining(); + int remain = b.remaining(); if (length < remain) { + b.put(bytes, offset, length); + length = 0; + } else if (b.position() == 0) { + // buffer = ByteBuffer.wrap(bytes, offset, length); + buffer = ByteBuffer.allocate(length); buffer.put(bytes, offset, length); + updateBuffer(false); + buffer = b; length = 0; } else { - buffer.put(bytes, offset, remain); + b.put(bytes, offset, remain); offset += remain; length -= remain; - updateBuffer(); + updateBuffer(true); + b = buffer; } } @@ -160,7 +187,7 @@ public ClickHouseOutputStream writeCustom(ClickHouseDataUpdater writer) throws I written = writer.update(bytes, position, limit); if (written < 0) { ((Buffer) buffer).position(limit); - updateBuffer(); + updateBuffer(true); } } while (written < 0); ((Buffer) buffer).position(position + written); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java index dc8b46124..dce29b064 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java @@ -104,54 +104,55 @@ private void writeArray(ClickHouseValue value, ClickHouseConfig config, ClickHou private ClickHouseValue readArray(ClickHouseValue ref, ClickHouseConfig config, ClickHouseColumn nestedColumn, ClickHouseColumn baseColumn, ClickHouseInputStream input, int length, int level) throws IOException { Class javaClass = baseColumn.getPrimitiveClass(); - if (level > 1 || !javaClass.isPrimitive()) { - Object[] array = (Object[]) ClickHouseValues.createPrimitiveArray(javaClass, length, level); + if (level > 1 || baseColumn.isNullable() || !javaClass.isPrimitive()) { + Object[] array = baseColumn.isNullable() + ? ClickHouseValues.createObjectArray(baseColumn.getObjectClass(), length, level) + : (Object[]) ClickHouseValues.createPrimitiveArray(javaClass, length, level); for (int i = 0; i < length; i++) { array[i] = deserialize(null, config, nestedColumn, input).asObject(); } ref.update(array); } else { if (byte.class == javaClass) { - byte[] array = new byte[length]; - for (int i = 0; i < length; i++) { - array[i] = deserialize(null, config, baseColumn, input).asByte(); - } - ref.update(array); + ref.update(input.readBuffer(length).compact().array()); } else if (short.class == javaClass) { - short[] array = new short[length]; - for (int i = 0; i < length; i++) { - array[i] = deserialize(null, config, baseColumn, input).asShort(); + if (baseColumn.getDataType().getByteLength() != 2) { + short[] array = new short[length]; + for (int i = 0; i < length; i++) { + array[i] = deserialize(null, config, baseColumn, input).asShort(); + } + ref.update(array); + } else { + ref.update(input.readBuffer(length * 2).asShortArray()); } - ref.update(array); } else if (int.class == javaClass) { - int[] array = new int[length]; - for (int i = 0; i < length; i++) { - array[i] = deserialize(null, config, baseColumn, input).asInteger(); + if (baseColumn.getDataType().getByteLength() != 4) { + int[] array = new int[length]; + for (int i = 0; i < length; i++) { + array[i] = deserialize(null, config, baseColumn, input).asInteger(); + } + ref.update(array); + } else { + ref.update(input.readBuffer(length * 4).asIntegerArray()); } - ref.update(array); } else if (long.class == javaClass) { - long[] array = new long[length]; - for (int i = 0; i < length; i++) { - array[i] = deserialize(null, config, baseColumn, input).asLong(); + if (baseColumn.getDataType().getByteLength() != 8) { + long[] array = new long[length]; + for (int i = 0; i < length; i++) { + array[i] = deserialize(null, config, baseColumn, input).asLong(); + } + ref.update(array); + } else { + ref.update(input.readBuffer(length * 8).asLongArray()); } - ref.update(array); } else if (float.class == javaClass) { - float[] array = new float[length]; - for (int i = 0; i < length; i++) { - array[i] = deserialize(null, config, baseColumn, input).asFloat(); - } - ref.update(array); + ref.update(input.readBuffer(length * 4).asFloatArray()); } else if (double.class == javaClass) { - double[] array = new double[length]; - for (int i = 0; i < length; i++) { - array[i] = deserialize(null, config, baseColumn, input).asDouble(); - } - ref.update(array); + ref.update(input.readBuffer(length * 8).asDoubleArray()); } else { throw new IllegalArgumentException("Unsupported primitive type: " + javaClass); } } - return ref; } @@ -195,8 +196,7 @@ private void buildMappingsForDataTypes() { // enums buildMappings(deserializers, serializers, (r, f, c, i) -> ClickHouseEnumValue.of(r, c.getEnumConstants(), i.readByte()), - (v, f, c, o) -> BinaryStreamUtils.writeInt8(o, v.asByte()), ClickHouseDataType.Enum, - ClickHouseDataType.Enum8); + (v, f, c, o) -> o.writeByte(v.asByte()), ClickHouseDataType.Enum, ClickHouseDataType.Enum8); buildMappings(deserializers, serializers, (r, f, c, i) -> ClickHouseEnumValue.of(r, c.getEnumConstants(), BinaryStreamUtils.readInt16(i)), (v, f, c, o) -> BinaryStreamUtils.writeInt16(o, v.asShort()), ClickHouseDataType.Enum16); @@ -223,13 +223,13 @@ private void buildMappingsForDataTypes() { (r, f, c, i) -> ClickHouseLongValue.of(r, false, BinaryStreamUtils.readUnsignedInt32(i)), (v, f, c, o) -> BinaryStreamUtils.writeUnsignedInt32(o, v.asLong()), ClickHouseDataType.UInt32); buildMappings(deserializers, serializers, - (r, f, c, i) -> ClickHouseLongValue.of(r, false, BinaryStreamUtils.readInt64(i)), + (r, f, c, i) -> ClickHouseLongValue.of(r, false, i.readBuffer(8).asLong()), (v, f, c, o) -> BinaryStreamUtils.writeInt64(o, v.asLong()), ClickHouseDataType.IntervalYear, ClickHouseDataType.IntervalQuarter, ClickHouseDataType.IntervalMonth, ClickHouseDataType.IntervalWeek, ClickHouseDataType.IntervalDay, ClickHouseDataType.IntervalHour, ClickHouseDataType.IntervalMinute, ClickHouseDataType.IntervalSecond, ClickHouseDataType.Int64); buildMappings(deserializers, serializers, - (r, f, c, i) -> ClickHouseLongValue.of(r, true, BinaryStreamUtils.readInt64(i)), + (r, f, c, i) -> ClickHouseLongValue.of(r, true, i.readBuffer(8).asLong()), (v, f, c, o) -> BinaryStreamUtils.writeInt64(o, v.asLong()), ClickHouseDataType.UInt64); buildMappings(deserializers, serializers, (r, f, c, i) -> ClickHouseBigIntegerValue.of(r, BinaryStreamUtils.readInt128(i)), @@ -362,7 +362,7 @@ private void buildMappingsForDataTypes() { (v, f, c, o) -> serialize(v, f, c.getNestedColumns().get(0), o), ClickHouseDataType.SimpleAggregateFunction); buildMappings(deserializers, serializers, (r, f, c, i) -> { - int length = BinaryStreamUtils.readVarInt(i); + int length = i.readVarInt(); if (r == null) { r = ClickHouseValues.newValue(f, c); } @@ -373,7 +373,7 @@ private void buildMappingsForDataTypes() { Map map = new LinkedHashMap<>(); ClickHouseColumn keyCol = c.getKeyInfo(); ClickHouseColumn valCol = c.getValueInfo(); - for (int k = 0, len = BinaryStreamUtils.readVarInt(i); k < len; k++) { + for (int k = 0, len = i.readVarInt(); k < len; k++) { map.put(deserialize(null, f, keyCol, i).asObject(), deserialize(null, f, valCol, i).asObject()); } return ClickHouseMapValue.of(map, keyCol.getObjectClass(), valCol.getObjectClass()); @@ -398,7 +398,7 @@ private void buildMappingsForDataTypes() { int l = 0; for (ClickHouseColumn col : c.getNestedColumns()) { names[l] = col.getColumnName(); - int k = BinaryStreamUtils.readVarInt(i); + int k = i.readVarInt(); Object[] nvalues = new Object[k]; for (int j = 0; j < k; j++) { nvalues[j] = deserialize(null, f, col, i).asObject(); @@ -491,17 +491,37 @@ public static MappedFunctions getMappedFunctions() { return MappedFunctions.instance; } + @Override + protected ClickHouseRecord createRecord() { + return new ClickHouseSimpleRecord(getColumns(), templates); + } + @Override protected void readAndFill(ClickHouseRecord r) throws IOException { MappedFunctions m = getMappedFunctions(); - for (; readPosition < columns.length; readPosition++) { - templates[readPosition] = m.deserialize(r.getValue(readPosition), config, columns[readPosition], input); + int p = readPosition; + ClickHouseColumn[] c = columns; + ClickHouseConfig f = config; + ClickHouseInputStream i = input; + ClickHouseValue[] t = templates; + try { + for (int len = c.length; p < len; p++) { + ClickHouseValue n = m.deserialize(r.getValue(p), f, c[p], i); + if (n != t[p]) { + t[p] = n; + } + } + } finally { + readPosition = p; } } @Override protected void readAndFill(ClickHouseValue value, ClickHouseColumn column) throws IOException { - templates[readPosition] = getMappedFunctions().deserialize(value, config, column, input); + ClickHouseValue v = getMappedFunctions().deserialize(value, config, column, input); + if (v != value) { + templates[readPosition] = v; + } } @Override diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTabSeparatedProcessor.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTabSeparatedProcessor.java index 303fa6466..6c70cc408 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTabSeparatedProcessor.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTabSeparatedProcessor.java @@ -226,6 +226,11 @@ protected TextHandler getTextHandler() { return textHandler; } + @Override + protected ClickHouseRecord createRecord() { + return new ClickHouseSimpleRecord(getColumns(), templates); + } + @Override protected void readAndFill(ClickHouseRecord r) throws IOException { ClickHouseByteBuffer buf = input.readCustom(getTextHandler()::readLine); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteArrayInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteArrayInputStream.java index 31540e313..7c2a2b697 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteArrayInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteArrayInputStream.java @@ -2,6 +2,7 @@ import java.io.EOFException; import java.io.IOException; +import java.io.OutputStream; import java.util.LinkedList; import com.clickhouse.client.ClickHouseByteBuffer; @@ -18,8 +19,8 @@ public abstract class AbstractByteArrayInputStream extends ClickHouseInputStream protected int position; protected int limit; - protected AbstractByteArrayInputStream(Runnable postCloseAction) { - super(postCloseAction); + protected AbstractByteArrayInputStream(OutputStream copyTo, Runnable postCloseAction) { + super(copyTo, postCloseAction); buffer = ClickHouseByteBuffer.EMPTY_BYTES; position = 0; @@ -56,6 +57,9 @@ public int available() throws IOException { @Override public void close() throws IOException { + if (closed) { + return; + } buffer = ClickHouseByteBuffer.EMPTY_BYTES; super.close(); } @@ -75,13 +79,16 @@ public long pipe(ClickHouseOutputStream output) throws IOException { } ensureOpen(); - int remain = limit - position; + byte[] b = buffer; + int l = limit; + int p = position; + int remain = l - p; if (remain > 0) { - output.write(buffer, position, remain); + output.transferBytes(b, p, remain); count += remain; - while ((remain = updateBuffer()) > 0) { - output.write(buffer, 0, remain); + b = buffer; + output.transferBytes(b, 0, remain); count += remain; } } @@ -221,8 +228,11 @@ public byte[] readBytes(int length) throws IOException { ensureOpen(); byte[] bytes = new byte[length]; + byte[] b = buffer; int count = 0; - int remain = limit - position; + int l = limit; + int p = position; + int remain = l - p; while (length > 0) { if (remain < 1) { closeQuietly(); @@ -231,15 +241,17 @@ public byte[] readBytes(int length) throws IOException { } if (remain >= length) { - System.arraycopy(buffer, position, bytes, count, length); - position += length; + System.arraycopy(b, p, bytes, count, length); + position = p + length; count += length; length = 0; } else { - System.arraycopy(buffer, position, bytes, count, remain); + System.arraycopy(b, p, bytes, count, remain); count += remain; length -= remain; remain = updateBuffer(); + p = position; + b = buffer; } } return bytes; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteArrayOutputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteArrayOutputStream.java index a68506626..3875ebc0e 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteArrayOutputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteArrayOutputStream.java @@ -10,7 +10,12 @@ public abstract class AbstractByteArrayOutputStream extends ClickHouseOutputStre protected int position; - protected abstract void flushBuffer() throws IOException; + protected void flushBuffer() throws IOException { + flushBuffer(buffer, 0, position); + position = 0; + } + + protected abstract void flushBuffer(byte[] bytes, int offset, int length) throws IOException; protected AbstractByteArrayOutputStream(int bufferSize, Runnable postCloseAction) { super(postCloseAction); @@ -20,6 +25,25 @@ protected AbstractByteArrayOutputStream(int bufferSize, Runnable postCloseAction position = 0; } + @Override + public ClickHouseOutputStream transferBytes(byte[] bytes, int offset, int length) throws IOException { + if (bytes == null) { + throw new NullPointerException(); + } else if (offset < 0 || length < 0 || length > bytes.length - offset) { + throw new IndexOutOfBoundsException(); + } else if (length == 0) { + return this; + } + ensureOpen(); + + if (position > 0) { + flushBuffer(); + } + flushBuffer(bytes, offset, length); + position = 0; + return this; + } + @Override public ClickHouseOutputStream writeByte(byte b) throws IOException { ensureOpen(); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteBufferInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteBufferInputStream.java index 3750eb50e..3bad964fd 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteBufferInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/AbstractByteBufferInputStream.java @@ -2,6 +2,7 @@ import java.io.EOFException; import java.io.IOException; +import java.io.OutputStream; import java.nio.Buffer; import java.nio.ByteBuffer; import java.util.LinkedList; @@ -18,8 +19,8 @@ public abstract class AbstractByteBufferInputStream extends ClickHouseInputStream { protected ByteBuffer buffer; - protected AbstractByteBufferInputStream(Runnable postCloseAction) { - super(postCloseAction); + protected AbstractByteBufferInputStream(OutputStream copyTo, Runnable postCloseAction) { + super(copyTo, postCloseAction); this.buffer = null; } @@ -52,6 +53,9 @@ public int available() throws IOException { @Override public void close() throws IOException { + if (closed) { + return; + } buffer = null; super.close(); } @@ -77,19 +81,22 @@ public long pipe(ClickHouseOutputStream output) throws IOException { } ensureOpen(); - while (buffer != ClickHouseByteBuffer.EMPTY_BUFFER) { - int remain = buffer.remaining(); + ByteBuffer b = buffer; + int remain = b.remaining(); + while (b != ClickHouseByteBuffer.EMPTY_BUFFER) { if (remain > 0) { - if (buffer.hasArray()) { - output.write(buffer.array(), buffer.position(), remain); + if (b.hasArray()) { + output.transferBytes(b.array(), b.position(), remain); + b.limit(b.position()); } else { byte[] bytes = new byte[remain]; buffer.get(bytes); - output.write(bytes); + output.transferBytes(bytes, 0, remain); } count += remain; } - updateBuffer(); + remain = updateBuffer(); + b = buffer; } close(); return count; @@ -238,24 +245,25 @@ public byte[] readBytes(int length) throws IOException { byte[] bytes = new byte[length]; int offset = 0; - int len = length; - while (len > 0) { - if (buffer == ClickHouseByteBuffer.EMPTY_BUFFER) { + ByteBuffer b = buffer; + int remain = b.remaining(); + while (length > 0) { + if (b == ClickHouseByteBuffer.EMPTY_BUFFER) { closeQuietly(); throw offset == 0 ? new EOFException() - : new IOException(ClickHouseUtils.format(ERROR_INCOMPLETE_READ, offset, length)); + : new IOException(ClickHouseUtils.format(ERROR_INCOMPLETE_READ, offset, bytes.length)); } - int remain = buffer.remaining(); - if (remain >= len) { - buffer.get(bytes, offset, len); - offset += len; - len = 0; + if (remain >= length) { + b.get(bytes, offset, length); + offset += length; + length = 0; } else { - buffer.get(bytes, offset, remain); + b.get(bytes, offset, remain); offset += remain; - len -= remain; - updateBuffer(); + length -= remain; + remain = updateBuffer(); + b = buffer; } } @@ -270,8 +278,7 @@ public long skip(long n) throws IOException { if (n == Long.MAX_VALUE) { long counter = buffer.remaining(); while (buffer != ClickHouseByteBuffer.EMPTY_BUFFER) { - counter += buffer.limit(); - updateBuffer(); + counter += updateBuffer(); } return counter; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/AdaptiveQueue.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/AdaptiveQueue.java new file mode 100644 index 000000000..505047c6b --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/AdaptiveQueue.java @@ -0,0 +1,53 @@ +package com.clickhouse.client.stream; + +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; + +public final class AdaptiveQueue { + private final CapacityPolicy policy; + private final LinkedList queue; + + public AdaptiveQueue(CapacityPolicy policy, E... array) { + this.policy = policy; + this.queue = array == null || array.length == 0 ? new LinkedList<>() : new LinkedList<>(Arrays.asList(array)); + } + + public AdaptiveQueue(CapacityPolicy policy, List list) { + this.policy = policy; + this.queue = new LinkedList<>(list); + } + + public synchronized void add(E e) { + queue.add(e); + if (policy != null) { + policy.ensureCapacity(0); + } + } + + public synchronized void clear() { + queue.clear(); + if (policy != null) { + policy.ensureCapacity(0); + } + } + + public synchronized boolean offer(E e) { + if (policy == null || policy.ensureCapacity(queue.size())) { + queue.addLast(e); + return true; + } + return false; + } + + public synchronized E poll() { + if (!queue.isEmpty()) { + return queue.removeFirst(); + } + return null; + } + + public synchronized int size() { + return queue.size(); + } +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/BlockingInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/BlockingInputStream.java index 3b82135a7..8ff4b92c9 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/BlockingInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/BlockingInputStream.java @@ -1,6 +1,7 @@ package com.clickhouse.client.stream; import java.io.IOException; +import java.nio.Buffer; import java.nio.ByteBuffer; import java.util.concurrent.BlockingQueue; import java.util.concurrent.TimeUnit; @@ -21,7 +22,7 @@ public class BlockingInputStream extends AbstractByteBufferInputStream { private final int timeout; public BlockingInputStream(BlockingQueue queue, int timeout, Runnable postCloseAction) { - super(postCloseAction); + super(null, postCloseAction); this.queue = ClickHouseChecker.nonNull(queue, "Queue"); this.timeout = timeout > 0 ? timeout : 0; @@ -38,17 +39,31 @@ protected void ensureOpen() throws IOException { @Override protected int updateBuffer() throws IOException { + ByteBuffer b; try { if (timeout > 0) { - buffer = queue.poll(timeout, TimeUnit.MILLISECONDS); - if (buffer == null) { + b = queue.poll(timeout, TimeUnit.MILLISECONDS); + if (b == null) { throw new IOException(ClickHouseUtils.format("Read timed out after %d ms", timeout)); } } else { - buffer = queue.take(); + b = queue.take(); } - return buffer.remaining(); + buffer = b; + int remain = b.remaining(); + if (remain > 0 && copyTo != null) { + int position = b.position(); + if (b.hasArray()) { + copyTo.write(b.array(), position, remain); + } else { + byte[] bytes = new byte[remain]; + b.get(bytes); + copyTo.write(bytes); + ((Buffer) b).position(position); + } + } + return remain; } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException("Thread was interrupted when getting next buffer from queue", e); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/BlockingPipedOutputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/BlockingPipedOutputStream.java new file mode 100644 index 000000000..6032d5b10 --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/BlockingPipedOutputStream.java @@ -0,0 +1,210 @@ +package com.clickhouse.client.stream; + +import java.io.IOException; +import java.nio.Buffer; +import java.nio.ByteBuffer; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +import com.clickhouse.client.ClickHouseByteBuffer; +import com.clickhouse.client.ClickHouseDataUpdater; +import com.clickhouse.client.ClickHouseInputStream; +import com.clickhouse.client.ClickHouseOutputStream; +import com.clickhouse.client.ClickHousePipedOutputStream; +import com.clickhouse.client.ClickHouseUtils; +import com.clickhouse.client.config.ClickHouseClientOption; + +/** + * A combination of {@link java.io.PipedOutputStream} and + * {@link java.io.PipedInputStream} for streaming data between server and + * client. To avoid dead lock and high memory usage, please make sure writer and + * reader are on two separate threads. + */ +public class BlockingPipedOutputStream extends ClickHousePipedOutputStream { + protected final BlockingQueue queue; + + private final int bufferSize; + private final int timeout; + + private ByteBuffer buffer; + + public BlockingPipedOutputStream(int bufferSize, int queueLength, int timeout, Runnable postCloseAction) { + super(postCloseAction); + + // DisruptorBlockingQueue? Did not see much difference here... + this.queue = queueLength <= 0 ? new LinkedBlockingQueue<>() : new ArrayBlockingQueue<>(queueLength); + + // may need an initialBufferSize and a monitor to update bufferSize in runtime + this.bufferSize = ClickHouseUtils.getBufferSize(bufferSize, + (int) ClickHouseClientOption.WRITE_BUFFER_SIZE.getDefaultValue(), + (int) ClickHouseClientOption.MAX_BUFFER_SIZE.getDefaultValue()); + this.timeout = timeout; + + this.buffer = ByteBuffer.allocate(this.bufferSize); + } + + private void updateBuffer(boolean allocateNewBuffer) throws IOException { + ByteBuffer b = buffer; + if (b.hasRemaining()) { + ((Buffer) b).limit(b.position()); + } + ((Buffer) b).rewind(); + + updateBuffer(b); + + if (allocateNewBuffer) { + buffer = ByteBuffer.allocate(bufferSize); + } + } + + private void updateBuffer(ByteBuffer b) throws IOException { + try { + if (timeout > 0) { + if (!queue.offer(b, timeout, TimeUnit.MILLISECONDS)) { + throw new IOException(ClickHouseUtils.format("Write timed out after %d ms", timeout)); + } + } else { + queue.put(b); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Thread was interrupted when putting buffer into queue", e); + } + } + + @Override + public ClickHouseInputStream getInputStream(Runnable postCloseAction) { + return new BlockingInputStream(queue, timeout, postCloseAction); + } + + @Override + public void close() throws IOException { + if (closed) { + return; + } + + if (buffer.position() > 0) { + updateBuffer(false); + } + + buffer = ClickHouseByteBuffer.EMPTY_BUFFER; + try { + if (timeout > 0) { + if (!queue.offer(buffer, timeout, TimeUnit.MILLISECONDS)) { + throw new IOException(ClickHouseUtils.format("Close stream timed out after %d ms", timeout)); + } + } else { + queue.put(buffer); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Thread was interrupted when putting EMPTY buffer into queue", e); + } finally { + super.close(); + } + } + + @Override + public void flush() throws IOException { + ensureOpen(); + + if (buffer.position() > 0) { + updateBuffer(true); + } + } + + @Override + public ClickHouseOutputStream transferBytes(byte[] bytes, int offset, int length) throws IOException { + if (bytes == null) { + throw new NullPointerException(); + } else if (offset < 0 || length < 0 || length > bytes.length - offset) { + throw new IndexOutOfBoundsException(); + } else if (length == 0) { + return this; + } + ensureOpen(); + + ByteBuffer b = buffer; + if (b.position() > 0) { + updateBuffer(true); + } + updateBuffer(ByteBuffer.wrap(bytes, offset, length)); + + return this; + } + + @Override + public ClickHouseOutputStream writeByte(byte b) throws IOException { + ensureOpen(); + + buffer.put(b); + if (!buffer.hasRemaining()) { + updateBuffer(true); + } + return this; + } + + @Override + public ClickHouseOutputStream writeBytes(byte[] bytes, int offset, int length) throws IOException { + if (bytes == null) { + throw new NullPointerException(); + } else if (offset < 0 || length < 0 || length > bytes.length - offset) { + throw new IndexOutOfBoundsException(); + } else if (length == 0) { + return this; + } + ensureOpen(); + + ByteBuffer b = buffer; + while (length > 0) { + int remain = b.remaining(); + if (length < remain) { + b.put(bytes, offset, length); + length = 0; + } else if (b.position() == 0) { + // buffer = ByteBuffer.wrap(bytes, offset, length); + buffer = ByteBuffer.allocate(length); + buffer.put(bytes, offset, length); + updateBuffer(false); + buffer = b; + length = 0; + } else { + b.put(bytes, offset, remain); + offset += remain; + length -= remain; + updateBuffer(true); + b = buffer; + } + } + + return this; + } + + @Override + public ClickHouseOutputStream writeCustom(ClickHouseDataUpdater writer) throws IOException { + ensureOpen(); + + int position = 0; + int written = 0; + do { + position = buffer.position(); + int limit = buffer.limit(); + byte[] bytes; + if (buffer.hasArray()) { + bytes = buffer.array(); + } else { + bytes = new byte[limit - position]; + buffer.get(bytes); + } + written = writer.update(bytes, position, limit); + if (written < 0) { + ((Buffer) buffer).position(limit); + updateBuffer(true); + } + } while (written < 0); + ((Buffer) buffer).position(position + written); + return this; + } +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/CapacityPolicy.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/CapacityPolicy.java new file mode 100644 index 000000000..25573af95 --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/CapacityPolicy.java @@ -0,0 +1,56 @@ +package com.clickhouse.client.stream; + +@FunctionalInterface +public interface CapacityPolicy { + static class FixedCapacity implements CapacityPolicy { + private final int capacity; + + protected FixedCapacity(int capacity) { + this.capacity = capacity < 1 ? 0 : capacity; + } + + @Override + public boolean ensureCapacity(int current) { + return capacity < 1 || current < capacity; + } + } + + static class LinearDynamicCapacity implements CapacityPolicy { + private volatile int capacity; + private volatile int count; + + private final int maxSize; + private final int threshold; + + protected LinearDynamicCapacity(int initialSize, int maxSize, int threshold) { + this.capacity = initialSize < 1 ? 1 : initialSize; + this.count = 0; + + this.maxSize = maxSize < 1 ? Integer.MAX_VALUE : Math.max(maxSize, initialSize); + this.threshold = threshold < 1 ? 100 : threshold; + } + + @Override + public boolean ensureCapacity(int current) { + if (current < capacity) { + count = 0; + return true; + } else if (capacity < maxSize && ++count >= threshold) { + count = 0; + capacity++; + return true; + } + return false; + } + } + + static CapacityPolicy fixedCapacity(int capacity) { + return new FixedCapacity(capacity); + } + + static CapacityPolicy linearDynamicCapacity(int initialSize, int maxSize, int threshold) { + return new LinearDynamicCapacity(initialSize, maxSize, threshold); + } + + boolean ensureCapacity(int current); +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/EmptyInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/EmptyInputStream.java index 0add51560..5fe211039 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/EmptyInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/EmptyInputStream.java @@ -15,7 +15,7 @@ public final class EmptyInputStream extends ClickHouseInputStream { public static final EmptyInputStream INSTANCE = new EmptyInputStream(); private EmptyInputStream() { - super(null); + super(null, null); } @Override diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableByteArrayInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableByteArrayInputStream.java index dad6eb1bb..7dbb7b842 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableByteArrayInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableByteArrayInputStream.java @@ -1,5 +1,6 @@ package com.clickhouse.client.stream; +import java.io.IOException; import java.util.Iterator; import com.clickhouse.client.ClickHouseByteBuffer; @@ -9,13 +10,13 @@ public class IterableByteArrayInputStream extends AbstractByteArrayInputStream { private final Iterator it; public IterableByteArrayInputStream(Iterable source, Runnable postCloseAction) { - super(postCloseAction); + super(null, postCloseAction); it = ClickHouseChecker.nonNull(source, "Source").iterator(); } @Override - protected int updateBuffer() { + protected int updateBuffer() throws IOException { position = 0; while (it.hasNext()) { @@ -23,6 +24,9 @@ protected int updateBuffer() { int len = bytes != null ? bytes.length : 0; if (len > 0) { buffer = bytes; + if (copyTo != null) { + copyTo.write(bytes); + } return limit = len; } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableByteBufferInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableByteBufferInputStream.java index 42769dd7f..038ce6b9c 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableByteBufferInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableByteBufferInputStream.java @@ -1,6 +1,7 @@ package com.clickhouse.client.stream; import java.io.IOException; +import java.nio.Buffer; import java.nio.ByteBuffer; import java.util.Iterator; @@ -11,7 +12,7 @@ public class IterableByteBufferInputStream extends AbstractByteBufferInputStream private final Iterator it; public IterableByteBufferInputStream(Iterable source, Runnable postCloseAction) { - super(postCloseAction); + super(null, postCloseAction); it = ClickHouseChecker.nonNull(source, "Source").iterator(); } @@ -22,6 +23,17 @@ protected int updateBuffer() throws IOException { ByteBuffer bytes = it.next(); if (bytes != null && bytes.hasRemaining()) { buffer = bytes; + if (copyTo != null) { + int position = bytes.position(); + if (bytes.hasArray()) { + copyTo.write(bytes.array(), position, bytes.remaining()); + } else { + byte[] b = new byte[bytes.remaining()]; + bytes.get(b); + copyTo.write(b); + ((Buffer) bytes).position(position); + } + } return bytes.remaining(); } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableMultipleInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableMultipleInputStream.java index b0a38c148..fa4556779 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableMultipleInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableMultipleInputStream.java @@ -51,14 +51,16 @@ protected int updateBuffer() throws IOException { len -= read; } } - + if (copyTo != null) { + copyTo.write(buffer, 0, off); + } limit = off; return limit - position; } public IterableMultipleInputStream(Iterable source, Function converter, Runnable postCloseAction) { - super(postCloseAction); + super(null, postCloseAction); func = ClickHouseChecker.nonNull(converter, "Converter"); it = ClickHouseChecker.nonNull(source, "Source").iterator(); @@ -77,27 +79,30 @@ public void close() throws IOException { return; } - LinkedList errors = new LinkedList<>(); try { - in.close(); - } catch (Exception e) { - errors.add(e.getMessage()); - } - - while (it.hasNext()) { + LinkedList errors = new LinkedList<>(); try { - InputStream i = func.apply(it.next()); - if (i != null) { - i.close(); - } + in.close(); } catch (Exception e) { errors.add(e.getMessage()); } - } - closed = true; - if (!errors.isEmpty()) { - throw new IOException("Failed to close input stream: " + String.join("\n", errors)); + while (it.hasNext()) { + try { + InputStream i = func.apply(it.next()); + if (i != null) { + i.close(); + } + } catch (Exception e) { + errors.add(e.getMessage()); + } + } + + if (!errors.isEmpty()) { + throw new IOException("Failed to close input stream: " + String.join("\n", errors)); + } + } finally { + super.close(); } } @@ -146,12 +151,11 @@ public long pipe(ClickHouseOutputStream output) throws IOException { if (output == null || output.isClosed()) { return count; } - ensureOpen(); int remain = limit - position; if (remain > 0) { - output.write(buffer, position, remain); + output.transferBytes(buffer, position, remain); count += remain; position = limit; } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableObjectInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableObjectInputStream.java index 21dfe5cc1..d9bab532f 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableObjectInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/IterableObjectInputStream.java @@ -1,5 +1,6 @@ package com.clickhouse.client.stream; +import java.io.IOException; import java.util.Iterator; import java.util.function.Function; @@ -11,20 +12,23 @@ public class IterableObjectInputStream extends AbstractByteArrayInputStream { private final Iterator it; public IterableObjectInputStream(Iterable source, Function converter, Runnable postCloseAction) { - super(postCloseAction); + super(null, postCloseAction); func = ClickHouseChecker.nonNull(converter, "Converter"); it = ClickHouseChecker.nonNull(source, "Source").iterator(); } @Override - protected int updateBuffer() { + protected int updateBuffer() throws IOException { position = 0; while (it.hasNext()) { T obj = it.next(); byte[] bytes = obj != null ? func.apply(obj) : null; if (bytes != null && bytes.length > 0) { buffer = bytes; + if (copyTo != null) { + copyTo.write(bytes); + } return limit = bytes.length; } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/Lz4InputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/Lz4InputStream.java index b0cd2941a..353531cdf 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/Lz4InputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/Lz4InputStream.java @@ -76,6 +76,9 @@ protected int updateBuffer() throws IOException { buffer = new byte[uncompressedSize]; decompressor.decompress(block, offset, buffer, 0, uncompressedSize); + if (copyTo != null) { + copyTo.write(buffer); + } return limit = buffer.length; } @@ -84,7 +87,7 @@ public Lz4InputStream(InputStream stream) { } public Lz4InputStream(InputStream stream, Runnable postCloseAction) { - super(postCloseAction); + super(null, postCloseAction); this.decompressor = factory.fastDecompressor(); this.stream = ClickHouseChecker.nonNull(stream, "InputStream"); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/Lz4OutputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/Lz4OutputStream.java index b18d88297..f70ff7e8d 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/Lz4OutputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/Lz4OutputStream.java @@ -31,6 +31,22 @@ protected void flushBuffer() throws IOException { position = 0; } + @Override + protected void flushBuffer(byte[] bytes, int offset, int length) throws IOException { + int maxLen = compressor.maxCompressedLength(length) + 15; + byte[] block = maxLen < compressedBlock.length ? compressedBlock : new byte[maxLen]; + block[16] = Lz4InputStream.MAGIC; + + int compressed = compressor.compress(bytes, offset, length, block, 25); + int compressedSizeWithHeader = compressed + 9; + BinaryStreamUtils.setInt32(block, 17, compressedSizeWithHeader); + BinaryStreamUtils.setInt32(block, 21, length); + long[] hash = ClickHouseCityHash.cityHash128(block, 16, compressedSizeWithHeader); + BinaryStreamUtils.setInt64(block, 0, hash[0]); + BinaryStreamUtils.setInt64(block, 8, hash[1]); + output.write(block, 0, compressed + 25); + } + public Lz4OutputStream(OutputStream stream, int maxCompressBlockSize, Runnable postCloseAction) { super(maxCompressBlockSize, postCloseAction); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/NonBlockingInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/NonBlockingInputStream.java new file mode 100644 index 000000000..42b31eec1 --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/NonBlockingInputStream.java @@ -0,0 +1,311 @@ +package com.clickhouse.client.stream; + +import java.io.EOFException; +import java.io.IOException; +import java.util.LinkedList; + +import com.clickhouse.client.ClickHouseByteBuffer; +import com.clickhouse.client.ClickHouseChecker; +import com.clickhouse.client.ClickHouseDataUpdater; +import com.clickhouse.client.ClickHouseInputStream; +import com.clickhouse.client.ClickHouseOutputStream; +import com.clickhouse.client.ClickHouseUtils; +import com.clickhouse.client.logging.Logger; +import com.clickhouse.client.logging.LoggerFactory; + +public class NonBlockingInputStream extends ClickHouseInputStream { + private static final Logger log = LoggerFactory.getLogger(NonBlockingInputStream.class); + + private final AdaptiveQueue queue; + private final int timeout; + + private byte[] buffer; + private int position; + + public NonBlockingInputStream(AdaptiveQueue queue, int timeout, Runnable postCloseAction) { + super(null, postCloseAction); + + this.queue = ClickHouseChecker.nonNull(queue, "Queue"); + this.timeout = timeout > 0 ? timeout : 0; + + this.buffer = null; + this.position = 0; + } + + @Override + protected void ensureOpen() throws IOException { + if (closed) { + log.debug("Blocking input stream(queue: %d, buffer: %d) has been closed", + queue.size(), buffer != null ? buffer.length - position : 0); + } + super.ensureOpen(); + + if (buffer == null || (buffer != ClickHouseByteBuffer.EMPTY_BYTES && position >= buffer.length)) { + updateBuffer(); + } + } + + /** + * Update inner byte array along with {@code position} and {@code limit} as + * needed. + * + * @return remaining bytes in buffer + * @throws IOException when failed to read value + */ + protected int updateBuffer() throws IOException { + AdaptiveQueue q = queue; + long t = timeout; + long startTime = t < 1L ? 0L : System.currentTimeMillis(); + byte[] b; + while ((b = q.poll()) == null) { + if (closed) { + throw new IOException("Cannot operate on a closed input stream"); + } else if (t > 0L && System.currentTimeMillis() - startTime >= t) { + throw new IOException(ClickHouseUtils.format("Read timed out after %d ms", t)); + } + } + + buffer = b; + position = 0; + int remain = b.length; + if (remain > 0 && copyTo != null) { + copyTo.write(b, 0, remain); + } + return remain; + } + + @Override + public int available() throws IOException { + if (closed || buffer == ClickHouseByteBuffer.EMPTY_BYTES) { + return 0; + } + + int remain = buffer != null ? buffer.length - position : 0; + return remain > 0 ? remain : updateBuffer(); + } + + @Override + public void close() throws IOException { + if (closed) { + return; + } + buffer = ClickHouseByteBuffer.EMPTY_BYTES; + super.close(); + } + + @Override + public int peek() throws IOException { + ensureOpen(); + + return buffer != ClickHouseByteBuffer.EMPTY_BYTES ? 0xFF & buffer[position] : -1; + } + + @Override + public long pipe(ClickHouseOutputStream output) throws IOException { + long count = 0L; + if (output == null || output.isClosed()) { + return count; + } + ensureOpen(); + + byte[] b = buffer; + int l = b.length; + int p = position; + int remain = l - p; + if (remain > 0) { + output.transferBytes(b, p, remain); + count += remain; + while ((remain = updateBuffer()) > 0) { + b = buffer; + output.transferBytes(b, 0, remain); + count += remain; + } + } + close(); + return count; + } + + @Override + public int read() throws IOException { + ensureOpen(); + + return buffer != ClickHouseByteBuffer.EMPTY_BYTES ? 0xFF & buffer[position++] : -1; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + if (b == null) { + throw new NullPointerException(ERROR_NULL_BYTES); + } else if (buffer == b) { + // in case b is the byte array return from ClickHouseByteBuffer.array() + throw new IllegalArgumentException(ERROR_REUSE_BUFFER); + } else if (off < 0 || len < 0 || len > b.length - off) { + throw new IndexOutOfBoundsException(); + } else if (len == 0) { + return 0; + } + ensureOpen(); + + int remain = buffer.length - position; + if (remain < 1) { + return -1; + } else if (remain >= len) { + System.arraycopy(buffer, position, b, off, len); + position += len; + return len; + } else { + System.arraycopy(buffer, position, b, off, remain); + off += remain; + len -= remain; + } + + int count = remain; // should be greater than zero + while (len > 0 && (remain = updateBuffer()) > 0) { + if (remain >= len) { + System.arraycopy(buffer, position, b, off, len); + position += len; + return count + len; + } else { + System.arraycopy(buffer, position, b, off, remain); + off += remain; + len -= remain; + count += remain; + } + } + + return count; + } + + @Override + public ClickHouseByteBuffer readBuffer(int length) throws IOException { + if (length < 1) { + return byteBuffer.reset(); + } + ensureOpen(); + + int limit = buffer.length; + if (position >= limit) { + closeQuietly(); + throw new EOFException(); + } + + int newLimit = position + length; + if (limit >= newLimit) { + byteBuffer.update(buffer, position, length); + position = newLimit; + } else { + byteBuffer.update(readBytes(length)); + } + return byteBuffer; + } + + @Override + public ClickHouseByteBuffer readCustom(ClickHouseDataUpdater reader) throws IOException { + if (reader == null) { + return byteBuffer.reset(); + } + ensureOpen(); + + LinkedList list = new LinkedList<>(); + int offset = position; + int length = 0; + boolean more = true; + while (more) { + int limit = buffer.length; + int remain = limit - position; + if (remain < 1) { + closeQuietly(); + more = false; + } else { + int read = reader.update(buffer, position, limit); + if (read == -1) { + length += remain; + position = limit; + list.add(buffer); + if (updateBuffer() < 1) { + closeQuietly(); + more = false; + } + } else { + if (read > 0) { + length += read; + position += read; + list.add(buffer); + } + more = false; + } + } + } + return byteBuffer.update(list, offset, length); + } + + @Override + public byte readByte() throws IOException { + ensureOpen(); + + if (position >= buffer.length) { + closeQuietly(); + throw new EOFException(); + } + + return buffer[position++]; + } + + @Override + public byte[] readBytes(int length) throws IOException { + if (length < 1) { + return ClickHouseByteBuffer.EMPTY_BYTES; + } + ensureOpen(); + + byte[] bytes = new byte[length]; + byte[] b = buffer; + int count = 0; + int l = buffer.length; + int p = position; + int remain = l - p; + while (length > 0) { + if (remain < 1) { + closeQuietly(); + throw count == 0 ? new EOFException() + : new IOException(ClickHouseUtils.format(ERROR_INCOMPLETE_READ, count, bytes.length)); + } + + if (remain >= length) { + System.arraycopy(b, p, bytes, count, length); + position = p + length; + count += length; + length = 0; + } else { + System.arraycopy(b, p, bytes, count, remain); + count += remain; + length -= remain; + remain = updateBuffer(); + p = position; + b = buffer; + } + } + return bytes; + } + + @Override + public long skip(long n) throws IOException { + ensureOpen(); + + if (buffer == ClickHouseByteBuffer.EMPTY_BYTES) { + return 0L; + } + + // peforms better but this is a bit tricky + if (n == Long.MAX_VALUE) { + long counter = (long) buffer.length - position; + while (updateBuffer() > 0) { + counter += buffer.length; + } + + return counter; + } + + return super.skip(n); + } +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/NonBlockingPipedOutputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/NonBlockingPipedOutputStream.java new file mode 100644 index 000000000..ff3175923 --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/NonBlockingPipedOutputStream.java @@ -0,0 +1,203 @@ +package com.clickhouse.client.stream; + +import java.io.IOException; + +import com.clickhouse.client.ClickHouseByteBuffer; +import com.clickhouse.client.ClickHouseDataUpdater; +import com.clickhouse.client.ClickHouseInputStream; +import com.clickhouse.client.ClickHouseOutputStream; +import com.clickhouse.client.ClickHousePipedOutputStream; +import com.clickhouse.client.ClickHouseUtils; +import com.clickhouse.client.config.ClickHouseClientOption; + +/** + * A combination of {@link java.io.PipedOutputStream} and + * {@link java.io.PipedInputStream} for streaming data between server and + * client. To avoid dead lock and high memory usage, please make sure writer and + * reader are on two separate threads. + */ +public class NonBlockingPipedOutputStream extends ClickHousePipedOutputStream { + protected final AdaptiveQueue queue; + + protected final int bufferSize; + protected final int timeout; + protected final byte[][] buckets; + + protected int current; + + protected byte[] buffer; + protected int position; + + private byte[] allocateBuffer() { + position = 0; + byte[] b; + if (buckets.length - queue.size() > 1) { + b = buckets[current]; + if (b == null) { + b = new byte[bufferSize]; + buckets[current] = b; + } + + if (++current >= buckets.length) { + current = 0; + } + } else { + b = new byte[bufferSize]; + } + return b; + } + + private void updateBuffer(boolean allocateNewBuffer) throws IOException { + updateBuffer(buffer, 0, position); + + if (allocateNewBuffer) { + buffer = allocateBuffer(); + } else { + position = 0; + } + } + + private void updateBuffer(byte[] bytes, int offset, int length) throws IOException { + byte[] b; + if (length < buffer.length) { + b = new byte[length]; + System.arraycopy(bytes, offset, b, 0, length); + } else { + b = bytes; + } + + AdaptiveQueue q = queue; + long t = timeout; + long startTime = t < 1L ? 0L : System.currentTimeMillis(); + + // wait as needed + while (!q.offer(b)) { + if (t > 0L && System.currentTimeMillis() - startTime >= t) { + throw new IOException(ClickHouseUtils.format("Write timed out after %d ms", t)); + } + } + } + + public NonBlockingPipedOutputStream(int bufferSize, int queueLength, int timeout, CapacityPolicy policy, + Runnable postCloseAction) { + super(postCloseAction); + + this.queue = new AdaptiveQueue<>( + policy != null ? policy : CapacityPolicy.linearDynamicCapacity(1, queueLength, 0)); + + // may need an initialBufferSize and a monitor to update bufferSize in runtime + this.bufferSize = ClickHouseUtils.getBufferSize(bufferSize, + (int) ClickHouseClientOption.WRITE_BUFFER_SIZE.getDefaultValue(), + (int) ClickHouseClientOption.MAX_BUFFER_SIZE.getDefaultValue()); + this.timeout = timeout; + this.buckets = queueLength < 2 ? new byte[0][] : new byte[queueLength][]; + + this.current = queueLength < 2 ? -1 : 0; + this.buffer = allocateBuffer(); + } + + @Override + public ClickHouseInputStream getInputStream(Runnable postCloseAction) { + return new NonBlockingInputStream(queue, timeout, postCloseAction); + } + + @Override + public void close() throws IOException { + if (closed) { + return; + } + + try { + if (position > 0) { + updateBuffer(false); + } + } finally { + queue.add(buffer = ClickHouseByteBuffer.EMPTY_BYTES); + for (int i = 0, len = buckets.length; i < len; i++) { + buckets[i] = null; + } + super.close(); + } + } + + @Override + public void flush() throws IOException { + ensureOpen(); + + if (position > 0) { + updateBuffer(true); + } + } + + @Override + public ClickHouseOutputStream transferBytes(byte[] bytes, int offset, int length) throws IOException { + if (bytes == null) { + throw new NullPointerException(); + } else if (offset < 0 || length < 0 || length > bytes.length - offset) { + throw new IndexOutOfBoundsException(); + } else if (length == 0) { + return this; + } + ensureOpen(); + + if (position > 0) { + updateBuffer(true); + } + updateBuffer(bytes, offset, length); + + return this; + } + + @Override + public ClickHouseOutputStream writeByte(byte b) throws IOException { + ensureOpen(); + + buffer[position++] = b; + if (position >= buffer.length) { + updateBuffer(true); + } + return this; + } + + @Override + public ClickHouseOutputStream writeBytes(byte[] bytes, int offset, int length) throws IOException { + if (bytes == null) { + throw new NullPointerException(); + } else if (offset < 0 || length < 0 || length > bytes.length - offset) { + throw new IndexOutOfBoundsException(); + } else if (length == 0) { + return this; + } + ensureOpen(); + + while (length > 0) { + int limit = buffer.length; + int remain = limit - position; + if (length < remain) { + System.arraycopy(bytes, offset, buffer, position, length); + position += length; + length = 0; + } else { + System.arraycopy(bytes, offset, buffer, position, remain); + position = limit; + offset += remain; + length -= remain; + updateBuffer(true); + } + } + return this; + } + + @Override + public ClickHouseOutputStream writeCustom(ClickHouseDataUpdater writer) throws IOException { + ensureOpen(); + + int written = 0; + while ((written = writer.update(buffer, position, buffer.length)) < 0) { + position = buffer.length; + updateBuffer(true); + } + position += written; + return this; + } +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/WrappedInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/WrappedInputStream.java index 016874b51..088e30447 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/WrappedInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/WrappedInputStream.java @@ -36,13 +36,15 @@ protected int updateBuffer() throws IOException { len -= read; } } - + if (copyTo != null) { + copyTo.write(buffer, 0, off); + } limit = off; return limit - position; } public WrappedInputStream(InputStream input, int bufferSize, Runnable postCloseAction) { - super(postCloseAction); + super(null, postCloseAction); in = ClickHouseChecker.nonNull(input, "InputStream"); // fixed buffer @@ -112,18 +114,19 @@ public long pipe(ClickHouseOutputStream output) throws IOException { if (output == null || output.isClosed()) { return count; } - ensureOpen(); - int remain = limit - position; + int l = limit; + int p = position; + int remain = l - p; if (remain > 0) { - output.write(buffer, position, remain); + output.writeBytes(buffer, p, remain); count += remain; - position = limit; + position = l; } - while ((remain = in.read(buffer)) != -1) { - output.write(buffer, 0, remain); + while ((remain = updateBuffer()) > 0) { + output.writeBytes(buffer, 0, remain); count += remain; } close(); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/stream/WrappedOutputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/stream/WrappedOutputStream.java index 76b955863..ff4febe62 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/stream/WrappedOutputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/stream/WrappedOutputStream.java @@ -14,9 +14,8 @@ public class WrappedOutputStream extends AbstractByteArrayOutputStream { private final OutputStream output; @Override - protected void flushBuffer() throws IOException { - output.write(buffer, 0, position); - position = 0; + protected void flushBuffer(byte[] bytes, int offset, int length) throws IOException { + output.write(bytes, offset, length); } public WrappedOutputStream(OutputStream stream, int bufferSize, Runnable postCloseAction) { diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseClientTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseClientTest.java index 2dd8c22a2..9cd1c7c3e 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseClientTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseClientTest.java @@ -2,9 +2,36 @@ import org.testng.Assert; import org.testng.annotations.Test; + +import java.io.ByteArrayOutputStream; + import com.clickhouse.client.ClickHouseRequest.Mutation; public class ClickHouseClientTest { + @Test(groups = { "unit" }) + public void testGetAsyncRequestOutputStream() throws Exception { + ClickHouseConfig config = new ClickHouseConfig(); + for (int i = 0; i < 256; i++) { + ByteArrayOutputStream bas = new ByteArrayOutputStream(); + try (ClickHouseOutputStream chOut = ClickHouseClient.getAsyncRequestOutputStream(config, bas, null)) { + chOut.write(i); + } + Assert.assertEquals(bas.toByteArray(), new byte[] { (byte) i }); + } + } + + @Test(groups = { "unit" }) + public void testGetRequestOutputStream() throws Exception { + ClickHouseConfig config = new ClickHouseConfig(); + for (int i = 0; i < 256; i++) { + ByteArrayOutputStream bas = new ByteArrayOutputStream(); + try (ClickHouseOutputStream chOut = ClickHouseClient.getRequestOutputStream(config, bas, null)) { + chOut.write(i); + } + Assert.assertEquals(bas.toByteArray(), new byte[] { (byte) i }); + } + } + @Test(groups = { "unit" }) public void testQuery() throws Exception { ClickHouseClient client = ClickHouseClient.builder().build(); diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseDataStreamFactoryTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseDataStreamFactoryTest.java index 0b00a5c38..f8335f359 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseDataStreamFactoryTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseDataStreamFactoryTest.java @@ -1,5 +1,7 @@ package com.clickhouse.client; +import java.util.concurrent.CompletableFuture; + import org.testng.Assert; import org.testng.annotations.Test; @@ -8,4 +10,40 @@ public class ClickHouseDataStreamFactoryTest { public void testGetInstance() throws Exception { Assert.assertNotNull(ClickHouseDataStreamFactory.getInstance()); } + + @Test(groups = { "unit" }) + public void testCreatePipedOutputStream() throws Exception { + ClickHouseConfig config = new ClickHouseConfig(); + + // read in worker thread + for (int i = 0; i < 256; i++) { + CompletableFuture future; + try (ClickHousePipedOutputStream out = ClickHouseDataStreamFactory.getInstance() + .createPipedOutputStream(config, null)) { + future = ClickHouseClient.submit(() -> { + try (ClickHouseInputStream in = out.getInputStream()) { + return in.read(); + } + }); + out.write(i); + } + Assert.assertEquals(future.get(), i); + } + + // write in worker thread + for (int i = 0; i < 256; i++) { + ClickHousePipedOutputStream out = ClickHouseDataStreamFactory.getInstance().createPipedOutputStream(config, + null); + final int num = i; + try (ClickHouseInputStream in = out.getInputStream()) { + ClickHouseClient.submit(() -> { + try (ClickHouseOutputStream o = out) { + o.write(num); + } + return num; + }); + Assert.assertEquals(in.read(), num); + } + } + } } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java index 2cd8ce28d..0caa0eab6 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java @@ -65,7 +65,8 @@ public void testNullEmptyOrClosedInput() throws IOException { public void testWrappedInput() throws IOException { int sample = 10000; byte[] bytes = new byte[sample]; - try (InputStream in = generateInputStream(bytes); ClickHouseInputStream chIn = ClickHouseInputStream.of(in)) { + try (InputStream in = generateInputStream(bytes); + ClickHouseInputStream chIn = ClickHouseInputStream.of(in)) { for (int i = 0; i < sample; i++) { Assert.assertTrue(chIn.available() > 0); Assert.assertEquals(chIn.readByte(), bytes[i]); @@ -78,7 +79,8 @@ public void testWrappedInput() throws IOException { Assert.assertTrue(chIn.isClosed(), "Should have been closed automatically"); } - try (InputStream in = generateInputStream(bytes); ClickHouseInputStream chIn = ClickHouseInputStream.of(in)) { + try (InputStream in = generateInputStream(bytes); + ClickHouseInputStream chIn = ClickHouseInputStream.of(in)) { Assert.assertEquals(chIn.readBytes(sample), bytes); Assert.assertFalse(chIn.isClosed(), "Should not be closed"); Assert.assertThrows(EOFException.class, () -> chIn.readBytes(1)); @@ -181,24 +183,112 @@ public void testBlockingInputAsync() throws IOException { Assert.assertTrue(in.isClosed(), "Should have been closed"); } + @Test(groups = { "unit" }) + public void testReadVarInt() throws IOException { + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { 0x00 }).readVarInt(), 0); + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { 0x01 }).readVarInt(), 1); + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { 0x02 }).readVarInt(), 2); + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { (byte) 0x7F }).readVarInt(), 127); + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { (byte) 0x80, 0x01 }).readVarInt(), 128); + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { (byte) 0xFF, 0x01 }).readVarInt(), 255); + Assert.assertEquals( + ClickHouseInputStream.of(new byte[] { (byte) 0xDD, (byte) 0xC7, 0x01 }).readVarInt(), + 25565); + Assert.assertEquals( + ClickHouseInputStream.of(new byte[] { (byte) 0xFF, (byte) 0xFF, 0x7F }).readVarInt(), + 2097151); + Assert.assertEquals( + ClickHouseInputStream.of( + new byte[] { (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, 0x07 }) + .readVarInt(), + 2147483647); + Assert.assertEquals( + ClickHouseInputStream + .of(new byte[] { (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, + 0x07, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0x7F }) + .readVarInt(), + 2147483647); + Assert.assertEquals(ClickHouseInputStream + .of(new byte[] { (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, 0x0F }) + .readVarInt(), -1); + Assert.assertEquals( + ClickHouseInputStream + .of(new byte[] { (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, 0x08 }) + .readVarInt(), + -2147483648); + Assert.assertEquals( + ClickHouseInputStream + .of(new byte[] { (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, + 0x08, (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, 0x01 }) + .readVarInt(), + -2147483648); + } + + @Test(groups = { "unit" }) + public void testReadVarLong() throws IOException { + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { 0x00 }).readVarLong(), 0L); + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { 0x01 }).readVarLong(), 1L); + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { 0x02 }).readVarLong(), 2L); + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { (byte) 0x7F }).readVarLong(), 127L); + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { (byte) 0x80, 0x01 }).readVarLong(), 128L); + Assert.assertEquals(ClickHouseInputStream.of(new byte[] { (byte) 0xFF, 0x01 }).readVarLong(), 255L); + Assert.assertEquals( + ClickHouseInputStream.of(new byte[] { (byte) 0xDD, (byte) 0xC7, 0x01 }).readVarLong(), + 25565L); + Assert.assertEquals( + ClickHouseInputStream.of(new byte[] { (byte) 0xFF, (byte) 0xFF, 0x7F }).readVarLong(), + 2097151L); + Assert.assertEquals( + ClickHouseInputStream.of( + new byte[] { (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, 0x07 }) + .readVarLong(), + 2147483647L); + Assert.assertEquals( + ClickHouseInputStream + .of(new byte[] { (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, + (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0x7F }) + .readVarLong(), + 9223372036854775807L); + Assert.assertEquals(ClickHouseInputStream + .of(new byte[] { (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, + (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0x01 }) + .readVarLong(), -1L); + Assert.assertEquals( + ClickHouseInputStream + .of(new byte[] { (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, + (byte) 0xF8, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, 0x01 }) + .readVarLong(), + -2147483648L); + Assert.assertEquals( + ClickHouseInputStream + .of(new byte[] { (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, + (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, 0x01 }) + .readVarLong(), + -9223372036854775808L); + } + @Test(groups = { "unit" }) public void testSkipInput() throws IOException { Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[0])).skip(0L), 0L); Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[0])).skip(1L), 0L); - Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[0])).skip(Long.MAX_VALUE), 0L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[0])).skip(Long.MAX_VALUE), + 0L); Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[1])).skip(0L), 0L); Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[1])).skip(1L), 1L); - Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[1])).skip(Long.MAX_VALUE), 1L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[1])).skip(Long.MAX_VALUE), + 1L); Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[2])).skip(0L), 0L); Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[2])).skip(1L), 1L); - Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[2])).skip(Long.MAX_VALUE), 2L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[2])).skip(Long.MAX_VALUE), + 2L); Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[5]), 4).skip(0L), 0L); Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[3]), 2).skip(1L), 1L); Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[7]), 6).skip(2L), 2L); - Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[8]), 6).skip(Long.MAX_VALUE), 8L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[8]), 6).skip(Long.MAX_VALUE), + 8L); ClickHouseInputStream in = ClickHouseInputStream .of(new ByteArrayInputStream(new byte[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 }), 8); diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java index 11d936d4c..304e448cb 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java @@ -80,8 +80,8 @@ public void testChangeListener() { final List changedSettings = new ArrayList<>(); ClickHouseConfigChangeListener> listener = new ClickHouseConfigChangeListener>() { @Override - public void optionChanged(ClickHouseRequest source, ClickHouseOption option, Serializable oldValue, - Serializable newValue) { + public void optionChanged(ClickHouseRequest source, ClickHouseOption option, + Serializable oldValue, Serializable newValue) { changedOptions.add(new Object[] { source, option, oldValue, newValue }); } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseValuesTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseValuesTest.java index e20b2b8dc..bb0bb6215 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseValuesTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseValuesTest.java @@ -70,11 +70,16 @@ public void testCreateArray() { public void testNewArray() { ClickHouseConfig config = new ClickHouseConfig(); ClickHouseValue v = ClickHouseValues.newValue(config, ClickHouseColumn.of("a", "Array(UInt32)")); - Assert.assertEquals(v.asObject(), new long[0]); + Assert.assertEquals(v.update(new long[] { 1L }).asObject(), new long[] { 1L }); + v = ClickHouseValues.newValue(config, ClickHouseColumn.of("a", "Array(Nullable(UInt64))")); + Assert.assertEquals(v.update(new Long[] { 1L }).asObject(), new Long[] { 1L }); v = ClickHouseValues.newValue(config, ClickHouseColumn.of("a", "Array(Array(UInt16))")); Assert.assertEquals(v.asObject(), new int[0][]); - v = ClickHouseValues.newValue(config, ClickHouseColumn.of("a", "Array(Array(Array(Nullable(UInt8))))")); + v = ClickHouseValues.newValue(config, ClickHouseColumn.of("a", "Array(Array(Array(UInt8)))")); Assert.assertEquals(v.asObject(), new short[0][][]); + v = ClickHouseValues.newValue(config, ClickHouseColumn.of("a", "Array(Array(Array(Nullable(UInt8))))")); + Assert.assertEquals(v.update(new Short[][][] { new Short[][] { new Short[] { (short) 1 } } }).asObject(), + new Short[][][] { new Short[][] { new Short[] { (short) 1 } } }); v = ClickHouseValues.newValue(config, ClickHouseColumn.of("a", "Array(Array(Array(Array(LowCardinality(String)))))")); Assert.assertEquals(v.asObject(), new String[0][][][]); diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClientIntegrationTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClientIntegrationTest.java index 8d77d18d2..5618b7a8b 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClientIntegrationTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClientIntegrationTest.java @@ -37,7 +37,6 @@ import com.clickhouse.client.data.ClickHouseIpv6Value; import com.clickhouse.client.data.ClickHouseLongValue; import com.clickhouse.client.data.ClickHouseOffsetDateTimeValue; -import com.clickhouse.client.data.ClickHousePipedStream; import com.clickhouse.client.data.ClickHouseStringValue; import org.testng.Assert; @@ -45,8 +44,8 @@ import org.testng.annotations.Test; public abstract class ClientIntegrationTest extends BaseIntegrationTest { - protected ClickHouseResponseSummary execute(ClickHouseRequest request, String sql) throws Exception { - try (ClickHouseResponse response = request.query(sql).execute().get()) { + protected ClickHouseResponseSummary execute(ClickHouseRequest request, String sql) throws ClickHouseException { + try (ClickHouseResponse response = request.query(sql).executeAndWait()) { for (ClickHouseRecord record : response.records()) { for (ClickHouseValue value : record) { Assert.assertNotNull(value, "Value should never be null"); @@ -181,7 +180,7 @@ public void testOpenCloseClient() throws Exception { @Test(dataProvider = "compressionMatrix", groups = { "integration" }) public void testCompression(boolean compressRequest, boolean compressResponse) - throws Exception { + throws ClickHouseException { ClickHouseNode server = getServer(); String uuid = UUID.randomUUID().toString(); for (ClickHouseFormat format : new ClickHouseFormat[] { @@ -194,7 +193,7 @@ public void testCompression(boolean compressRequest, boolean compressResponse) .decompressClientRequest(compressRequest); boolean hasResult = false; try (ClickHouseResponse resp = request - .query("select :uuid").params(ClickHouseStringValue.of(uuid)).execute().get()) { + .query("select :uuid").params(ClickHouseStringValue.of(uuid)).executeAndWait()) { Assert.assertEquals(resp.firstRecord().getValue(0).asString(), uuid); hasResult = true; } @@ -203,35 +202,35 @@ public void testCompression(boolean compressRequest, boolean compressResponse) // empty results try (ClickHouseResponse resp = request .query("create database if not exists system") - .execute().get()) { + .executeAndWait()) { ClickHouseResponseSummary summary = resp.getSummary(); Assert.assertEquals(summary.getReadRows(), 0L); Assert.assertEquals(summary.getWrittenRows(), 0L); } // let's also check if failures can be captured successfully as well - Exception exp = null; + ClickHouseException exp = null; try (ClickHouseResponse resp = request .use(uuid) .query("select currentUser(), timezone(), version(), getSetting('readonly') readonly FORMAT RowBinaryWithNamesAndTypes") - .execute().get()) { + .executeAndWait()) { Assert.fail("Query should fail"); - } catch (Exception e) { + } catch (ClickHouseException e) { exp = e; } - Assert.assertEquals(((ClickHouseException) exp.getCause()).getErrorCode(), 81); + Assert.assertEquals(exp.getErrorCode(), 81); } } } @Test(groups = { "integration" }) - public void testFormat() throws Exception { + public void testFormat() throws ClickHouseException { String sql = "select 1, 2"; ClickHouseNode node = getServer(); try (ClickHouseClient client = getClient()) { try (ClickHouseResponse response = client.connect(node) - .format(ClickHouseFormat.RowBinaryWithNamesAndTypes).query(sql).execute().get()) { + .format(ClickHouseFormat.RowBinaryWithNamesAndTypes).query(sql).executeAndWait()) { Assert.assertEquals(response.getColumns().size(), 2); int counter = 0; for (ClickHouseRecord record : response.records()) { @@ -243,8 +242,8 @@ public void testFormat() throws Exception { } // now let's try again using unsupported formats - try (ClickHouseResponse response = client.connect(node).query(sql).format(ClickHouseFormat.CSV).execute() - .get()) { + try (ClickHouseResponse response = client.connect(node).query(sql).format(ClickHouseFormat.CSV) + .executeAndWait()) { String results = new BufferedReader( new InputStreamReader(response.getInputStream(), StandardCharsets.UTF_8)).lines() .collect(Collectors.joining("\n")); @@ -252,7 +251,7 @@ public void testFormat() throws Exception { } try (ClickHouseResponse response = client.connect(node).query(sql).format(ClickHouseFormat.JSONEachRow) - .execute().get()) { + .executeAndWait()) { String results = new BufferedReader( new InputStreamReader(response.getInputStream(), StandardCharsets.UTF_8)).lines() .collect(Collectors.joining("\n")); @@ -875,6 +874,34 @@ public void testQueryWithMultipleExternalTables() throws Exception { } } + @Test(groups = { "integration" }) + public void testCustomRead() throws Exception { + long limit = 1000L; + long count = 0L; + ClickHouseNode server = getServer(); + try (ClickHouseClient client = getClient()) { + ClickHouseRequest request = client.connect(server).format(ClickHouseFormat.RowBinaryWithNamesAndTypes) + .query("select * from numbers(:limit)").params(String.valueOf(limit)); + ClickHouseConfig config = request.getConfig(); + try (ClickHouseResponse response = request.executeAndWait()) { + ClickHouseInputStream input = response.getInputStream(); + List list = response.getColumns(); + ClickHouseColumn[] columns = list.toArray(new ClickHouseColumn[0]); + ClickHouseValue[] values = ClickHouseValues.newValues(config, columns); + ClickHouseDataProcessor processor = ClickHouseDataStreamFactory.getInstance() + .getProcessor(config, input, null, null, list); + int len = columns.length; + while (input.available() > 0) { + for (int i = 0; i < len; i++) { + Assert.assertEquals(processor.read(values[i], columns[i]).asLong(), count++); + } + } + } + } + + Assert.assertEquals(count, 1000L); + } + @Test(groups = { "integration" }) public void testDump() throws Exception { ClickHouseNode server = getServer(); @@ -883,7 +910,8 @@ public void testDump() throws Exception { Assert.assertEquals(Files.size(temp), 0L); int lines = 10000; - ClickHouseResponseSummary summary = ClickHouseClient.dump(server, "select * from system.numbers limit " + lines, + ClickHouseResponseSummary summary = ClickHouseClient.dump(server, + ClickHouseUtils.format("select * from numbers(%d)", lines), ClickHouseFormat.TabSeparated, ClickHouseCompression.NONE, temp.toString()).get(); Assert.assertNotNull(summary); // Assert.assertEquals(summary.getReadRows(), lines); @@ -997,10 +1025,10 @@ public void testLoadRawData() throws Exception { CompletableFuture future; // single producer → single consumer // important to close the stream *before* retrieving response - try (ClickHousePipedStream stream = new ClickHousePipedStream(config.getWriteBufferSize(), - config.getMaxQueuedRequests(), config.getSocketTimeout())) { + try (ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance() + .createPipedOutputStream(config, null)) { // start the worker thread which transfer data from the input into ClickHouse - future = request.data(stream.getInput()).send(); + future = request.data(stream.getInputStream()).send(); // write bytes into the piped stream for (int i = 0; i < rows; i++) { BinaryStreamUtils.writeInt64(stream, i); diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHousePipedStreamTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHousePipedStreamTest.java index db0896ea0..e793f3bd5 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHousePipedStreamTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHousePipedStreamTest.java @@ -3,11 +3,15 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.Buffer; +import java.util.Arrays; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import com.clickhouse.client.ClickHouseByteBuffer; @@ -207,13 +211,16 @@ public void testWriteBytes() throws Exception { @Test(groups = { "unit" }) public void testPipedStream() throws Exception { + final int timeout = 10000; ExecutorService executor = Executors.newFixedThreadPool(2); for (int bufferSize = -1; bufferSize < 10; bufferSize++) { for (int queueLength = -1; queueLength < 10; queueLength++) { - ClickHousePipedStream stream = new ClickHousePipedStream(bufferSize, queueLength, 10000); + ClickHousePipedStream stream = new ClickHousePipedStream(bufferSize, queueLength, timeout); try (InputStream in = stream.getInput(); OutputStream out = stream) { - int count = 10000; - CountDownLatch latch = new CountDownLatch(count + 1); + final int count = 10000; + final AtomicInteger p = new AtomicInteger(0); + final AtomicInteger n = new AtomicInteger(0); + final CountDownLatch latch = new CountDownLatch(count + 1); executor.execute(() -> { for (int i = 0; i < count; i++) { byte[] bytes = new byte[] { (byte) (0xFF & i), (byte) (0xFF & i + 1), @@ -238,23 +245,39 @@ public void testPipedStream() throws Exception { (byte) (0xFF & i + 2) }; byte[] b = new byte[bytes.length]; try { - Assert.assertEquals(in.read(b), b.length); - latch.countDown(); - Assert.assertEquals(b, bytes); + if (in.read(b) == b.length && Arrays.equals(b, bytes)) { + p.incrementAndGet(); + } else { + n.incrementAndGet(); + } } catch (IOException e) { - Assert.fail("Failed to read", e); + Thread.currentThread().interrupt(); + throw new UncheckedIOException(e); + } finally { + latch.countDown(); } } try { - Assert.assertEquals(in.read(), -1); - latch.countDown(); + if (in.read() == -1) { + p.incrementAndGet(); + } else { + n.incrementAndGet(); + } } catch (IOException e) { - Assert.fail("Failed to read EOF", e); + Thread.currentThread().interrupt(); + throw new UncheckedIOException(e); + } finally { + latch.countDown(); } }); - latch.await(); + if (!latch.await(timeout / 1000, TimeUnit.SECONDS)) { + Assert.fail(String.format("Countdown latch(%d of %d) timed out after waiting %d seconds", + count + 1, latch.getCount(), timeout / 1000)); + } + Assert.assertEquals(n.get(), 0); + Assert.assertEquals(p.get(), count + 1); } } } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessorTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessorTest.java index c516fcb6a..cd21e82af 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessorTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessorTest.java @@ -49,8 +49,8 @@ public void testDeserializeArray() throws IOException { Assert.assertEquals(shortArray[1], Short.valueOf("2")); value = ClickHouseRowBinaryProcessor.getMappedFunctions().deserialize(null, config, - ClickHouseColumn.of("a", "Array(Nullable(Int8))"), - BinaryStreamUtilsTest.generateInput(2, 0, 1, 0, 2)); + ClickHouseColumn.of("a", "Array(Int8)"), + BinaryStreamUtilsTest.generateInput(2, 1, 2)); Assert.assertTrue(value instanceof ClickHouseByteArrayValue); Assert.assertEquals(value.asObject(), new byte[] { 1, 2 }); Object[] byteArray = value.asArray(); @@ -58,6 +58,16 @@ public void testDeserializeArray() throws IOException { Assert.assertEquals(byteArray[0], Byte.valueOf("1")); Assert.assertEquals(byteArray[1], Byte.valueOf("2")); + value = ClickHouseRowBinaryProcessor.getMappedFunctions().deserialize(null, config, + ClickHouseColumn.of("a", "Array(Nullable(Int8))"), + BinaryStreamUtilsTest.generateInput(2, 0, 1, 0, 2)); + Assert.assertTrue(value instanceof ClickHouseArrayValue); + Assert.assertEquals(value.asObject(), new Byte[] { (byte) 1, (byte) 2 }); + byteArray = value.asArray(); + Assert.assertEquals(byteArray.length, 2); + Assert.assertEquals(byteArray[0], Byte.valueOf("1")); + Assert.assertEquals(byteArray[1], Byte.valueOf("2")); + value = ClickHouseRowBinaryProcessor.getMappedFunctions().deserialize(null, config, ClickHouseColumn.of("a", "Array(Array(UInt8))"), BinaryStreamUtilsTest.generateInput(1, 2, 1, 2)); diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/stream/InputStreamImplTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/stream/InputStreamImplTest.java index fd3539a36..cf9238218 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/stream/InputStreamImplTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/stream/InputStreamImplTest.java @@ -14,6 +14,7 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collections; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; @@ -184,6 +185,14 @@ private Object[][] getEmptyInputStreams() { { ClickHouseInputStream.of(null, generateTempUrl()) }, { ClickHouseInputStream.of(generateTempUrl(), generateTempUrl()) }, { ClickHouseInputStream.of((URL) null, (URL) null) }, + // blocking and non-blocking + { new BlockingInputStream( + new LinkedBlockingQueue<>(Collections.singletonList(ClickHouseByteBuffer.EMPTY_BUFFER)), 0, + null) }, + { new NonBlockingInputStream( + new AdaptiveQueue<>(CapacityPolicy.linearDynamicCapacity(0, 0, 0), + ClickHouseByteBuffer.EMPTY_BYTES), + 0, null) } }; } @@ -252,6 +261,40 @@ private Object[][] getInputStreamsWithData() { generateTempUrl(0x68), generateTempUrl(0x69), generateTempUrl(0x70)) }, { ClickHouseInputStream.of(generateTempUrl(0x65, 0x66, 0x67), generateTempUrl(0x68, 0x69), generateTempUrl(0x70)) }, + // blocking and non-blocking + { new BlockingInputStream( + new LinkedBlockingQueue<>( + Arrays.asList(ByteBuffer.wrap(new byte[] { 0x65, 0x66, 0x67, 0x68, 0x69, 0x70 }), + ClickHouseByteBuffer.EMPTY_BUFFER)), + 0, null) }, + { new BlockingInputStream( + new LinkedBlockingQueue<>(Arrays.asList(ByteBuffer.wrap(new byte[] { 0x65 }), + ByteBuffer.wrap(new byte[] { 0x66 }), ByteBuffer.wrap(new byte[] { 0x67 }), + ByteBuffer.wrap(new byte[] { 0x68 }), ByteBuffer.wrap(new byte[] { 0x69 }), + ByteBuffer.wrap(new byte[] { 0x70 }), ClickHouseByteBuffer.EMPTY_BUFFER)), + 0, null) }, + { new BlockingInputStream( + new LinkedBlockingQueue<>(Arrays.asList(ByteBuffer.wrap(new byte[] { 0x65, 0x66 }), + ByteBuffer.wrap(new byte[] { 0x67 }), + ByteBuffer.wrap(new byte[] { 0x68, 0x69 }), + ByteBuffer.wrap(new byte[] { 0x70 }), ClickHouseByteBuffer.EMPTY_BUFFER)), + 0, null) }, + { new NonBlockingInputStream(new AdaptiveQueue(CapacityPolicy.linearDynamicCapacity(0, 0, 0), + Arrays.asList(new byte[] { 0x65, 0x66, 0x67, 0x68, 0x69, 0x70 }, + ClickHouseByteBuffer.EMPTY_BYTES)), + 0, null) }, + { new NonBlockingInputStream( + new AdaptiveQueue(CapacityPolicy.linearDynamicCapacity(0, 0, 0), + Arrays.asList(new byte[] { 0x65 }, new byte[] { 0x66 }, new byte[] { 0x67 }, + new byte[] { 0x68 }, new byte[] { 0x69 }, new byte[] { 0x70 }, + ClickHouseByteBuffer.EMPTY_BYTES)), + 0, null) }, + { new NonBlockingInputStream( + new AdaptiveQueue(CapacityPolicy.linearDynamicCapacity(0, 0, 0), + Arrays.asList(new byte[] { 0x65, 0x66 }, new byte[] { 0x67 }, new byte[] { 0x68, 0x69 }, + new byte[] { 0x70 }, ClickHouseByteBuffer.EMPTY_BYTES)), + 0, null) }, + }; } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/stream/NonBlockingPipedOutputStreamTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/stream/NonBlockingPipedOutputStreamTest.java new file mode 100644 index 000000000..572a7e6f7 --- /dev/null +++ b/clickhouse-client/src/test/java/com/clickhouse/client/stream/NonBlockingPipedOutputStreamTest.java @@ -0,0 +1,288 @@ +package com.clickhouse.client.stream; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import com.clickhouse.client.ClickHouseByteBuffer; + +import org.testng.Assert; +import org.testng.annotations.Test; + +public class NonBlockingPipedOutputStreamTest { + @Test(groups = { "unit" }) + public void testRead() throws Exception { + NonBlockingPipedOutputStream stream = new NonBlockingPipedOutputStream(4, 3, 1, CapacityPolicy.fixedCapacity(3), + null); + Assert.assertEquals(stream.queue.size(), 0); + try (InputStream in = stream.getInputStream()) { + in.read(); + Assert.fail("Read should be timed out"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("Read timed out") == 0); + } + + stream.queue.add(new byte[] { (byte) 3 }); + Assert.assertEquals(stream.queue.size(), 1); + try (InputStream in = stream.getInputStream()) { + Assert.assertEquals(in.read(), 3); + in.read(); + Assert.fail("Read should be timed out"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("Read timed out") == 0); + } + + stream.queue.add(new byte[] { (byte) 3, (byte) 4 }); + Assert.assertEquals(stream.queue.size(), 1); + try (InputStream in = stream.getInputStream()) { + Assert.assertEquals(in.read(), 3); + Assert.assertEquals(in.read(), 4); + in.read(); + Assert.fail("Read should be timed out"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("Read timed out") == 0); + } + + stream.queue.clear(); + stream.queue.add(ClickHouseByteBuffer.EMPTY_BYTES); + Assert.assertEquals(stream.queue.size(), 1); + try (InputStream in = stream.getInputStream()) { + Assert.assertEquals(in.read(), -1); + } + + stream.queue.add(new byte[] { (byte) 3, (byte) 4 }); + // stream.queue.put(buf); + stream.queue.add(ClickHouseByteBuffer.EMPTY_BYTES); + Assert.assertEquals(stream.queue.size(), 2); + try (InputStream in = stream.getInputStream()) { + Assert.assertEquals(in.read(), 3); + Assert.assertEquals(in.read(), 4); + Assert.assertEquals(in.read(), -1); + } + + try (InputStream in = stream.getInputStream()) { + in.close(); + in.read(); + Assert.fail("Read should fail"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("closed") > 0); + } + } + + @Test(groups = { "unit" }) + public void testReadBytes() throws Exception { + NonBlockingPipedOutputStream stream = new NonBlockingPipedOutputStream(4, 3, 1, CapacityPolicy.fixedCapacity(3), + null); + Assert.assertEquals(stream.queue.size(), 0); + byte[] bytes = new byte[3]; + try (InputStream in = stream.getInputStream()) { + in.read(bytes); + Assert.fail("Read should be timed out"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("Read timed out") == 0); + } + try (InputStream in = stream.getInputStream()) { + in.read(bytes, 0, 1); + Assert.fail("Read should be timed out"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("Read timed out") == 0); + } + + stream.queue.add(new byte[] { (byte) 3, (byte) 4 }); + Assert.assertEquals(stream.queue.size(), 1); + try (InputStream in = stream.getInputStream()) { + in.read(bytes); + Assert.fail("Read should be timed out"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("Read timed out") == 0); + } + stream.queue.add(new byte[] { (byte) 3, (byte) 4 }); + Assert.assertEquals(stream.queue.size(), 1); + try (InputStream in = stream.getInputStream()) { + Assert.assertEquals(in.read(bytes, 0, 2), 2); + in.read(bytes, 0, 1); + Assert.fail("Read should be timed out"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("Read timed out") == 0); + } + stream.queue.add(new byte[] { (byte) 3, (byte) 4 }); + Assert.assertEquals(stream.queue.size(), 1); + try (InputStream in = stream.getInputStream()) { + in.read(bytes, 0, 3); + Assert.fail("Read should be timed out"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("Read timed out") == 0); + } + + stream.queue.add(new byte[] { (byte) 3, (byte) 4 }); + stream.queue.add(ClickHouseByteBuffer.EMPTY_BYTES); + Assert.assertEquals(stream.queue.size(), 2); + try (InputStream in = stream.getInputStream()) { + Assert.assertEquals(in.read(bytes, 0, 3), 2); + Assert.assertEquals(in.read(bytes, 0, 1), -1); + Assert.assertEquals(in.read(bytes, 0, 2), -1); + } + + try (InputStream in = stream.getInputStream()) { + in.close(); + in.read(bytes, 0, 3); + Assert.fail("Read should fail"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("closed") > 0); + } + } + + @Test(groups = { "unit" }) + public void testWrite() throws Exception { + NonBlockingPipedOutputStream stream = new NonBlockingPipedOutputStream(2, 3, 2, CapacityPolicy.fixedCapacity(3), + null); + Assert.assertEquals(stream.queue.size(), 0); + try (OutputStream out = stream) { + out.write(5); + Assert.assertEquals(stream.queue.size(), 0); + out.write(6); + Assert.assertEquals(stream.queue.size(), 1); + out.write(7); + Assert.assertEquals(stream.queue.size(), 1); + out.flush(); + Assert.assertEquals(stream.queue.size(), 2); + Assert.assertEquals(stream.queue.poll(), new byte[] { (byte) 5, (byte) 6 }); + Assert.assertEquals(stream.queue.poll(), new byte[] { (byte) 7 }); + } + + stream = new NonBlockingPipedOutputStream(1, 1, 2, CapacityPolicy.fixedCapacity(1), null); + Assert.assertEquals(stream.queue.size(), 0); + try (OutputStream out = stream) { + out.write(5); + Assert.assertEquals(stream.queue.size(), 1); + out.write(6); + Assert.fail("Write should be timed out"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("Write timed out") == 0); + } + + Assert.assertEquals(stream.queue.size(), 2); + stream.queue.clear(); + Assert.assertEquals(stream.queue.size(), 0); + try (OutputStream out = stream) { + out.close(); + out.write(1); + Assert.fail("Write should fail"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("closed") > 0); + } + } + + @Test(groups = { "unit" }) + public void testWriteBytes() throws Exception { + NonBlockingPipedOutputStream stream = new NonBlockingPipedOutputStream(2, 3, 2, CapacityPolicy.fixedCapacity(3), + null); + Assert.assertEquals(stream.queue.size(), 0); + try (OutputStream out = stream) { + out.write(new byte[] { (byte) 9, (byte) 10 }); + Assert.assertEquals(stream.queue.size(), 1); + out.flush(); + Assert.assertEquals(stream.queue.size(), 1); + out.write(new byte[] { (byte) 11, (byte) 12 }, 1, 1); + Assert.assertEquals(stream.queue.size(), 1); + out.flush(); + Assert.assertEquals(stream.queue.size(), 2); + Assert.assertEquals(stream.queue.poll(), new byte[] { (byte) 9, (byte) 10 }); + Assert.assertEquals(stream.queue.poll(), new byte[] { (byte) 12 }); + } + + try (OutputStream out = stream) { + out.close(); + out.write(new byte[] { (byte) 13, (byte) 14 }, 0, 1); + Assert.fail("Write should fail"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().indexOf("closed") > 0); + } + } + + @Test(groups = { "unit" }) + public void testPipedStream() throws Exception { + final int timeout = 10000; + ExecutorService executor = Executors.newFixedThreadPool(2); + for (int bufferSize = -1; bufferSize < 10; bufferSize++) { + for (int queueLength = -1; queueLength < 10; queueLength++) { + final NonBlockingPipedOutputStream stream = new NonBlockingPipedOutputStream(bufferSize, queueLength, + timeout, CapacityPolicy.fixedCapacity(queueLength), null); + try (InputStream in = stream.getInputStream(); OutputStream out = stream) { + final int count = 10000; + final AtomicInteger p = new AtomicInteger(0); + final AtomicInteger n = new AtomicInteger(0); + final CountDownLatch latch = new CountDownLatch(count + 1); + executor.execute(() -> { + for (int i = 0; i < count; i++) { + byte[] bytes = new byte[] { (byte) (0xFF & i), (byte) (0xFF & i + 1), + (byte) (0xFF & i + 2) }; + try { + out.write(bytes); + } catch (IOException e) { + Thread.currentThread().interrupt(); + throw new UncheckedIOException(e); + // Assert.fail("Failed to write", e); + } + } + + try { + out.close(); + } catch (IOException e) { + Thread.currentThread().interrupt(); + throw new UncheckedIOException(e); + // Assert.fail("Failed to write", e); + } + }); + + executor.execute(() -> { + for (int i = 0; i < count; i++) { + byte[] bytes = new byte[] { (byte) (0xFF & i), (byte) (0xFF & i + 1), + (byte) (0xFF & i + 2) }; + byte[] b = new byte[bytes.length]; + try { + if (in.read(b) == b.length && Arrays.equals(b, bytes)) { + p.incrementAndGet(); + } else { + n.incrementAndGet(); + } + } catch (IOException e) { + Thread.currentThread().interrupt(); + throw new UncheckedIOException(e); + } finally { + latch.countDown(); + } + } + + try { + if (in.read() == -1) { + p.incrementAndGet(); + } else { + n.incrementAndGet(); + } + } catch (IOException e) { + Thread.currentThread().interrupt(); + throw new UncheckedIOException(e); + } finally { + latch.countDown(); + } + }); + + if (!latch.await(timeout / 1000, TimeUnit.SECONDS)) { + Assert.fail(String.format("Countdown latch(%d of %d) timed out after waiting %d seconds", + count + 1, latch.getCount(), timeout / 1000)); + } + Assert.assertEquals(n.get(), 0); + Assert.assertEquals(p.get(), count + 1); + } + } + } + } +} diff --git a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcClient.java b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcClient.java index 975e0d524..409688e4b 100644 --- a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcClient.java +++ b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcClient.java @@ -71,9 +71,7 @@ protected static QueryInfo convert(ClickHouseNode server, ClickHouseRequest r builder.setQueryId(optionalValue.get()); } - ClickHouseCompression outputCompression = config.isCompressServerResponse() - ? config.getCompressAlgorithmForServerResponse() - : ClickHouseCompression.NONE; + ClickHouseCompression outputCompression = config.getResponseCompressAlgorithm(); builder.setOutputCompressionType(outputCompression.encoding()); // builder.setNextQueryInfo(true); @@ -81,9 +79,7 @@ protected static QueryInfo convert(ClickHouseNode server, ClickHouseRequest r builder.putSettings(s.getKey(), String.valueOf(s.getValue())); } - ClickHouseCompression inputCompression = config.isDecompressClientRequet() - ? config.getDecompressAlgorithmForClientRequest() - : ClickHouseCompression.NONE; + ClickHouseCompression inputCompression = config.getRequestCompressAlgorithm(); Optional input = request.getInputStream(); if (input.isPresent()) { builder.setInputCompressionType(inputCompression.encoding()); diff --git a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java index b43184dde..277d1166e 100644 --- a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java +++ b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java @@ -22,9 +22,9 @@ public class ClickHouseGrpcResponse extends ClickHouseStreamResponse { private final Result result; static ClickHouseInputStream getInput(ClickHouseConfig config, InputStream input) { - if (config.isCompressServerResponse() - && config.getCompressAlgorithmForServerResponse() == ClickHouseCompression.LZ4) { - return ClickHouseInputStream.of( + final ClickHouseInputStream in; + if (config.getResponseCompressAlgorithm() == ClickHouseCompression.LZ4) { + in = ClickHouseInputStream.of( ClickHouseDeferredValue.of(() -> { try { return new FramedLZ4CompressorInputStream(input); @@ -34,8 +34,10 @@ static ClickHouseInputStream getInput(ClickHouseConfig config, InputStream input }), config.getReadBufferSize(), null); } else { - return ClickHouseClient.getResponseInputStream(config, input, null); + in = ClickHouseInputStream.of(input, config.getBufferSize(), config.getResponseCompressAlgorithm(), null); } + + return in; } protected ClickHouseGrpcResponse(ClickHouseConfig config, Map settings, diff --git a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseStreamObserver.java b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseStreamObserver.java index e4902a29f..6aa8ae85e 100644 --- a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseStreamObserver.java +++ b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseStreamObserver.java @@ -11,9 +11,9 @@ import com.clickhouse.client.ClickHouseException; import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseNode; +import com.clickhouse.client.ClickHousePipedOutputStream; import com.clickhouse.client.ClickHouseResponseSummary; import com.clickhouse.client.ClickHouseUtils; -import com.clickhouse.client.data.ClickHousePipedStream; import com.clickhouse.client.grpc.impl.Exception; import com.clickhouse.client.grpc.impl.LogEntry; import com.clickhouse.client.grpc.impl.Progress; @@ -30,7 +30,7 @@ public class ClickHouseStreamObserver implements StreamObserver { private final CountDownLatch startLatch; private final CountDownLatch finishLatch; - private final ClickHousePipedStream stream; + private final ClickHousePipedOutputStream stream; private final ClickHouseInputStream input; private final ClickHouseResponseSummary summary; @@ -43,8 +43,8 @@ protected ClickHouseStreamObserver(ClickHouseConfig config, ClickHouseNode serve this.startLatch = new CountDownLatch(1); this.finishLatch = new CountDownLatch(1); - this.stream = ClickHouseDataStreamFactory.getInstance().createPipedStream(config); - this.input = ClickHouseGrpcResponse.getInput(config, this.stream.getInput()); + this.stream = ClickHouseDataStreamFactory.getInstance().createPipedOutputStream(config, null); + this.input = ClickHouseGrpcResponse.getInput(config, this.stream.getInputStream()); this.summary = new ClickHouseResponseSummary(null, null); @@ -139,7 +139,7 @@ public void onNext(Result value) { if (updateStatus(value)) { try { // TODO close output stream if value.getOutput().isEmpty()? - value.getOutput().writeTo(stream); + stream.transferBytes(value.getOutput().toByteArray()); } catch (IOException e) { onError(e); } diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java index 7eb6d0ba5..09f99f1cd 100644 --- a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java @@ -58,10 +58,12 @@ static String buildQueryParams(ClickHouseRequest request) { appendQueryParameter(builder, cp.getKey(), cp.getValue()); } - if (config.isCompressServerResponse()) { + if (config.isResponseCompressed()) { + // request server to compress response appendQueryParameter(builder, "compress", "1"); } - if (config.isDecompressClientRequet()) { + if (config.isRequestCompressed()) { + // inform server that client's request is compressed appendQueryParameter(builder, "decompress", "1"); } @@ -191,12 +193,12 @@ protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest r } // Also, you can use the ‘default_format’ URL parameter map.put("X-ClickHouse-Format", config.getFormat().name()); - if (config.isCompressServerResponse()) { - map.put("Accept-Encoding", config.getCompressAlgorithmForServerResponse().encoding()); + if (config.isResponseCompressed()) { + map.put("Accept-Encoding", config.getResponseCompressAlgorithm().encoding()); } - if (config.isDecompressClientRequet() - && config.getDecompressAlgorithmForClientRequest() != ClickHouseCompression.LZ4) { - map.put("Content-Encoding", config.getDecompressAlgorithmForClientRequest().encoding()); + if (config.isRequestCompressed() + && config.getRequestCompressAlgorithm() != ClickHouseCompression.LZ4) { + map.put("Content-Encoding", config.getRequestCompressAlgorithm().encoding()); } this.defaultHeaders = Collections.unmodifiableMap(map); diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java index bf54d352f..c26c8543b 100644 --- a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java @@ -2,6 +2,7 @@ import com.clickhouse.client.ClickHouseChecker; import com.clickhouse.client.ClickHouseClient; +import com.clickhouse.client.ClickHouseConfig; import com.clickhouse.client.ClickHouseFormat; import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseNode; @@ -25,7 +26,6 @@ import java.net.URL; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.TimeZone; @@ -65,20 +65,25 @@ private ClickHouseHttpResponse buildResponse() throws IOException { String queryId = getResponseHeader("X-ClickHouse-Query-Id", ""); String summary = getResponseHeader("X-ClickHouse-Summary", "{}"); - ClickHouseFormat format = config.getFormat(); - TimeZone timeZone = config.getServerTimeZone(); + ClickHouseConfig c = config; + ClickHouseFormat format = c.getFormat(); + TimeZone timeZone = c.getServerTimeZone(); + boolean hasQueryResult = false; // queryId, format and timeZone are only available for queries if (!ClickHouseChecker.isNullOrEmpty(queryId)) { String value = getResponseHeader("X-ClickHouse-Format", ""); - format = !ClickHouseChecker.isNullOrEmpty(value) ? ClickHouseFormat.valueOf(value) - : format; + if (!ClickHouseChecker.isNullOrEmpty(value)) { + format = ClickHouseFormat.valueOf(value); + hasQueryResult = true; + } value = getResponseHeader("X-ClickHouse-Timezone", ""); timeZone = !ClickHouseChecker.isNullOrEmpty(value) ? TimeZone.getTimeZone(value) : timeZone; } return new ClickHouseHttpResponse(this, - ClickHouseClient.getResponseInputStream(config, conn.getInputStream(), null), + hasQueryResult ? ClickHouseClient.getAsyncResponseInputStream(c, conn.getInputStream(), null) + : ClickHouseClient.getResponseInputStream(c, conn.getInputStream(), null), displayName, queryId, summary, format, timeZone); } @@ -182,8 +187,16 @@ protected ClickHouseHttpResponse post(String sql, InputStream data, List> postRequest(HttpRequest req private ClickHouseHttpResponse postStream(HttpRequest.Builder reqBuilder, String boundary, String sql, InputStream data, List tables) throws IOException { - ClickHousePipedStream stream = new ClickHousePipedStream(config.getWriteBufferSize(), - config.getMaxQueuedBuffers(), config.getSocketTimeout()); - reqBuilder.POST(HttpRequest.BodyPublishers.ofInputStream(stream::getInput)); + ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance().createPipedOutputStream(config, + null); + reqBuilder.POST(HttpRequest.BodyPublishers.ofInputStream(stream::getInputStream)); // running in async is necessary to avoid deadlock of the piped stream CompletableFuture> f = postRequest(reqBuilder.build()); diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/InputBasedPreparedStatement.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/InputBasedPreparedStatement.java index 479b7d72a..8f5111cbf 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/InputBasedPreparedStatement.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/InputBasedPreparedStatement.java @@ -20,11 +20,12 @@ import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.ClickHouseConfig; +import com.clickhouse.client.ClickHouseDataStreamFactory; +import com.clickhouse.client.ClickHousePipedOutputStream; import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValue; import com.clickhouse.client.ClickHouseValues; -import com.clickhouse.client.data.ClickHousePipedStream; import com.clickhouse.client.logging.Logger; import com.clickhouse.client.logging.LoggerFactory; import com.clickhouse.jdbc.ClickHousePreparedStatement; @@ -42,7 +43,7 @@ public class InputBasedPreparedStatement extends AbstractPreparedStatement imple private final boolean[] flags; private int counter; - private ClickHousePipedStream stream; + private ClickHousePipedOutputStream stream; protected InputBasedPreparedStatement(ClickHouseConnectionImpl connection, ClickHouseRequest request, List columns, int resultSetType, int resultSetConcurrency, int resultSetHoldability) @@ -69,7 +70,7 @@ protected InputBasedPreparedStatement(ClickHouseConnectionImpl connection, Click counter = 0; // it's important to make sure the queue has unlimited length - stream = new ClickHousePipedStream(config.getWriteBufferSize(), 0, config.getSocketTimeout()); + stream = ClickHouseDataStreamFactory.getInstance().createPipedOutputStream(config, null); } protected void ensureParams() throws SQLException { @@ -105,7 +106,7 @@ protected long[] executeAny(boolean asBatch) throws SQLException { long rows = 0; try { stream.close(); - rows = executeInsert(getRequest().getStatements(false).get(0), stream.getInput()); + rows = executeInsert(getRequest().getStatements(false).get(0), stream.getInputStream()); if (asBatch && getResultSet() != null) { throw SqlExceptionUtils.queryInBatchError(results); } @@ -324,9 +325,14 @@ public void addBatch() throws SQLException { public void clearBatch() throws SQLException { ensureOpen(); + // just in case + try { + stream.close(); + } catch (Exception e) { + // ignore + } counter = 0; - ClickHouseConfig config = getConfig(); - stream = new ClickHousePipedStream(config.getWriteBufferSize(), 0, config.getSocketTimeout()); + stream = ClickHouseDataStreamFactory.getInstance().createPipedOutputStream(getConfig(), null); } @Override diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java index 75829cc01..e6e2d8e30 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java @@ -25,13 +25,14 @@ import java.util.concurrent.CompletionException; import com.clickhouse.client.ClickHouseConfig; +import com.clickhouse.client.ClickHouseDataStreamFactory; import com.clickhouse.client.ClickHouseFormat; import com.clickhouse.client.ClickHouseInputStream; +import com.clickhouse.client.ClickHousePipedOutputStream; import com.clickhouse.client.ClickHouseProtocol; import com.clickhouse.client.config.ClickHouseClientOption; import com.clickhouse.client.data.ClickHouseBitmap; import com.clickhouse.client.data.ClickHouseExternalTable; -import com.clickhouse.client.data.ClickHousePipedStream; import com.clickhouse.jdbc.internal.InputBasedPreparedStatement; import com.clickhouse.jdbc.internal.SqlBasedPreparedStatement; @@ -872,11 +873,11 @@ public void testLoadRawData() throws Exception { + "create table test_jdbc_load_raw_data(s String)engine=Memory"), "Should not have result set"); ClickHouseConfig config = stmt.getConfig(); CompletableFuture future; - try (ClickHousePipedStream stream = new ClickHousePipedStream(config.getWriteBufferSize(), - config.getMaxQueuedRequests(), config.getSocketTimeout())) { + try (ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance() + .createPipedOutputStream(config, null)) { ps.setObject(1, ClickHouseExternalTable.builder().name("raw_data") .columns("s String").format(ClickHouseFormat.RowBinary) - .content(stream.getInput()) + .content(stream.getInputStream()) .build()); future = CompletableFuture.supplyAsync(() -> { try { @@ -1081,7 +1082,7 @@ public void testInsertStringAsArray() throws Exception { Assert.assertTrue(rs.next()); Assert.assertEquals(rs.getInt(1), 1); Assert.assertEquals(rs.getObject(2), new short[] { 1, 2, 3 }); - Assert.assertEquals(rs.getObject(3), new long[] { 3, 0, 1 }); + Assert.assertEquals(rs.getObject(3), new Long[] { 3L, null, 1L }); Assert.assertFalse(rs.next()); } diff --git a/examples/grpc/src/main/java/com/clickhouse/examples/jdbc/Main.java b/examples/grpc/src/main/java/com/clickhouse/examples/jdbc/Main.java index 43fde6395..5f2b30f19 100644 --- a/examples/grpc/src/main/java/com/clickhouse/examples/jdbc/Main.java +++ b/examples/grpc/src/main/java/com/clickhouse/examples/jdbc/Main.java @@ -8,16 +8,17 @@ import com.clickhouse.client.ClickHouseClient; import com.clickhouse.client.ClickHouseConfig; import com.clickhouse.client.ClickHouseCredentials; +import com.clickhouse.client.ClickHouseDataStreamFactory; import com.clickhouse.client.ClickHouseException; import com.clickhouse.client.ClickHouseFormat; import com.clickhouse.client.ClickHouseNode; +import com.clickhouse.client.ClickHousePipedOutputStream; import com.clickhouse.client.ClickHouseProtocol; import com.clickhouse.client.ClickHouseRecord; import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseResponse; import com.clickhouse.client.ClickHouseResponseSummary; import com.clickhouse.client.data.BinaryStreamUtils; -import com.clickhouse.client.data.ClickHousePipedStream; public class Main { static void dropAndCreateTable(ClickHouseNode server, String table) throws ClickHouseException { @@ -42,10 +43,10 @@ static long insert(ClickHouseNode server, String table) throws ClickHouseExcepti ClickHouseConfig config = request.getConfig(); CompletableFuture future; // back-pressuring is not supported, you can adjust the first two arguments - try (ClickHousePipedStream stream = new ClickHousePipedStream(config.getWriteBufferSize(), - config.getMaxQueuedBuffers(), config.getSocketTimeout())) { + try (ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance() + .createPipedOutputStream(config, null)) { // in async mode, which is default, execution happens in a worker thread - future = request.data(stream.getInput()).execute(); + future = request.data(stream.getInputStream()).execute(); // writing happens in main thread for (int i = 0; i < 1000000; i++) {