-
Notifications
You must be signed in to change notification settings - Fork 2.8k
/
StatementResource.java
294 lines (261 loc) · 12 KB
/
StatementResource.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.prestosql.server.protocol;
import com.google.common.collect.Ordering;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import io.airlift.concurrent.BoundedExecutor;
import io.airlift.units.DataSize;
import io.airlift.units.Duration;
import io.prestosql.client.QueryResults;
import io.prestosql.execution.QueryManager;
import io.prestosql.memory.context.SimpleLocalMemoryContext;
import io.prestosql.metadata.SessionPropertyManager;
import io.prestosql.operator.ExchangeClient;
import io.prestosql.operator.ExchangeClientSupplier;
import io.prestosql.server.ForStatementResource;
import io.prestosql.server.HttpRequestSessionContext;
import io.prestosql.server.SessionContext;
import io.prestosql.spi.QueryId;
import io.prestosql.spi.block.BlockEncodingSerde;
import javax.annotation.Nullable;
import javax.annotation.PreDestroy;
import javax.inject.Inject;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.DELETE;
import javax.ws.rs.GET;
import javax.ws.rs.HeaderParam;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.container.AsyncResponse;
import javax.ws.rs.container.Suspended;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.ResponseBuilder;
import javax.ws.rs.core.Response.Status;
import javax.ws.rs.core.UriInfo;
import java.io.UnsupportedEncodingException;
import java.net.URLEncoder;
import java.util.Map.Entry;
import java.util.OptionalLong;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ScheduledExecutorService;
import static com.google.common.base.Strings.isNullOrEmpty;
import static com.google.common.net.HttpHeaders.X_FORWARDED_PROTO;
import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
import static io.airlift.concurrent.Threads.threadsNamed;
import static io.airlift.http.server.AsyncResponseHandler.bindAsyncResponse;
import static io.airlift.units.DataSize.Unit.MEGABYTE;
import static io.prestosql.client.PrestoHeaders.PRESTO_ADDED_PREPARE;
import static io.prestosql.client.PrestoHeaders.PRESTO_CLEAR_SESSION;
import static io.prestosql.client.PrestoHeaders.PRESTO_CLEAR_TRANSACTION_ID;
import static io.prestosql.client.PrestoHeaders.PRESTO_DEALLOCATED_PREPARE;
import static io.prestosql.client.PrestoHeaders.PRESTO_SET_CATALOG;
import static io.prestosql.client.PrestoHeaders.PRESTO_SET_PATH;
import static io.prestosql.client.PrestoHeaders.PRESTO_SET_ROLE;
import static io.prestosql.client.PrestoHeaders.PRESTO_SET_SCHEMA;
import static io.prestosql.client.PrestoHeaders.PRESTO_SET_SESSION;
import static io.prestosql.client.PrestoHeaders.PRESTO_STARTED_TRANSACTION_ID;
import static io.prestosql.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
@Path("/v1/statement")
public class StatementResource
{
private static final Duration MAX_WAIT_TIME = new Duration(1, SECONDS);
private static final Ordering<Comparable<Duration>> WAIT_ORDERING = Ordering.natural().nullsLast();
private static final DataSize DEFAULT_TARGET_RESULT_SIZE = new DataSize(1, MEGABYTE);
private static final DataSize MAX_TARGET_RESULT_SIZE = new DataSize(128, MEGABYTE);
private final QueryManager queryManager;
private final SessionPropertyManager sessionPropertyManager;
private final ExchangeClientSupplier exchangeClientSupplier;
private final BlockEncodingSerde blockEncodingSerde;
private final BoundedExecutor responseExecutor;
private final ScheduledExecutorService timeoutExecutor;
private final ConcurrentMap<QueryId, Query> queries = new ConcurrentHashMap<>();
private final ScheduledExecutorService queryPurger = newSingleThreadScheduledExecutor(threadsNamed("query-purger"));
@Inject
public StatementResource(
QueryManager queryManager,
SessionPropertyManager sessionPropertyManager,
ExchangeClientSupplier exchangeClientSupplier,
BlockEncodingSerde blockEncodingSerde,
@ForStatementResource BoundedExecutor responseExecutor,
@ForStatementResource ScheduledExecutorService timeoutExecutor)
{
this.queryManager = requireNonNull(queryManager, "queryManager is null");
this.sessionPropertyManager = requireNonNull(sessionPropertyManager, "sessionPropertyManager is null");
this.exchangeClientSupplier = requireNonNull(exchangeClientSupplier, "exchangeClientSupplier is null");
this.blockEncodingSerde = requireNonNull(blockEncodingSerde, "blockEncodingSerde is null");
this.responseExecutor = requireNonNull(responseExecutor, "responseExecutor is null");
this.timeoutExecutor = requireNonNull(timeoutExecutor, "timeoutExecutor is null");
queryPurger.scheduleWithFixedDelay(new PurgeQueriesRunnable(queries, queryManager), 200, 200, MILLISECONDS);
}
@PreDestroy
public void stop()
{
queryPurger.shutdownNow();
}
@POST
@Produces(MediaType.APPLICATION_JSON)
public Response createQuery(
String statement,
@HeaderParam(X_FORWARDED_PROTO) String proto,
@Context HttpServletRequest servletRequest,
@Context UriInfo uriInfo)
{
if (isNullOrEmpty(statement)) {
throw new WebApplicationException(Response
.status(Status.BAD_REQUEST)
.type(MediaType.TEXT_PLAIN)
.entity("SQL statement is empty")
.build());
}
if (isNullOrEmpty(proto)) {
proto = uriInfo.getRequestUri().getScheme();
}
SessionContext sessionContext = new HttpRequestSessionContext(servletRequest);
ExchangeClient exchangeClient = exchangeClientSupplier.get(new SimpleLocalMemoryContext(newSimpleAggregatedMemoryContext(), StatementResource.class.getSimpleName()));
Query query = Query.create(
sessionContext,
statement,
queryManager,
sessionPropertyManager,
exchangeClient,
responseExecutor,
timeoutExecutor,
blockEncodingSerde);
queries.put(query.getQueryId(), query);
QueryResults queryResults = query.getNextResult(OptionalLong.empty(), uriInfo, proto, DEFAULT_TARGET_RESULT_SIZE);
return toResponse(query, queryResults);
}
@GET
@Path("{queryId}/{slug}/{token}")
@Produces(MediaType.APPLICATION_JSON)
public void getQueryResults(
@PathParam("queryId") QueryId queryId,
@PathParam("slug") String slug,
@PathParam("token") long token,
@QueryParam("maxWait") Duration maxWait,
@QueryParam("targetResultSize") DataSize targetResultSize,
@HeaderParam(X_FORWARDED_PROTO) String proto,
@Context UriInfo uriInfo,
@Suspended AsyncResponse asyncResponse)
{
Query query = getQuery(queryId, slug);
if (query == null) {
asyncResponse.resume(Response.status(Status.NOT_FOUND).build());
return;
}
if (isNullOrEmpty(proto)) {
proto = uriInfo.getRequestUri().getScheme();
}
asyncQueryResults(query, OptionalLong.of(token), maxWait, targetResultSize, uriInfo, proto, asyncResponse);
}
@Nullable
private Query getQuery(QueryId queryId, String slug)
{
Query query = queries.get(queryId);
if (query != null && query.isSlugValid(slug)) {
return query;
}
return null;
}
private void asyncQueryResults(
Query query,
OptionalLong token,
Duration maxWait,
DataSize targetResultSize,
UriInfo uriInfo,
String scheme,
AsyncResponse asyncResponse)
{
Duration wait = WAIT_ORDERING.min(MAX_WAIT_TIME, maxWait);
if (targetResultSize == null) {
targetResultSize = DEFAULT_TARGET_RESULT_SIZE;
}
else {
targetResultSize = Ordering.natural().min(targetResultSize, MAX_TARGET_RESULT_SIZE);
}
ListenableFuture<QueryResults> queryResultsFuture = query.waitForResults(token, uriInfo, scheme, wait, targetResultSize);
ListenableFuture<Response> response = Futures.transform(queryResultsFuture, queryResults -> toResponse(query, queryResults), directExecutor());
bindAsyncResponse(asyncResponse, response, responseExecutor);
}
private static Response toResponse(Query query, QueryResults queryResults)
{
ResponseBuilder response = Response.ok(queryResults);
query.getSetCatalog().ifPresent(catalog -> response.header(PRESTO_SET_CATALOG, catalog));
query.getSetSchema().ifPresent(schema -> response.header(PRESTO_SET_SCHEMA, schema));
query.getSetPath().ifPresent(path -> response.header(PRESTO_SET_PATH, path));
// add set session properties
query.getSetSessionProperties().entrySet()
.forEach(entry -> response.header(PRESTO_SET_SESSION, entry.getKey() + '=' + urlEncode(entry.getValue())));
// add clear session properties
query.getResetSessionProperties()
.forEach(name -> response.header(PRESTO_CLEAR_SESSION, name));
// add set roles
query.getSetRoles().entrySet()
.forEach(entry -> response.header(PRESTO_SET_ROLE, entry.getKey() + '=' + urlEncode(entry.getValue().toString())));
// add added prepare statements
for (Entry<String, String> entry : query.getAddedPreparedStatements().entrySet()) {
String encodedKey = urlEncode(entry.getKey());
String encodedValue = urlEncode(entry.getValue());
response.header(PRESTO_ADDED_PREPARE, encodedKey + '=' + encodedValue);
}
// add deallocated prepare statements
for (String name : query.getDeallocatedPreparedStatements()) {
response.header(PRESTO_DEALLOCATED_PREPARE, urlEncode(name));
}
// add new transaction ID
query.getStartedTransactionId()
.ifPresent(transactionId -> response.header(PRESTO_STARTED_TRANSACTION_ID, transactionId));
// add clear transaction ID directive
if (query.isClearTransactionId()) {
response.header(PRESTO_CLEAR_TRANSACTION_ID, true);
}
return response.build();
}
@DELETE
@Path("{queryId}/{slug}/{token}")
@Produces(MediaType.APPLICATION_JSON)
public Response cancelQuery(
@PathParam("queryId") QueryId queryId,
@PathParam("slug") String slug,
@PathParam("token") long token)
{
Query query = getQuery(queryId, slug);
if (query == null) {
return Response.status(Status.NOT_FOUND).build();
}
query.cancel();
return Response.noContent().build();
}
private static String urlEncode(String value)
{
try {
return URLEncoder.encode(value, "UTF-8");
}
catch (UnsupportedEncodingException e) {
throw new AssertionError(e);
}
}
}