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

Improved Connection Count server select strategy #15975

Merged
Merged
Show file tree
Hide file tree
Changes from 8 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
35 changes: 21 additions & 14 deletions server/src/main/java/org/apache/druid/client/DirectDruidClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -455,22 +455,29 @@ private void checkTotalBytesLimit(long bytes)
throw new QueryTimeoutException(StringUtils.nonStrictFormat("Query[%s] url[%s] timed out.", query.getId(), url));
}

future = httpClient.go(
new Request(
HttpMethod.POST,
new URL(url)
).setContent(objectMapper.writeValueAsBytes(Queries.withTimeout(query, timeLeft)))
.setHeader(
HttpHeaders.Names.CONTENT_TYPE,
isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON
),
responseHandler,
Duration.millis(timeLeft)
);
// increment is moved up so that if future initialization is queued by some other process,
// we can increment the count earlier so that we can route the request to a different server
openConnections.getAndIncrement();
try {
future = httpClient.go(
new Request(
HttpMethod.POST,
new URL(url)
).setContent(objectMapper.writeValueAsBytes(Queries.withTimeout(query, timeLeft)))
.setHeader(
HttpHeaders.Names.CONTENT_TYPE,
isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON
),
responseHandler,
Duration.millis(timeLeft)
);
}
catch (Exception e) {
openConnections.getAndDecrement();
throw e;
}

queryWatcher.registerQueryFuture(query, future);

openConnections.getAndIncrement();
Futures.addCallback(
future,
new FutureCallback<InputStream>()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@

package org.apache.druid.client;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
Expand All @@ -27,6 +29,7 @@
import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy;
import org.apache.druid.client.selector.QueryableDruidServer;
import org.apache.druid.client.selector.ServerSelector;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
Expand Down Expand Up @@ -97,6 +100,7 @@ public class DirectDruidClientTest
@Before
public void setup()
{
NullHandling.initializeForTests();
httpClient = EasyMock.createMock(HttpClient.class);
serverSelector = new ServerSelector(
dataSegment,
Expand Down Expand Up @@ -427,4 +431,47 @@ public void testQueryTimeoutFromFuture()
Assert.assertEquals(hostName, actualException.getHost());
EasyMock.verify(httpClient);
}

@Test
public void testRuntimeException() throws JsonProcessingException
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is too generic test name. I would suggest testConnectionCountAfterException

{
ObjectMapper mockObjectMapper = EasyMock.createMock(ObjectMapper.class);
EasyMock.expect(mockObjectMapper.writeValueAsBytes(EasyMock.anyObject()))
Copy link
Contributor

@abhishekagarwal87 abhishekagarwal87 Mar 4, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets intercept this call only when an object of type query is being serialized.

.andThrow(new JsonProcessingException("Error")
{
});

DirectDruidClient client2 = new DirectDruidClient(
new ReflectionQueryToolChestWarehouse(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
mockObjectMapper,
httpClient,
"http",
hostName,
new NoopServiceEmitter(),
queryCancellationExecutor
);

QueryableDruidServer queryableDruidServer2 = new QueryableDruidServer(
new DruidServer(
"test1",
"localhost",
null,
0,
ServerType.HISTORICAL,
DruidServer.DEFAULT_TIER,
0
),
client2
);

serverSelector.addServerAndUpdateSegment(queryableDruidServer2, serverSelector.getSegment());

TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();
query = query.withOverriddenContext(ImmutableMap.of(DirectDruidClient.QUERY_FAIL_TIME, Long.MAX_VALUE));

TimeBoundaryQuery finalQuery = query;
Assert.assertThrows(RuntimeException.class, () -> client2.run(QueryPlus.wrap(finalQuery)));
Assert.assertEquals(0, client2.getNumOpenConnections());
}
}
Loading