ferenc-csaky commented on code in PR #12:
URL:
https://github.com/apache/flink-connector-http/pull/12#discussion_r2665617620
##########
flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/JavaNetHttpPollingClient.java:
##########
@@ -326,22 +327,49 @@ private Collection<RowData> deserialize(String
responseBody) throws IOException
}
}
- private List<RowData> deserializeSingleValue(byte[] rawBytes) throws
IOException {
- return Optional.ofNullable(responseBodyDecoder.deserialize(rawBytes))
- .map(Collections::singletonList)
- .orElse(Collections.emptyList());
+ @VisibleForTesting
+ List<RowData> deserializeSingleValue(byte[] rawBytes) throws IOException {
+ List<RowData> result = new ArrayList<>();
+ responseBodyDecoder.deserialize(rawBytes,
createRowDataCollector(result));
+ return result;
+ }
+
+ @VisibleForTesting
+ Collector<RowData> createRowDataCollector(List<RowData> result) {
+ return new RowDataCollector(result);
+ }
+
+ /** A simple collector implementation that adds RowData records to a list.
*/
+ @VisibleForTesting
+ static class RowDataCollector implements Collector<RowData> {
Review Comment:
There is a `ListCollector` which does exactly this, so we can get rid of
this class.
##########
flink-connector-http/src/main/java/org/apache/flink/connector/http/table/lookup/JavaNetHttpPollingClient.java:
##########
@@ -326,22 +327,49 @@ private Collection<RowData> deserialize(String
responseBody) throws IOException
}
}
- private List<RowData> deserializeSingleValue(byte[] rawBytes) throws
IOException {
- return Optional.ofNullable(responseBodyDecoder.deserialize(rawBytes))
- .map(Collections::singletonList)
- .orElse(Collections.emptyList());
+ @VisibleForTesting
+ List<RowData> deserializeSingleValue(byte[] rawBytes) throws IOException {
+ List<RowData> result = new ArrayList<>();
+ responseBodyDecoder.deserialize(rawBytes,
createRowDataCollector(result));
+ return result;
+ }
+
+ @VisibleForTesting
+ Collector<RowData> createRowDataCollector(List<RowData> result) {
+ return new RowDataCollector(result);
+ }
Review Comment:
Any specific reason for `createRowDataCollector`? I do not see why not call
the ctor directly in `deserializeSingleValue`:
```java
List<RowData> deserializeSingleValue(byte[] rawBytes) throws IOException
{
List<RowData> result = new ArrayList<>();
responseBodyDecoder.deserialize(rawBytes, new ListCollector(result));
return result;
}
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]