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

feat(client): support push query termination in Java client #5371

Merged
merged 4 commits into from
May 16, 2020
Merged
Changes from 1 commit
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
Prev Previous commit
Next Next commit
refactor: make ClientImpl more readable
  • Loading branch information
vcrfxia committed May 16, 2020
commit 8e49c9f876b551f03e633866fd11a17661886496
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ private <T> CompletableFuture<T> makeQueryRequest(
"/query-stream",
requestBody,
cf,
response -> handleSuccessfulQueryResponse(response, cf, responseHandlerSupplier)
response -> handleQueryResponse(response, cf, responseHandlerSupplier)
);

return cf;
Expand All @@ -159,7 +159,7 @@ private CompletableFuture<Void> makeCloseQueryRequest(final String queryId) {
"/close-query",
new JsonObject().put("queryId", queryId),
cf,
response -> handleSuccessfulCloseQueryResponse(cf)
response -> handleCloseQueryResponse(response, cf)
);

return cf;
Expand All @@ -169,11 +169,11 @@ private <T> void makeRequest(
final String path,
final JsonObject requestBody,
final CompletableFuture<T> cf,
final Handler<HttpClientResponse> successfulResponseHandler) {
final Handler<HttpClientResponse> responseHandler) {
HttpClientRequest request = httpClient.request(HttpMethod.POST,
serverSocketAddress, clientOptions.getPort(), clientOptions.getHost(),
path,
response -> handleResponse(response, cf, successfulResponseHandler))
responseHandler)
.exceptionHandler(cf::completeExceptionally);
if (clientOptions.isUseBasicAuth()) {
request = configureBasicAuth(request);
Expand All @@ -185,41 +185,47 @@ private HttpClientRequest configureBasicAuth(final HttpClientRequest request) {
return request.putHeader(AUTHORIZATION.toString(), basicAuthHeader);
}

private static <T> void handleResponse(
private static <T> void handleQueryResponse(
final HttpClientResponse response,
final CompletableFuture<T> cf,
Copy link
Contributor

Choose a reason for hiding this comment

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

Seems odd to pass in both a cf and a Handler to signal back both exceptionally and successful results. Can't the cf do both?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Refactored this to be less convoluted.

final Handler<HttpClientResponse> successfulResponseHandler
) {
final ResponseHandlerSupplier<T> responseHandlerSupplier) {
if (response.statusCode() == OK.code()) {
successfulResponseHandler.handle(response);
final RecordParser recordParser = RecordParser.newDelimited("\n", response);
final QueryResponseHandler<T> responseHandler =
responseHandlerSupplier.get(Vertx.currentContext(), recordParser, cf);

recordParser.handler(responseHandler::handleBodyBuffer);
recordParser.endHandler(responseHandler::handleBodyEnd);
recordParser.exceptionHandler(responseHandler::handleException);
} else {
response.bodyHandler(buffer -> {
final JsonObject errorResponse = buffer.toJsonObject();
cf.completeExceptionally(new KsqlRestClientException(String.format(
"Received %d response from server: %s. Error code: %d",
response.statusCode(),
errorResponse.getString("message"),
errorResponse.getInteger("errorCode")
)));
});
handleErrorResponse(response, cf);
}
}

private static <T> void handleSuccessfulQueryResponse(
private static void handleCloseQueryResponse(
final HttpClientResponse response,
final CompletableFuture<T> cf,
final ResponseHandlerSupplier<T> responseHandlerSupplier) {
final RecordParser recordParser = RecordParser.newDelimited("\n", response);
final QueryResponseHandler<T> responseHandler =
responseHandlerSupplier.get(Vertx.currentContext(), recordParser, cf);

recordParser.handler(responseHandler::handleBodyBuffer);
recordParser.endHandler(responseHandler::handleBodyEnd);
recordParser.exceptionHandler(responseHandler::handleException);
final CompletableFuture<Void> cf
) {
if (response.statusCode() == OK.code()) {
cf.complete(null);
} else {
handleErrorResponse(response, cf);
}
}

private static void handleSuccessfulCloseQueryResponse(final CompletableFuture<Void> cf) {
cf.complete(null);
private static <T> void handleErrorResponse(
final HttpClientResponse response,
final CompletableFuture<T> cf
) {
response.bodyHandler(buffer -> {
final JsonObject errorResponse = buffer.toJsonObject();
cf.completeExceptionally(new KsqlRestClientException(String.format(
"Received %d response from server: %s. Error code: %d",
response.statusCode(),
errorResponse.getString("message"),
errorResponse.getInteger("errorCode")
)));
});
}

private static HttpClient createHttpClient(final Vertx vertx, final ClientOptions clientOptions) {
Expand Down