Merge pull request #647 from metamx/better-async

completely async request proxying + jetty update
This commit is contained in:
fjy 2014-07-25 17:36:15 -06:00
commit 7aef463457
9 changed files with 232 additions and 196 deletions

View File

@ -324,17 +324,17 @@
<dependency> <dependency>
<groupId>org.eclipse.jetty</groupId> <groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId> <artifactId>jetty-server</artifactId>
<version>9.2.1.v20140609</version> <version>9.2.2.v20140723</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.eclipse.jetty</groupId> <groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-servlet</artifactId> <artifactId>jetty-servlet</artifactId>
<version>9.2.1.v20140609</version> <version>9.2.2.v20140723</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.eclipse.jetty</groupId> <groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-servlets</artifactId> <artifactId>jetty-servlets</artifactId>
<version>9.2.1.v20140609</version> <version>9.2.2.v20140723</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>joda-time</groupId> <groupId>joda-time</groupId>

View File

@ -38,6 +38,7 @@ import org.jboss.netty.handler.codec.http.HttpHeaders;
import javax.inject.Inject; import javax.inject.Inject;
import java.io.IOException; import java.io.IOException;
import java.net.URI;
import java.net.URL; import java.net.URL;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -72,7 +73,7 @@ public class RoutingDruidClient<IntermediateType, FinalType>
} }
public ListenableFuture<FinalType> postQuery( public ListenableFuture<FinalType> postQuery(
String url, URI uri,
Query query, Query query,
HttpResponseHandler<IntermediateType, FinalType> responseHandler HttpResponseHandler<IntermediateType, FinalType> responseHandler
) )
@ -80,9 +81,9 @@ public class RoutingDruidClient<IntermediateType, FinalType>
final ListenableFuture<FinalType> future; final ListenableFuture<FinalType> future;
try { try {
log.debug("Querying url[%s]", url); log.debug("Querying url[%s]", uri);
future = httpClient future = httpClient
.post(new URL(url)) .post(uri.toURL())
.setContent(objectMapper.writeValueAsBytes(query)) .setContent(objectMapper.writeValueAsBytes(query))
.setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? QueryResource.APPLICATION_SMILE : QueryResource.APPLICATION_JSON) .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? QueryResource.APPLICATION_SMILE : QueryResource.APPLICATION_JSON)
.go(responseHandler); .go(responseHandler);
@ -115,13 +116,13 @@ public class RoutingDruidClient<IntermediateType, FinalType>
} }
public ListenableFuture<FinalType> get( public ListenableFuture<FinalType> get(
String url, URI uri,
HttpResponseHandler<IntermediateType, FinalType> responseHandler HttpResponseHandler<IntermediateType, FinalType> responseHandler
) )
{ {
try { try {
return httpClient return httpClient
.get(new URL(url)) .get(uri.toURL())
.go(responseHandler); .go(responseHandler);
} }
catch (IOException e) { catch (IOException e) {
@ -130,13 +131,13 @@ public class RoutingDruidClient<IntermediateType, FinalType>
} }
public ListenableFuture<FinalType> delete( public ListenableFuture<FinalType> delete(
String url, URI uri,
HttpResponseHandler<IntermediateType, FinalType> responseHandler HttpResponseHandler<IntermediateType, FinalType> responseHandler
) )
{ {
try { try {
return httpClient return httpClient
.delete(new URL(url)) .delete(uri.toURL())
.go(responseHandler); .go(responseHandler);
} }
catch (IOException e) { catch (IOException e) {

View File

@ -27,6 +27,9 @@ import com.google.common.base.Predicate;
import com.google.common.base.Predicates; import com.google.common.base.Predicates;
import com.google.common.collect.FluentIterable; import com.google.common.collect.FluentIterable;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.emitter.service.ServiceMetricEvent;
@ -37,7 +40,6 @@ import io.druid.guice.annotations.Json;
import io.druid.guice.annotations.Smile; import io.druid.guice.annotations.Smile;
import io.druid.query.DataSourceUtil; import io.druid.query.DataSourceUtil;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.log.RequestLogger; import io.druid.server.log.RequestLogger;
import io.druid.server.router.QueryHostFinder; import io.druid.server.router.QueryHostFinder;
import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffer;
@ -49,12 +51,13 @@ import org.joda.time.DateTime;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import javax.servlet.AsyncContext; import javax.servlet.AsyncContext;
import javax.servlet.ServletException; import javax.servlet.ServletException;
import javax.servlet.ServletOutputStream;
import javax.servlet.annotation.WebServlet; import javax.servlet.annotation.WebServlet;
import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse; import javax.servlet.http.HttpServletResponse;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.net.URI;
import java.util.Map; import java.util.Map;
import java.util.UUID; import java.util.UUID;
@ -67,7 +70,6 @@ public class AsyncQueryForwardingServlet extends HttpServlet
private static final EmittingLogger log = new EmittingLogger(AsyncQueryForwardingServlet.class); private static final EmittingLogger log = new EmittingLogger(AsyncQueryForwardingServlet.class);
private static final Joiner COMMA_JOIN = Joiner.on(","); private static final Joiner COMMA_JOIN = Joiner.on(",");
private final ServerConfig config;
private final ObjectMapper jsonMapper; private final ObjectMapper jsonMapper;
private final ObjectMapper smileMapper; private final ObjectMapper smileMapper;
private final QueryHostFinder hostFinder; private final QueryHostFinder hostFinder;
@ -76,7 +78,6 @@ public class AsyncQueryForwardingServlet extends HttpServlet
private final RequestLogger requestLogger; private final RequestLogger requestLogger;
public AsyncQueryForwardingServlet( public AsyncQueryForwardingServlet(
ServerConfig config,
@Json ObjectMapper jsonMapper, @Json ObjectMapper jsonMapper,
@Smile ObjectMapper smileMapper, @Smile ObjectMapper smileMapper,
QueryHostFinder hostFinder, QueryHostFinder hostFinder,
@ -85,7 +86,6 @@ public class AsyncQueryForwardingServlet extends HttpServlet
RequestLogger requestLogger RequestLogger requestLogger
) )
{ {
this.config = config;
this.jsonMapper = jsonMapper; this.jsonMapper = jsonMapper;
this.smileMapper = smileMapper; this.smileMapper = smileMapper;
this.hostFinder = hostFinder; this.hostFinder = hostFinder;
@ -98,92 +98,80 @@ public class AsyncQueryForwardingServlet extends HttpServlet
protected void doGet(HttpServletRequest req, HttpServletResponse res) protected void doGet(HttpServletRequest req, HttpServletResponse res)
throws ServletException, IOException throws ServletException, IOException
{ {
final AsyncContext asyncContext = req.startAsync(req, res); final AsyncContext asyncContext = req.startAsync();
// default async timeout to be same as maxIdleTime for now asyncContext.setTimeout(0);
asyncContext.setTimeout(config.getMaxIdleTime().toStandardDuration().getMillis());
asyncContext.start(
new Runnable()
{
@Override
public void run()
{
try {
final HttpResponseHandler<OutputStream, OutputStream> responseHandler = new PassthroughHttpResponseHandler(asyncContext, jsonMapper);
final String host = hostFinder.getDefaultHost(); final HttpResponseHandler<ServletOutputStream, ServletOutputStream> responseHandler =
routingDruidClient.get(makeUrl(host, (HttpServletRequest) asyncContext.getRequest()), responseHandler); new PassthroughHttpResponseHandler(res);
}
catch (Exception e) { final URI uri = rewriteURI(hostFinder.getDefaultHost(), req);
handleException(jsonMapper, asyncContext, e); asyncComplete(
} res,
} asyncContext,
} jsonMapper,
routingDruidClient.get(uri, responseHandler)
); );
} }
@Override @Override
protected void doDelete(HttpServletRequest req, HttpServletResponse res) throws ServletException, IOException protected void doDelete(HttpServletRequest req, HttpServletResponse res) throws ServletException, IOException
{ {
final AsyncContext asyncContext = req.startAsync(req, res); final AsyncContext asyncContext = req.startAsync();
asyncContext.start( asyncContext.setTimeout(0);
new Runnable()
{ final HttpResponseHandler<ServletOutputStream, ServletOutputStream> responseHandler =
@Override new PassthroughHttpResponseHandler(res);
public void run()
{
try {
final HttpResponseHandler<OutputStream, OutputStream> responseHandler = new PassthroughHttpResponseHandler(asyncContext, jsonMapper);
final String host = hostFinder.getDefaultHost(); final String host = hostFinder.getDefaultHost();
routingDruidClient.delete(makeUrl(host, (HttpServletRequest) asyncContext.getRequest()), responseHandler);
} asyncComplete(
catch (Exception e) { res,
handleException(jsonMapper, asyncContext, e); asyncContext,
} jsonMapper,
} routingDruidClient.delete(rewriteURI(host, (HttpServletRequest) asyncContext.getRequest()), responseHandler)
}
); );
} }
@Override @Override
protected void doPost(HttpServletRequest req, HttpServletResponse res) throws ServletException, IOException protected void doPost(final HttpServletRequest req, final HttpServletResponse res) throws ServletException, IOException
{ {
final long start = System.currentTimeMillis(); final long start = System.currentTimeMillis();
final AsyncContext asyncContext = req.startAsync(req, res); final boolean isSmile = QueryResource.APPLICATION_SMILE.equals(req.getContentType());
asyncContext.start(
new Runnable()
{
@Override
public void run()
{
final HttpServletRequest request = (HttpServletRequest) asyncContext.getRequest();
final boolean isSmile = QueryResource.APPLICATION_SMILE.equals(request.getContentType());
final ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; final ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper;
Query inputQuery = null;
try { try {
inputQuery = objectMapper.readValue(request.getInputStream(), Query.class); final Query inputQuery = objectMapper.readValue(req.getInputStream(), Query.class);
if (inputQuery.getId() == null) {
inputQuery = inputQuery.withId(UUID.randomUUID().toString());
}
final Query query = inputQuery;
if (log.isDebugEnabled()) { if (log.isDebugEnabled()) {
log.debug("Got query [%s]", inputQuery); log.debug("Got query [%s]", inputQuery);
} }
final HttpResponseHandler<OutputStream, OutputStream> responseHandler = new PassthroughHttpResponseHandler( final Query query;
asyncContext, if (inputQuery.getId() == null) {
objectMapper query = inputQuery.withId(UUID.randomUUID().toString());
) } else {
query = inputQuery;
}
URI rewrittenURI = rewriteURI(hostFinder.getHost(query), req);
final AsyncContext asyncContext = req.startAsync();
// let proxy http client timeout
asyncContext.setTimeout(0);
ListenableFuture future = routingDruidClient.postQuery(
rewrittenURI,
query,
new PassthroughHttpResponseHandler(res)
);
Futures.addCallback(
future,
new FutureCallback()
{ {
@Override @Override
public ClientResponse<OutputStream> done(ClientResponse<OutputStream> clientResponse) public void onSuccess(@Nullable Object o)
{ {
final long requestTime = System.currentTimeMillis() - start; final long requestTime = System.currentTimeMillis() - start;
log.debug("Request time: %d", requestTime);
emitter.emit( emitter.emit(
new ServiceMetricEvent.Builder() new ServiceMetricEvent.Builder()
.setUser2(DataSourceUtil.getMetricName(query.getDataSource())) .setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
@ -191,7 +179,7 @@ public class AsyncQueryForwardingServlet extends HttpServlet
.setUser4(query.getType()) .setUser4(query.getType())
.setUser5(COMMA_JOIN.join(query.getIntervals())) .setUser5(COMMA_JOIN.join(query.getIntervals()))
.setUser6(String.valueOf(query.hasFilters())) .setUser6(String.valueOf(query.hasFilters()))
.setUser7(request.getRemoteAddr()) .setUser7(req.getRemoteAddr())
.setUser8(query.getId()) .setUser8(query.getId())
.setUser9(query.getDuration().toPeriod().toStandardMinutes().toString()) .setUser9(query.getDuration().toPeriod().toStandardMinutes().toString())
.build("request/time", requestTime) .build("request/time", requestTime)
@ -201,7 +189,7 @@ public class AsyncQueryForwardingServlet extends HttpServlet
requestLogger.log( requestLogger.log(
new RequestLogLine( new RequestLogLine(
new DateTime(), new DateTime(),
request.getRemoteAddr(), req.getRemoteAddr(),
query, query,
new QueryStats( new QueryStats(
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
@ -215,61 +203,115 @@ public class AsyncQueryForwardingServlet extends HttpServlet
); );
} }
catch (Exception e) { catch (Exception e) {
throw Throwables.propagate(e); log.error(e, "Unable to log query [%s]!", query);
}
} }
return super.done(clientResponse); @Override
} public void onFailure(Throwable throwable)
}; {
routingDruidClient.postQuery(
makeUrl(hostFinder.getHost(inputQuery), request),
inputQuery,
responseHandler
);
}
catch (Exception e) {
handleException(objectMapper, asyncContext, e);
try { try {
final String errorMessage = throwable.getMessage();
requestLogger.log( requestLogger.log(
new RequestLogLine( new RequestLogLine(
new DateTime(), new DateTime(),
request.getRemoteAddr(), req.getRemoteAddr(),
inputQuery, query,
new QueryStats(ImmutableMap.<String, Object>of("success", false, "exception", e.toString())) new QueryStats(
ImmutableMap.<String, Object>of(
"success",
false,
"exception",
errorMessage == null ? "no message" : errorMessage)
)
) )
); );
} }
catch (Exception logError) { catch (IOException logError) {
log.error(logError, "Unable to log query [%s]!", inputQuery); log.error(logError, "Unable to log query [%s]!", query);
} }
log.makeAlert(e, "Exception handling request") log.makeAlert(throwable, "Exception handling request [%s]", query.getId())
.addData("query", inputQuery) .addData("query", query)
.addData("peer", request.getRemoteAddr()) .addData("peer", req.getRemoteAddr())
.emit(); .emit();
} }
} }
);
asyncComplete(
res,
asyncContext,
objectMapper,
future
);
} catch(IOException e) {
log.warn(e, "Exception parsing query");
final String errorMessage = e.getMessage() == null ? "no error message" : e.getMessage();
requestLogger.log(
new RequestLogLine(
new DateTime(),
req.getRemoteAddr(),
null,
new QueryStats(ImmutableMap.<String, Object>of("success", false, "exception", errorMessage))
)
);
res.setStatus(HttpServletResponse.SC_BAD_REQUEST);
objectMapper.writeValue(
res.getOutputStream(),
ImmutableMap.of("error", errorMessage)
);
} catch(Exception e) {
handleException(res, objectMapper, e);
}
}
private static void asyncComplete(
final HttpServletResponse res,
final AsyncContext asyncContext,
final ObjectMapper objectMapper,
ListenableFuture future
)
{
Futures.addCallback(
future,
new FutureCallback<Object>()
{
@Override
public void onSuccess(@Nullable Object o)
{
asyncContext.complete();
}
@Override
public void onFailure(Throwable throwable)
{
log.error(throwable, "Error processing query response");
try {
handleException(res, objectMapper, throwable);
} catch(Exception err) {
log.error(err, "Unable to handle exception response");
}
asyncContext.complete();
}
} }
); );
} }
private String makeUrl(final String host, final HttpServletRequest req) private URI rewriteURI(final String host, final HttpServletRequest req)
{ {
final StringBuilder uri = new StringBuilder("http://");
uri.append(host);
uri.append(req.getRequestURI());
final String queryString = req.getQueryString(); final String queryString = req.getQueryString();
final String requestURI = req.getRequestURI() == null ? "" : req.getRequestURI(); if (queryString != null) {
uri.append("?").append(queryString);
if (queryString == null) {
return String.format("http://%s%s", host, requestURI);
} }
return String.format("http://%s%s?%s", host, requestURI, queryString); return URI.create(uri.toString());
} }
private static void handleException(ObjectMapper objectMapper, AsyncContext asyncContext, Throwable exception) private static void handleException(HttpServletResponse response, ObjectMapper objectMapper, Throwable exception) throws IOException
{ {
try {
HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse();
if (!response.isCommitted()) { if (!response.isCommitted()) {
final String errorMessage = exception.getMessage() == null ? "null exception" : exception.getMessage(); final String errorMessage = exception.getMessage() == null ? "null exception" : exception.getMessage();
@ -277,37 +319,23 @@ public class AsyncQueryForwardingServlet extends HttpServlet
response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
objectMapper.writeValue( objectMapper.writeValue(
response.getOutputStream(), response.getOutputStream(),
ImmutableMap.of( ImmutableMap.of("error", errorMessage)
"error", errorMessage
)
); );
} }
response.flushBuffer(); response.flushBuffer();
} }
catch (IOException e) {
Throwables.propagate(e);
}
finally {
asyncContext.complete();
}
}
private static class PassthroughHttpResponseHandler implements HttpResponseHandler<OutputStream, OutputStream> private static class PassthroughHttpResponseHandler implements HttpResponseHandler<ServletOutputStream, ServletOutputStream>
{ {
private final AsyncContext asyncContext; private final HttpServletResponse response;
private final ObjectMapper objectMapper;
private final OutputStream outputStream;
public PassthroughHttpResponseHandler(AsyncContext asyncContext, ObjectMapper objectMapper) throws IOException public PassthroughHttpResponseHandler(HttpServletResponse response) throws IOException
{ {
this.asyncContext = asyncContext; this.response = response;
this.objectMapper = objectMapper;
this.outputStream = asyncContext.getResponse().getOutputStream();
} }
protected void copyStatusHeaders(HttpResponse clientResponse) protected void copyStatusHeaders(HttpResponse clientResponse)
{ {
final HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse();
response.setStatus(clientResponse.getStatus().getCode()); response.setStatus(clientResponse.getStatus().getCode());
response.setContentType(clientResponse.headers().get(HttpHeaders.Names.CONTENT_TYPE)); response.setContentType(clientResponse.headers().get(HttpHeaders.Names.CONTENT_TYPE));
@ -337,29 +365,29 @@ public class AsyncQueryForwardingServlet extends HttpServlet
} }
@Override @Override
public ClientResponse<OutputStream> handleResponse(HttpResponse clientResponse) public ClientResponse<ServletOutputStream> handleResponse(HttpResponse clientResponse)
{ {
copyStatusHeaders(clientResponse); copyStatusHeaders(clientResponse);
try { try {
final ServletOutputStream outputStream = response.getOutputStream();
ChannelBuffer buf = clientResponse.getContent(); ChannelBuffer buf = clientResponse.getContent();
buf.readBytes(outputStream, buf.readableBytes()); buf.readBytes(outputStream, buf.readableBytes());
return ClientResponse.unfinished(outputStream);
} }
catch (Exception e) { catch (Exception e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
} }
return ClientResponse.finished(outputStream);
} }
@Override @Override
public ClientResponse<OutputStream> handleChunk( public ClientResponse<ServletOutputStream> handleChunk(
ClientResponse<OutputStream> clientResponse, HttpChunk chunk ClientResponse<ServletOutputStream> clientResponse, HttpChunk chunk
) )
{ {
try { try {
ChannelBuffer buf = chunk.getContent(); ChannelBuffer buf = chunk.getContent();
buf.readBytes(outputStream, buf.readableBytes()); buf.readBytes(clientResponse.getObj(), buf.readableBytes());
} }
catch (Exception e) { catch (Exception e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
@ -368,25 +396,18 @@ public class AsyncQueryForwardingServlet extends HttpServlet
} }
@Override @Override
public ClientResponse<OutputStream> done(ClientResponse<OutputStream> clientResponse) public ClientResponse<ServletOutputStream> done(ClientResponse<ServletOutputStream> clientResponse)
{ {
asyncContext.complete();
return ClientResponse.finished(clientResponse.getObj()); return ClientResponse.finished(clientResponse.getObj());
} }
@Override @Override
public void exceptionCaught( public void exceptionCaught(
ClientResponse<OutputStream> clientResponse, ClientResponse<ServletOutputStream> clientResponse,
Throwable e Throwable e
) )
{ {
log.error(e, "Error processing query response"); // exceptions are handled on future callback
// throwing an exception here may cause resource leak
try {
handleException(objectMapper, asyncContext, e);
} catch(Exception err) {
log.error(err, "Unable to handle exception response");
}
} }
} }
} }

View File

@ -148,6 +148,16 @@ public class QueryResource
); );
} }
if ((boolean) query.getContextValue("b", false)) {
System.out.println("***NEW QUERY***");
while (true) {
System.out.println("SLEEPING");
Thread.sleep(10000);
}
} else if ((boolean) query.getContextValue("a", false)) {
return Response.ok("hi").build();
}
if (log.isDebugEnabled()) { if (log.isDebugEnabled()) {
log.debug("Got query [%s]", query); log.debug("Got query [%s]", query);
} }

View File

@ -30,6 +30,7 @@ import io.druid.server.QueryStats;
import io.druid.server.RequestLogLine; import io.druid.server.RequestLogLine;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import java.io.IOException;
import java.util.Map; import java.util.Map;
public class EmittingRequestLogger implements RequestLogger public class EmittingRequestLogger implements RequestLogger
@ -44,7 +45,7 @@ public class EmittingRequestLogger implements RequestLogger
} }
@Override @Override
public void log(final RequestLogLine requestLogLine) throws Exception public void log(final RequestLogLine requestLogLine) throws IOException
{ {
emitter.emit(new RequestLogEventBuilder(feed, requestLogLine)); emitter.emit(new RequestLogEventBuilder(feed, requestLogLine));
} }

View File

@ -110,7 +110,7 @@ public class FileRequestLogger implements RequestLogger
} }
@Override @Override
public void log(RequestLogLine requestLogLine) throws Exception public void log(RequestLogLine requestLogLine) throws IOException
{ {
synchronized (lock) { synchronized (lock) {
fileWriter.write( fileWriter.write(

View File

@ -21,12 +21,14 @@ package io.druid.server.log;
import io.druid.server.RequestLogLine; import io.druid.server.RequestLogLine;
import java.io.IOException;
/** /**
*/ */
public class NoopRequestLogger implements RequestLogger public class NoopRequestLogger implements RequestLogger
{ {
@Override @Override
public void log(RequestLogLine requestLogLine) throws Exception public void log(RequestLogLine requestLogLine) throws IOException
{ {
// This is a no op! // This is a no op!
} }

View File

@ -21,9 +21,11 @@ package io.druid.server.log;
import io.druid.server.RequestLogLine; import io.druid.server.RequestLogLine;
import java.io.IOException;
/** /**
*/ */
public interface RequestLogger public interface RequestLogger
{ {
public void log(RequestLogLine requestLogLine) throws Exception; public void log(RequestLogLine requestLogLine) throws IOException;
} }

View File

@ -80,7 +80,6 @@ public class RouterJettyServerInitializer implements JettyServerInitializer
queries.addServlet( queries.addServlet(
new ServletHolder( new ServletHolder(
new AsyncQueryForwardingServlet( new AsyncQueryForwardingServlet(
config,
jsonMapper, jsonMapper,
smileMapper, smileMapper,
hostFinder, hostFinder,