Skip to content

Commit 1a140ea

Browse files
grzegorz8Grzegorz Kołakowski
and
Grzegorz Kołakowski
authored
Support source lookup array results (#135)
--------- Co-authored-by: Grzegorz Kołakowski <[email protected]>
1 parent b95e7fb commit 1a140ea

File tree

11 files changed

+267
-37
lines changed

11 files changed

+267
-37
lines changed

CHANGELOG.md

+4
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,10 @@
22

33
## [Unreleased]
44

5+
### Added
6+
7+
- Allow to fetch multiple results from REST API endpoint (`gid.connector.http.source.lookup.result-type`).
8+
59
## [0.16.0] - 2024-10-18
610

711
### Added

README.md

+10
Original file line numberDiff line numberDiff line change
@@ -171,6 +171,16 @@ Because of that, if AsyncIO timer passes, Flink will throw TimeoutException whic
171171
The HTTP request timeouts on the other hand will not cause Job restart. In that case, exception will be logged into application logs.
172172
To avoid job restart on timeouts caused by Lookup queries, the value of `gid.connector.http.source.lookup.request.timeout` should be smaller than `table.exec.async-lookup.timeout`.
173173

174+
#### Lookup multiple results
175+
176+
Typically, join can return zero, one or more results. What is more, there are lots of possible REST API designs and
177+
pagination methods. Currently, the connector supports only two simple approaches (`gid.connector.http.source.lookup.result-type`):
178+
179+
- `single-value` - REST API returns single object.
180+
- `array` - REST API returns array of objects. Pagination is not supported yet.
181+
182+
Please be informed that the mechanism will be enhanced in the future. See [HTTP-118](https://github.com/getindata/flink-http-connector/issues/118).
183+
174184
### HTTP Sink
175185
The following example shows the minimum Table API example to create a [HttpDynamicSink](src/main/java/com/getindata/connectors/http/internal/table/HttpDynamicSink.java) that writes JSON values to an HTTP endpoint using POST method, assuming Flink has JAR of [JSON serializer](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/formats/json/) installed:
176186

pom.xml

+7
Original file line numberDiff line numberDiff line change
@@ -77,6 +77,7 @@ under the License.
7777
<maven.compiler.target>${target.java.version}</maven.compiler.target>
7878
<log4j.version>2.17.2</log4j.version>
7979
<lombok.version>1.18.22</lombok.version>
80+
<jackson.version>2.18.1</jackson.version>
8081
<junit4.version>4.13.2</junit4.version>
8182
<junit5.version>5.10.1</junit5.version>
8283
<junit.jupiter.version>${junit5.version}</junit.jupiter.version>
@@ -153,6 +154,12 @@ under the License.
153154
<scope>provided</scope>
154155
</dependency>
155156

157+
<dependency>
158+
<groupId>com.fasterxml.jackson.core</groupId>
159+
<artifactId>jackson-databind</artifactId>
160+
<version>${jackson.version}</version>
161+
</dependency>
162+
156163
<dependency>
157164
<groupId>org.projectlombok</groupId>
158165
<artifactId>lombok</artifactId>

src/main/java/com/getindata/connectors/http/internal/PollingClient.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,6 @@
11
package com.getindata.connectors.http.internal;
22

3-
import java.util.Optional;
3+
import java.util.Collection;
44

55
import org.apache.flink.table.data.RowData;
66

@@ -14,5 +14,5 @@ public interface PollingClient<T> {
1414
* @param lookupRow A {@link RowData} containing request parameters.
1515
* @return an optional result of data lookup.
1616
*/
17-
Optional<T> pull(RowData lookupRow);
17+
Collection<T> pull(RowData lookupRow);
1818
}

src/main/java/com/getindata/connectors/http/internal/config/HttpConnectorConfigConstants.java

+3
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,9 @@ public final class HttpConnectorConfigConstants {
4343
public static final String LOOKUP_SOURCE_HEADER_USE_RAW = GID_CONNECTOR_HTTP
4444
+ "source.lookup.use-raw-authorization-header";
4545

46+
public static final String RESULT_TYPE = GID_CONNECTOR_HTTP
47+
+ "source.lookup.result-type";
48+
4649
// --------- Error code handling configuration ---------
4750
public static final String HTTP_ERROR_SINK_CODE_WHITE_LIST =
4851
GID_CONNECTOR_HTTP + "sink.error.code.exclude";

src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpTableLookupFunction.java

+1-4
Original file line numberDiff line numberDiff line change
@@ -1,8 +1,6 @@
11
package com.getindata.connectors.http.internal.table.lookup;
22

33
import java.util.Collection;
4-
import java.util.Collections;
5-
import java.util.Optional;
64
import java.util.concurrent.atomic.AtomicInteger;
75

86
import lombok.AccessLevel;
@@ -66,7 +64,6 @@ public void open(FunctionContext context) throws Exception {
6664
@Override
6765
public Collection<RowData> lookup(RowData keyRow) {
6866
localHttpCallCounter.incrementAndGet();
69-
Optional<RowData> result = client.pull(keyRow);
70-
return result.map(Collections::singletonList).orElse(Collections.emptyList());
67+
return client.pull(keyRow);
7168
}
7269
}

src/main/java/com/getindata/connectors/http/internal/table/lookup/JavaNetHttpPollingClient.java

+63-8
Original file line numberDiff line numberDiff line change
@@ -4,9 +4,16 @@
44
import java.net.http.HttpClient;
55
import java.net.http.HttpResponse;
66
import java.net.http.HttpResponse.BodyHandlers;
7+
import java.util.ArrayList;
8+
import java.util.Collection;
79
import java.util.Collections;
10+
import java.util.List;
811
import java.util.Optional;
912

13+
import com.fasterxml.jackson.core.type.TypeReference;
14+
import com.fasterxml.jackson.databind.JsonNode;
15+
import com.fasterxml.jackson.databind.ObjectMapper;
16+
import com.fasterxml.jackson.databind.node.NullNode;
1017
import lombok.extern.slf4j.Slf4j;
1118
import org.apache.flink.annotation.VisibleForTesting;
1219
import org.apache.flink.api.common.serialization.DeserializationSchema;
@@ -19,6 +26,7 @@
1926
import com.getindata.connectors.http.internal.status.ComposeHttpStatusCodeChecker;
2027
import com.getindata.connectors.http.internal.status.ComposeHttpStatusCodeChecker.ComposeHttpStatusCodeCheckerConfig;
2128
import com.getindata.connectors.http.internal.status.HttpStatusCodeChecker;
29+
import static com.getindata.connectors.http.internal.config.HttpConnectorConfigConstants.RESULT_TYPE;
2230

2331
/**
2432
* An implementation of {@link PollingClient} that uses Java 11's {@link HttpClient}.
@@ -27,6 +35,9 @@
2735
@Slf4j
2836
public class JavaNetHttpPollingClient implements PollingClient<RowData> {
2937

38+
private static final String RESULT_TYPE_SINGLE_VALUE = "single-value";
39+
private static final String RESULT_TYPE_ARRAY = "array";
40+
3041
private final HttpClient httpClient;
3142

3243
private final HttpStatusCodeChecker statusCodeChecker;
@@ -35,8 +46,12 @@ public class JavaNetHttpPollingClient implements PollingClient<RowData> {
3546

3647
private final HttpRequestFactory requestFactory;
3748

49+
private final ObjectMapper objectMapper;
50+
3851
private final HttpPostRequestCallback<HttpLookupSourceRequestEntry> httpPostRequestCallback;
3952

53+
private final HttpLookupConfig options;
54+
4055
public JavaNetHttpPollingClient(
4156
HttpClient httpClient,
4257
DeserializationSchema<RowData> responseBodyDecoder,
@@ -47,6 +62,7 @@ public JavaNetHttpPollingClient(
4762
this.responseBodyDecoder = responseBodyDecoder;
4863
this.requestFactory = requestFactory;
4964

65+
this.objectMapper = new ObjectMapper();
5066
this.httpPostRequestCallback = options.getHttpPostRequestCallback();
5167

5268
// TODO Inject this via constructor when implementing a response processor.
@@ -61,21 +77,22 @@ public JavaNetHttpPollingClient(
6177
.build();
6278

6379
this.statusCodeChecker = new ComposeHttpStatusCodeChecker(checkerConfig);
80+
this.options = options;
6481
}
6582

6683
@Override
67-
public Optional<RowData> pull(RowData lookupRow) {
84+
public Collection<RowData> pull(RowData lookupRow) {
6885
try {
69-
log.debug("Optional<RowData> pull with Rowdata={}.", lookupRow);
86+
log.debug("Collection<RowData> pull with Rowdata={}.", lookupRow);
7087
return queryAndProcess(lookupRow);
7188
} catch (Exception e) {
7289
log.error("Exception during HTTP request.", e);
73-
return Optional.empty();
90+
return Collections.emptyList();
7491
}
7592
}
7693

7794
// TODO Add Retry Policy And configure TimeOut from properties
78-
private Optional<RowData> queryAndProcess(RowData lookupData) throws Exception {
95+
private Collection<RowData> queryAndProcess(RowData lookupData) throws Exception {
7996

8097
HttpLookupSourceRequestEntry request = requestFactory.buildLookupRequest(lookupData);
8198
HttpResponse<String> response = httpClient.send(
@@ -85,14 +102,14 @@ private Optional<RowData> queryAndProcess(RowData lookupData) throws Exception {
85102
return processHttpResponse(response, request);
86103
}
87104

88-
private Optional<RowData> processHttpResponse(
105+
private Collection<RowData> processHttpResponse(
89106
HttpResponse<String> response,
90107
HttpLookupSourceRequestEntry request) throws IOException {
91108

92109
this.httpPostRequestCallback.call(response, request, "endpoint", Collections.emptyMap());
93110

94111
if (response == null) {
95-
return Optional.empty();
112+
return Collections.emptyList();
96113
}
97114

98115
String responseBody = response.body();
@@ -102,14 +119,14 @@ private Optional<RowData> processHttpResponse(
102119
"with Server response body [%s] ", statusCode, responseBody));
103120

104121
if (notErrorCodeAndNotEmptyBody(responseBody, statusCode)) {
105-
return Optional.ofNullable(responseBodyDecoder.deserialize(responseBody.getBytes()));
122+
return deserialize(responseBody);
106123
} else {
107124
log.warn(
108125
String.format("Returned Http status code was invalid or returned body was empty. "
109126
+ "Status Code [%s]", statusCode)
110127
);
111128

112-
return Optional.empty();
129+
return Collections.emptyList();
113130
}
114131
}
115132

@@ -122,4 +139,42 @@ private boolean notErrorCodeAndNotEmptyBody(String body, int statusCode) {
122139
HttpRequestFactory getRequestFactory() {
123140
return this.requestFactory;
124141
}
142+
143+
private Collection<RowData> deserialize(String responseBody) throws IOException {
144+
byte[] rawBytes = responseBody.getBytes();
145+
String resultType =
146+
options.getProperties().getProperty(RESULT_TYPE, RESULT_TYPE_SINGLE_VALUE);
147+
if (resultType.equals(RESULT_TYPE_SINGLE_VALUE)) {
148+
return deserializeSingleValue(rawBytes);
149+
} else if (resultType.equals(RESULT_TYPE_ARRAY)) {
150+
return deserializeArray(rawBytes);
151+
} else {
152+
throw new IllegalStateException(
153+
String.format("Unknown lookup source result type '%s'.", resultType));
154+
}
155+
}
156+
157+
private List<RowData> deserializeSingleValue(byte[] rawBytes) throws IOException {
158+
return Optional.ofNullable(responseBodyDecoder.deserialize(rawBytes))
159+
.map(Collections::singletonList)
160+
.orElse(Collections.emptyList());
161+
}
162+
163+
private List<RowData> deserializeArray(byte[] rawBytes) throws IOException {
164+
List<JsonNode> rawObjects =
165+
objectMapper.readValue(rawBytes, new TypeReference<>() {
166+
});
167+
List<RowData> result = new ArrayList<>();
168+
for (JsonNode rawObject : rawObjects) {
169+
if (!(rawObject instanceof NullNode)) {
170+
RowData deserialized =
171+
responseBodyDecoder.deserialize(rawObject.toString().getBytes());
172+
// deserialize() returns null if deserialization fails
173+
if (deserialized != null) {
174+
result.add(deserialized);
175+
}
176+
}
177+
}
178+
return result;
179+
}
125180
}

0 commit comments

Comments
 (0)