-
Notifications
You must be signed in to change notification settings - Fork 3.8k
improve query timeout handling and limit max scatter-gather bytes #4229
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -42,6 +42,7 @@ | |
| import com.metamx.http.client.response.HttpResponseHandler; | ||
| import com.metamx.http.client.response.StatusResponseHandler; | ||
| import com.metamx.http.client.response.StatusResponseHolder; | ||
| import io.druid.common.utils.StringUtils; | ||
| import io.druid.java.util.common.IAE; | ||
| import io.druid.java.util.common.Pair; | ||
| import io.druid.java.util.common.RE; | ||
|
|
@@ -60,6 +61,7 @@ | |
| import io.druid.query.QueryToolChest; | ||
| import io.druid.query.QueryToolChestWarehouse; | ||
| import io.druid.query.QueryWatcher; | ||
| import io.druid.query.ResourceLimitExceededException; | ||
| import io.druid.query.Result; | ||
| import io.druid.query.aggregation.MetricManipulatorFns; | ||
| import org.jboss.netty.buffer.ChannelBuffer; | ||
|
|
@@ -68,6 +70,7 @@ | |
| import org.jboss.netty.handler.codec.http.HttpHeaders; | ||
| import org.jboss.netty.handler.codec.http.HttpMethod; | ||
| import org.jboss.netty.handler.codec.http.HttpResponse; | ||
| import org.joda.time.Duration; | ||
|
|
||
| import javax.ws.rs.core.MediaType; | ||
| import java.io.Closeable; | ||
|
|
@@ -84,14 +87,19 @@ | |
| import java.util.concurrent.Future; | ||
| import java.util.concurrent.LinkedBlockingQueue; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.TimeoutException; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
| import java.util.concurrent.atomic.AtomicReference; | ||
|
|
||
| /** | ||
| */ | ||
| public class DirectDruidClient<T> implements QueryRunner<T> | ||
| { | ||
| public static final String QUERY_FAIL_TIME = "queryFailTime"; | ||
| public static final String QUERY_TOTAL_BYTES_GATHERED = "queryTotalBytesGathered"; | ||
|
|
||
| private static final Logger log = new Logger(DirectDruidClient.class); | ||
|
|
||
| private static final Map<Class<? extends Query>, Pair<JavaType, JavaType>> typesMap = Maps.newConcurrentMap(); | ||
|
|
@@ -168,16 +176,24 @@ public Sequence<T> run(final QueryPlus<T> queryPlus, final Map<String, Object> c | |
| final QueryMetrics<? super Query<T>> queryMetrics = toolChest.makeMetrics(query); | ||
| queryMetrics.server(host); | ||
|
|
||
| long timeoutAt = ((Long) context.get(QUERY_FAIL_TIME)).longValue(); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It introduces NPE, if the query is made bypassing QueryResource, i. e. from inside QueryEngine of another type of query, via QuerySegmentWalker.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Druid SQL ran into that too and was fixed via #4305. I think this is fine since, imo, making queries without going through the resources is something you do 'at your own risk' and isn't an officially supported mode of operation.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. QueryResource is neither official API in #4433. Making a query as HTTP request to the same JVM runtime is ridiculous.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What I'm saying is that I would consider the whole concept of queries making other queries as not officially supported. If you want to do that in an extension then go for it but there is no official API for it.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Rationale for me is:
|
||
| long maxScatterGatherBytes = QueryContexts.getMaxScatterGatherBytes(query); | ||
| AtomicLong totalBytesGathered = (AtomicLong) context.get(QUERY_TOTAL_BYTES_GATHERED); | ||
|
|
||
| final HttpResponseHandler<InputStream, InputStream> responseHandler = new HttpResponseHandler<InputStream, InputStream>() | ||
| { | ||
| private long responseStartTimeNs; | ||
| private final AtomicLong byteCount = new AtomicLong(0); | ||
| private final BlockingQueue<InputStream> queue = new LinkedBlockingQueue<>(); | ||
| private final AtomicBoolean done = new AtomicBoolean(false); | ||
| private final AtomicReference<String> fail = new AtomicReference<>(); | ||
|
|
||
| @Override | ||
| public ClientResponse<InputStream> handleResponse(HttpResponse response) | ||
| { | ||
| checkQueryTimeout(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it would be nicer if both of these were to return booleans and the call sites were to just skip things or whatever if the limits are exceeded.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. they contain specific error message which would need to be repeated at all call sites. also, they are used in a relatively small scope and look ok. |
||
| checkTotalBytesLimit(response.getContent().readableBytes()); | ||
|
|
||
| log.debug("Initial response from url[%s] for queryId[%s]", url, query.getId()); | ||
| responseStartTimeNs = System.nanoTime(); | ||
| queryMetrics.reportNodeTimeToFirstByte(responseStartTimeNs - requestStartTimeNs).emit(emitter); | ||
|
|
@@ -222,6 +238,11 @@ public int read() throws IOException | |
| @Override | ||
| public boolean hasMoreElements() | ||
| { | ||
| if (fail.get() != null) { | ||
| throw new RE(fail.get()); | ||
| } | ||
| checkQueryTimeout(); | ||
|
|
||
| // Done is always true until the last stream has be put in the queue. | ||
| // Then the stream should be spouting good InputStreams. | ||
| synchronized (done) { | ||
|
|
@@ -232,8 +253,17 @@ public boolean hasMoreElements() | |
| @Override | ||
| public InputStream nextElement() | ||
| { | ||
| if (fail.get() != null) { | ||
| throw new RE(fail.get()); | ||
| } | ||
|
|
||
| try { | ||
| return queue.take(); | ||
| InputStream is = queue.poll(checkQueryTimeout(), TimeUnit.MILLISECONDS); | ||
| if (is != null) { | ||
| return is; | ||
| } else { | ||
| throw new RE("Query[%s] url[%s] timed out.", query.getId(), url); | ||
| } | ||
| } | ||
| catch (InterruptedException e) { | ||
| Thread.currentThread().interrupt(); | ||
|
|
@@ -250,8 +280,13 @@ public ClientResponse<InputStream> handleChunk( | |
| ClientResponse<InputStream> clientResponse, HttpChunk chunk | ||
| ) | ||
| { | ||
| checkQueryTimeout(); | ||
|
|
||
| final ChannelBuffer channelBuffer = chunk.getContent(); | ||
| final int bytes = channelBuffer.readableBytes(); | ||
|
|
||
| checkTotalBytesLimit(bytes); | ||
|
|
||
| if (bytes > 0) { | ||
| try { | ||
| queue.put(new ChannelBufferInputStream(channelBuffer)); | ||
|
|
@@ -308,34 +343,78 @@ public ClientResponse<InputStream> done(ClientResponse<InputStream> clientRespon | |
| @Override | ||
| public void exceptionCaught(final ClientResponse<InputStream> clientResponse, final Throwable e) | ||
| { | ||
| // Don't wait for lock in case the lock had something to do with the error | ||
| synchronized (done) { | ||
| done.set(true); | ||
| // Make a best effort to put a zero length buffer into the queue in case something is waiting on the take() | ||
| // If nothing is waiting on take(), this will be closed out anyways. | ||
| queue.offer( | ||
| new InputStream() | ||
| { | ||
| @Override | ||
| public int read() throws IOException | ||
| { | ||
| throw new IOException(e); | ||
| } | ||
| } | ||
| String msg = StringUtils.safeFormat( | ||
| "Query[%s] url[%s] failed with exception msg [%s]", | ||
| query.getId(), | ||
| url, | ||
| e.getMessage() | ||
| ); | ||
| setupResponseReadFailure(msg, e); | ||
| } | ||
|
|
||
| private void setupResponseReadFailure(String msg, Throwable th) | ||
| { | ||
| fail.set(msg); | ||
| queue.clear(); | ||
| queue.offer(new InputStream() | ||
| { | ||
| @Override | ||
| public int read() throws IOException | ||
| { | ||
| if (th != null) { | ||
| throw new IOException(msg, th); | ||
| } else { | ||
| throw new IOException(msg); | ||
| } | ||
| } | ||
| }); | ||
|
|
||
| } | ||
|
|
||
| // Returns remaining timeout or throws exception if timeout already elapsed. | ||
| private long checkQueryTimeout() | ||
| { | ||
| long timeLeft = timeoutAt - System.currentTimeMillis(); | ||
| if (timeLeft >= 0) { | ||
| String msg = StringUtils.safeFormat("Query[%s] url[%s] timed out.", query.getId(), url); | ||
| setupResponseReadFailure(msg, null); | ||
| throw new RE(msg); | ||
| } else { | ||
| return timeLeft; | ||
| } | ||
| } | ||
|
|
||
| private void checkTotalBytesLimit(long bytes) | ||
| { | ||
| if (maxScatterGatherBytes < Long.MAX_VALUE && totalBytesGathered.addAndGet(bytes) > maxScatterGatherBytes) { | ||
| String msg = StringUtils.safeFormat( | ||
| "Query[%s] url[%s] max scatter-gather bytes limit reached.", | ||
| query.getId(), | ||
| url | ||
| ); | ||
| setupResponseReadFailure(msg, null); | ||
| throw new RE(msg); | ||
| } | ||
| } | ||
| }; | ||
|
|
||
| long timeLeft = timeoutAt - System.currentTimeMillis(); | ||
|
|
||
| if (timeLeft <= 0) { | ||
| throw new RE("Query[%s] url[%s] timed out.", query.getId(), url); | ||
| } | ||
|
|
||
| future = httpClient.go( | ||
| new Request( | ||
| HttpMethod.POST, | ||
| new URL(url) | ||
| ).setContent(objectMapper.writeValueAsBytes(query)) | ||
| ).setContent(objectMapper.writeValueAsBytes(QueryContexts.withTimeout(query, timeLeft))) | ||
| .setHeader( | ||
| HttpHeaders.Names.CONTENT_TYPE, | ||
| isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON | ||
| ), | ||
| responseHandler | ||
| responseHandler, | ||
| Duration.millis(timeLeft) | ||
| ); | ||
|
|
||
| queryWatcher.registerQuery(query, future); | ||
|
|
@@ -368,8 +447,10 @@ public void onFailure(Throwable t) | |
| ? SmileMediaTypes.APPLICATION_JACKSON_SMILE | ||
| : MediaType.APPLICATION_JSON | ||
| ), | ||
| new StatusResponseHandler(Charsets.UTF_8) | ||
| ).get(); | ||
| new StatusResponseHandler(Charsets.UTF_8), | ||
| Duration.standardSeconds(1) | ||
| ).get(1, TimeUnit.SECONDS); | ||
|
|
||
| if (res.getStatus().getCode() >= 500) { | ||
| throw new RE( | ||
| "Error cancelling query[%s]: queriable node returned status[%d] [%s].", | ||
|
|
@@ -378,7 +459,7 @@ public void onFailure(Throwable t) | |
| ); | ||
| } | ||
| } | ||
| catch (IOException | ExecutionException | InterruptedException e) { | ||
| catch (IOException | ExecutionException | InterruptedException | TimeoutException e) { | ||
| Throwables.propagate(e); | ||
| } | ||
| } | ||
|
|
@@ -396,7 +477,7 @@ public void onFailure(Throwable t) | |
| @Override | ||
| public JsonParserIterator<T> make() | ||
| { | ||
| return new JsonParserIterator<T>(typeRef, future, url); | ||
| return new JsonParserIterator<T>(typeRef, future, url, query); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -428,13 +509,15 @@ private class JsonParserIterator<T> implements Iterator<T>, Closeable | |
| private ObjectCodec objectCodec; | ||
| private final JavaType typeRef; | ||
| private final Future<InputStream> future; | ||
| private final Query<T> query; | ||
| private final String url; | ||
|
|
||
| public JsonParserIterator(JavaType typeRef, Future<InputStream> future, String url) | ||
| public JsonParserIterator(JavaType typeRef, Future<InputStream> future, String url, Query<T> query) | ||
| { | ||
| this.typeRef = typeRef; | ||
| this.future = future; | ||
| this.url = url; | ||
| this.query = query; | ||
| jp = null; | ||
| } | ||
|
|
||
|
|
@@ -458,6 +541,7 @@ public boolean hasNext() | |
| public T next() | ||
| { | ||
| init(); | ||
|
|
||
| try { | ||
| final T retVal = objectCodec.readValue(jp, typeRef); | ||
| jp.nextToken(); | ||
|
|
@@ -478,7 +562,19 @@ private void init() | |
| { | ||
| if (jp == null) { | ||
| try { | ||
| jp = objectMapper.getFactory().createParser(future.get()); | ||
| InputStream is = future.get(); | ||
| if (is == null) { | ||
| throw new QueryInterruptedException( | ||
| new ResourceLimitExceededException( | ||
| "query[%s] url[%s] timed out or max bytes limit reached.", | ||
| query.getId(), | ||
| url | ||
| ), | ||
| host | ||
| ); | ||
| } else { | ||
| jp = objectMapper.getFactory().createParser(is); | ||
| } | ||
| final JsonToken nextToken = jp.nextToken(); | ||
| if (nextToken == JsonToken.START_OBJECT) { | ||
| QueryInterruptedException cause = jp.getCodec().readValue(jp, QueryInterruptedException.class); | ||
|
|
@@ -491,7 +587,13 @@ private void init() | |
| } | ||
| } | ||
| catch (IOException | InterruptedException | ExecutionException e) { | ||
| throw new RE(e, "Failure getting results from[%s] because of [%s]", url, e.getMessage()); | ||
| throw new RE( | ||
| e, | ||
| "Failure getting results for query[%s] url[%s] because of [%s]", | ||
| query.getId(), | ||
| url, | ||
| e.getMessage() | ||
| ); | ||
| } | ||
| catch (CancellationException e) { | ||
| throw new QueryInterruptedException(e, host); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder why this property is prefixed by
druid.serverinstead ofdruid.broker. Is it planned to be applied to other node types?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
i don't think it makes sense for any other node type. And now that you mention it, I think probably made more sense to call it
druid.broker.http.maxScatterGatherBytes. we can possibly change it in future or remove this config if backpressure story improves.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. It sounds good.