@Override
public Response apply(Request request)
{
if (request.getMethod().equalsIgnoreCase("DELETE")) {
return new TestingResponse(HttpStatus.NO_CONTENT, ImmutableListMultimap.<String, String>of(), new byte[0]);
}
// verify we got a data size and it parses correctly
assertTrue(!request.getHeaders().get(PrestoHeaders.PRESTO_MAX_SIZE).isEmpty());
DataSize maxSize = DataSize.valueOf(request.getHeader(PrestoHeaders.PRESTO_MAX_SIZE));
assertEquals(maxSize, expectedMaxSize);
RequestLocation requestLocation = new RequestLocation(request.getUri());
URI location = requestLocation.getLocation();
BlockingQueue<Page> pages = pagesByLocation.get(location);
long token = tokenByLocation.get(location);
// if location is complete return GONE
if (completeByLocation.get(location) == Boolean.TRUE && (pages == null || pages.isEmpty())) {
return new TestingResponse(HttpStatus.GONE, ImmutableListMultimap.of(
PRESTO_PAGE_TOKEN, String.valueOf(token),
PRESTO_PAGE_NEXT_TOKEN, String.valueOf(token)
), new byte[0]);
}
// if no pages, return NO CONTENT
if (pages == null) {
return new TestingResponse(HttpStatus.NO_CONTENT, ImmutableListMultimap.of(
PRESTO_PAGE_TOKEN, String.valueOf(token),
PRESTO_PAGE_NEXT_TOKEN, String.valueOf(token)
), new byte[0]);
}
assertEquals(requestLocation.getSequenceId(), token, "token");
// wait for a single page to arrive
Page page = null;
try {
page = pages.poll(10, TimeUnit.MILLISECONDS);
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
// if no page, return NO CONTENT
if (page == null) {
return new TestingResponse(HttpStatus.NO_CONTENT, ImmutableListMultimap.of(
PRESTO_PAGE_TOKEN, String.valueOf(token),
PRESTO_PAGE_NEXT_TOKEN, String.valueOf(token)
), new byte[0]);
}
// add pages up to the size limit
List<Page> responsePages = new ArrayList<>();
responsePages.add(page);
long responseSize = page.getDataSize().toBytes();
while (responseSize < maxSize.toBytes()) {
page = pages.poll();
if (page == null) {
break;
}
responsePages.add(page);
responseSize += page.getDataSize().toBytes();
}
// update sequence id
long nextToken = token + responsePages.size();
tokenByLocation.put(location, nextToken);
DynamicSliceOutput sliceOutput = new DynamicSliceOutput(64);
PagesSerde.writePages(sliceOutput, responsePages);
byte[] bytes = sliceOutput.slice().getBytes();
return new TestingResponse(HttpStatus.OK,
ImmutableListMultimap.of(
CONTENT_TYPE, PRESTO_PAGES,
PRESTO_PAGE_TOKEN, String.valueOf(token),
PRESTO_PAGE_NEXT_TOKEN, String.valueOf(nextToken)
),