-
Notifications
You must be signed in to change notification settings - Fork 45
/
Copy pathJavaNetHttpPollingClient.java
180 lines (148 loc) · 6.97 KB
/
JavaNetHttpPollingClient.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
package com.getindata.connectors.http.internal.table.lookup;
import java.io.IOException;
import java.net.http.HttpClient;
import java.net.http.HttpResponse;
import java.net.http.HttpResponse.BodyHandlers;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.NullNode;
import lombok.extern.slf4j.Slf4j;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.table.data.RowData;
import org.apache.flink.util.StringUtils;
import com.getindata.connectors.http.HttpPostRequestCallback;
import com.getindata.connectors.http.internal.PollingClient;
import com.getindata.connectors.http.internal.config.HttpConnectorConfigConstants;
import com.getindata.connectors.http.internal.status.ComposeHttpStatusCodeChecker;
import com.getindata.connectors.http.internal.status.ComposeHttpStatusCodeChecker.ComposeHttpStatusCodeCheckerConfig;
import com.getindata.connectors.http.internal.status.HttpStatusCodeChecker;
import static com.getindata.connectors.http.internal.config.HttpConnectorConfigConstants.RESULT_TYPE;
/**
* An implementation of {@link PollingClient} that uses Java 11's {@link HttpClient}.
* This implementation supports HTTP traffic only.
*/
@Slf4j
public class JavaNetHttpPollingClient implements PollingClient<RowData> {
private static final String RESULT_TYPE_SINGLE_VALUE = "single-value";
private static final String RESULT_TYPE_ARRAY = "array";
private final HttpClient httpClient;
private final HttpStatusCodeChecker statusCodeChecker;
private final DeserializationSchema<RowData> responseBodyDecoder;
private final HttpRequestFactory requestFactory;
private final ObjectMapper objectMapper;
private final HttpPostRequestCallback<HttpLookupSourceRequestEntry> httpPostRequestCallback;
private final HttpLookupConfig options;
public JavaNetHttpPollingClient(
HttpClient httpClient,
DeserializationSchema<RowData> responseBodyDecoder,
HttpLookupConfig options,
HttpRequestFactory requestFactory) {
this.httpClient = httpClient;
this.responseBodyDecoder = responseBodyDecoder;
this.requestFactory = requestFactory;
this.objectMapper = new ObjectMapper();
this.httpPostRequestCallback = options.getHttpPostRequestCallback();
// TODO Inject this via constructor when implementing a response processor.
// Processor will be injected and it will wrap statusChecker implementation.
ComposeHttpStatusCodeCheckerConfig checkerConfig =
ComposeHttpStatusCodeCheckerConfig.builder()
.properties(options.getProperties())
.whiteListPrefix(
HttpConnectorConfigConstants.HTTP_ERROR_SOURCE_LOOKUP_CODE_WHITE_LIST
)
.errorCodePrefix(HttpConnectorConfigConstants.HTTP_ERROR_SOURCE_LOOKUP_CODES_LIST)
.build();
this.statusCodeChecker = new ComposeHttpStatusCodeChecker(checkerConfig);
this.options = options;
}
@Override
public Collection<RowData> pull(RowData lookupRow) {
try {
log.debug("Collection<RowData> pull with Rowdata={}.", lookupRow);
return queryAndProcess(lookupRow);
} catch (Exception e) {
log.error("Exception during HTTP request.", e);
return Collections.emptyList();
}
}
// TODO Add Retry Policy And configure TimeOut from properties
private Collection<RowData> queryAndProcess(RowData lookupData) throws Exception {
HttpLookupSourceRequestEntry request = requestFactory.buildLookupRequest(lookupData);
HttpResponse<String> response = httpClient.send(
request.getHttpRequest(),
BodyHandlers.ofString()
);
return processHttpResponse(response, request);
}
private Collection<RowData> processHttpResponse(
HttpResponse<String> response,
HttpLookupSourceRequestEntry request) throws IOException {
this.httpPostRequestCallback.call(response, request, "endpoint", Collections.emptyMap());
if (response == null) {
return Collections.emptyList();
}
String responseBody = response.body();
int statusCode = response.statusCode();
log.debug(String.format("Received status code [%s] for RestTableSource request " +
"with Server response body [%s] ", statusCode, responseBody));
if (notErrorCodeAndNotEmptyBody(responseBody, statusCode)) {
return deserialize(responseBody);
} else {
log.warn(
String.format("Returned Http status code was invalid or returned body was empty. "
+ "Status Code [%s]", statusCode)
);
return Collections.emptyList();
}
}
private boolean notErrorCodeAndNotEmptyBody(String body, int statusCode) {
return !(StringUtils.isNullOrWhitespaceOnly(body) || statusCodeChecker.isErrorCode(
statusCode));
}
@VisibleForTesting
HttpRequestFactory getRequestFactory() {
return this.requestFactory;
}
private Collection<RowData> deserialize(String responseBody) throws IOException {
byte[] rawBytes = responseBody.getBytes();
String resultType =
options.getProperties().getProperty(RESULT_TYPE, RESULT_TYPE_SINGLE_VALUE);
if (resultType.equals(RESULT_TYPE_SINGLE_VALUE)) {
return deserializeSingleValue(rawBytes);
} else if (resultType.equals(RESULT_TYPE_ARRAY)) {
return deserializeArray(rawBytes);
} else {
throw new IllegalStateException(
String.format("Unknown lookup source result type '%s'.", resultType));
}
}
private List<RowData> deserializeSingleValue(byte[] rawBytes) throws IOException {
return Optional.ofNullable(responseBodyDecoder.deserialize(rawBytes))
.map(Collections::singletonList)
.orElse(Collections.emptyList());
}
private List<RowData> deserializeArray(byte[] rawBytes) throws IOException {
List<JsonNode> rawObjects =
objectMapper.readValue(rawBytes, new TypeReference<>() {
});
List<RowData> result = new ArrayList<>();
for (JsonNode rawObject : rawObjects) {
if (!(rawObject instanceof NullNode)) {
RowData deserialized =
responseBodyDecoder.deserialize(rawObject.toString().getBytes());
// deserialize() returns null if deserialization fails
if (deserialized != null) {
result.add(deserialized);
}
}
}
return result;
}
}