Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[client-v2] Fix compression issue when concurrent requests #1791

Merged
merged 1 commit into from
Aug 26, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ public int read(byte[] b, int off, int len) throws IOException {
static final byte MAGIC = (byte) 0x82;
static final int HEADER_LENGTH = 25;

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

/**
* Method ensures to read all bytes from the input stream.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@

public class QueryServerContentCompressionTests extends QueryTests {
static {
System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "DEBUG");
// System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "DEBUG");
}
QueryServerContentCompressionTests() {
super(true, false);
Expand Down
62 changes: 48 additions & 14 deletions client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java
Original file line number Diff line number Diff line change
Expand Up @@ -66,13 +66,17 @@
import java.util.Properties;
import java.util.Random;
import java.util.UUID;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.BaseStream;
import java.util.stream.IntStream;

public class QueryTests extends BaseIntegrationTest {

Expand Down Expand Up @@ -1264,32 +1268,62 @@ public void testClientUseOwnTimeZone() {

@Test
public void testAsyncQuery() {
try (Client client = newClient().useAsyncRequests(true).build();
QueryResponse response =
try (Client client = newClient().useAsyncRequests(true).build()){
simpleRequest(client);
} catch (Exception e) {
Assert.fail("Failed to get server time zone from header", e);
}
}

protected void simpleRequest(Client client) throws Exception {
try (QueryResponse response =
client.query("SELECT number FROM system.numbers LIMIT 1000_000").get(1, TimeUnit.SECONDS)) {
ClickHouseBinaryFormatReader reader =
new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), response.getSettings());
ClickHouseBinaryFormatReader reader =
new RowBinaryWithNamesAndTypesFormatReader(response.getInputStream(), response.getSettings());

int count = 0;
while (reader.hasNext()) {
reader.next();
count++;
}
int count = 0;
while (reader.hasNext()) {
reader.next();
count++;
}

Assert.assertEquals(count, 1000_000);
} catch (Exception e) {
Assert.fail("Failed to get server time zone from header", e);
Assert.assertEquals(count, 1000_000);
}
}

private Client.Builder newClient() {
@Test
public void testConcurrentQueries() throws Exception{
final Client client = newClient().build();
final int concurrency = 10;
CountDownLatch latch = new CountDownLatch(concurrency);
Runnable task = () -> {
try {
simpleRequest(client);
} catch (Exception e) {
e.printStackTrace();
Assert.fail("Failed", e);
} finally {
latch.countDown();
}
};

ExecutorService executor = Executors.newFixedThreadPool(concurrency);
IntStream.range(0,concurrency).forEach(i -> executor.submit(task));
executor.shutdown();
executor.awaitTermination(10, TimeUnit.SECONDS);
latch.await();
Assert.assertEquals(latch.getCount(), 0);
}

protected Client.Builder newClient() {
ClickHouseNode node = getServer(ClickHouseProtocol.HTTP);
return new Client.Builder()
.addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), false)
.setUsername("default")
.setPassword("")
.compressClientRequest(false)
.compressServerResponse(false)
.compressServerResponse(true)
.useHttpCompression(useHttpCompression)
.useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true"));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ public Client chDirectClient(@Value("${db.url}") String dbUrl, @Value("${db.user
.addEndpoint(dbUrl)
.setUsername(dbUser)
.setPassword(dbPassword)
.compressServerResponse(false)
.useNewImplementation(true) // using new transport layer implementation

// sets the maximum number of connections to the server at a time
Expand Down
Loading