diff --git a/CHANGELOG.md b/CHANGELOG.md
index 9a514f3a..e38f2349 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -2,6 +2,10 @@
## [Unreleased]
+### Added
+
+- Added support for caching of lookup joins.
+
### Fixed
- Fixed issue in the logging code of the `JavaNetHttpPollingClient` which prevents showing the status code and response body when the log level is configured at DEBUG (or lower) level.
diff --git a/README.md b/README.md
index 2fb28f5f..3b3ce57a 100644
--- a/README.md
+++ b/README.md
@@ -417,24 +417,31 @@ is set to `'true'`, it will be used as header value as is, without any extra mod
## Table API Connector Options
### HTTP TableLookup Source
-| Option | Required | Description/Value |
-|---------------------------------------------------------------|----------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| connector | required | The Value should be set to _rest-lookup_ |
-| format | required | Flink's format name that should be used to decode REST response, Use `json` for a typical REST endpoint. |
-| url | required | The base URL that should be use for GET requests. For example _http://localhost:8080/client_ |
-| asyncPolling | optional | true/false - determines whether Async Pooling should be used. Mechanism is based on Flink's Async I/O. |
-| lookup-method | optional | GET/POST/PUT (and any other) - determines what REST method should be used for lookup REST query. If not specified, `GET` method will be used. |
-| gid.connector.http.lookup.error.code | optional | List of HTTP status codes that should be treated as errors by HTTP Source, separated with comma. |
-| gid.connector.http.lookup.error.code.exclude | optional | List of HTTP status codes that should be excluded from the `gid.connector.http.lookup.error.code` list, separated with comma. |
-| gid.connector.http.security.cert.server | optional | Path to trusted HTTP server certificate that should be add to connectors key store. More than one path can be specified using `,` as path delimiter. |
-| gid.connector.http.security.cert.client | optional | Path to trusted certificate that should be used by connector's HTTP client for mTLS communication. |
-| gid.connector.http.security.key.client | optional | Path to trusted private key that should be used by connector's HTTP client for mTLS communication. |
-| gid.connector.http.security.cert.server.allowSelfSigned | optional | Accept untrusted certificates for TLS communication. |
-| gid.connector.http.source.lookup.request.timeout | optional | Sets HTTP request timeout in seconds. If not specified, the default value of 30 seconds will be used. |
-| gid.connector.http.source.lookup.request.thread-pool.size | optional | Sets the size of pool thread for HTTP lookup request processing. Increasing this value would mean that more concurrent requests can be processed in the same time. If not specified, the default value of 8 threads will be used. |
-| gid.connector.http.source.lookup.response.thread-pool.size | optional | Sets the size of pool thread for HTTP lookup response processing. Increasing this value would mean that more concurrent requests can be processed in the same time. If not specified, the default value of 4 threads will be used. |
-| gid.connector.http.source.lookup.use-raw-authorization-header | optional | If set to `'true'`, uses the raw value set for the `Authorization` header, without transformation for Basic Authentication (base64, addition of "Basic " prefix). If not specified, defaults to `'false'`. |
-| gid.connector.http.source.lookup.request-callback | optional | Specify which `HttpLookupPostRequestCallback` implementation to use. By default, it is set to `slf4j-lookup-logger` corresponding to `Slf4jHttpLookupPostRequestCallback`. |
+| Option | Required | Description/Value |
+|---------------------------------------------------------------|----------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| connector | required | The Value should be set to _rest-lookup_ |
+| format | required | Flink's format name that should be used to decode REST response, Use `json` for a typical REST endpoint. |
+| url | required | The base URL that should be use for GET requests. For example _http://localhost:8080/client_ |
+| asyncPolling | optional | true/false - determines whether Async Polling should be used. Mechanism is based on Flink's Async I/O. |
+| lookup-method | optional | GET/POST/PUT (and any other) - determines what REST method should be used for lookup REST query. If not specified, `GET` method will be used. |
+| lookup.cache | optional | Enum possible values: `NONE`, `PARTIAL`. The cache strategy for the lookup table. Currently supports `NONE` (no caching) and `PARTIAL` (caching entries on lookup operation in external API). |
+| lookup.partial-cache.max-rows | optional | The max number of rows of lookup cache, over this value, the oldest rows will be expired. `lookup.cache` must be set to `PARTIAL` to use this option. See the following Lookup Cache section for more details. |
+| lookup.partial-cache.expire-after-write | optional | The max time to live for each rows in lookup cache after writing into the cache. Specify as a [Duration](https://nightlies.apache.org/flink/flink-docs-release-1.19/docs/deployment/config/#duration). `lookup.cache` must be set to `PARTIAL` to use this option. See the following Lookup Cache section for more details. |
+| lookup.partial-cache.expire-after-access | optional | The max time to live for each rows in lookup cache after accessing the entry in the cache. Specify as a [Duration](https://nightlies.apache.org/flink/flink-docs-release-1.19/docs/deployment/config/#duration). `lookup.cache` must be set to `PARTIAL` to use this option. See the following Lookup Cache section for more details. |
+| lookup.partial-cache.cache-missing-key | optional | This is a boolean that defaults to true. Whether to store an empty value into the cache if the lookup key doesn't match any rows in the table. `lookup.cache` must be set to `PARTIAL` to use this option. See the following Lookup Cache section for more details. |
+| lookup.max-retries | optional | The max retry times if the lookup failed; default is 3. See the following Lookup Cache section for more details. |
+| gid.connector.http.lookup.error.code | optional | List of HTTP status codes that should be treated as errors by HTTP Source, separated with comma. |
+| gid.connector.http.lookup.error.code.exclude | optional | List of HTTP status codes that should be excluded from the `gid.connector.http.lookup.error.code` list, separated with comma. |
+| gid.connector.http.security.cert.server | optional | Path to trusted HTTP server certificate that should be add to connectors key store. More than one path can be specified using `,` as path delimiter. |
+| gid.connector.http.security.cert.client | optional | Path to trusted certificate that should be used by connector's HTTP client for mTLS communication. |
+| gid.connector.http.security.key.client | optional | Path to trusted private key that should be used by connector's HTTP client for mTLS communication. |
+| gid.connector.http.security.cert.server.allowSelfSigned | optional | Accept untrusted certificates for TLS communication. |
+| gid.connector.http.source.lookup.request.timeout | optional | Sets HTTP request timeout in seconds. If not specified, the default value of 30 seconds will be used. |
+| gid.connector.http.source.lookup.request.thread-pool.size | optional | Sets the size of pool thread for HTTP lookup request processing. Increasing this value would mean that more concurrent requests can be processed in the same time. If not specified, the default value of 8 threads will be used. |
+| gid.connector.http.source.lookup.response.thread-pool.size | optional | Sets the size of pool thread for HTTP lookup response processing. Increasing this value would mean that more concurrent requests can be processed in the same time. If not specified, the default value of 4 threads will be used. |
+| gid.connector.http.source.lookup.use-raw-authorization-header | optional | If set to `'true'`, uses the raw value set for the `Authorization` header, without transformation for Basic Authentication (base64, addition of "Basic " prefix). If not specified, defaults to `'false'`. |
+| gid.connector.http.source.lookup.request-callback | optional | Specify which `HttpLookupPostRequestCallback` implementation to use. By default, it is set to `slf4j-lookup-logger` corresponding to `Slf4jHttpLookupPostRequestCallback`. |
+
### HTTP Sink
| Option | Required | Description/Value |
@@ -460,6 +467,24 @@ is set to `'true'`, it will be used as header value as is, without any extra mod
| gid.connector.http.sink.writer.request.mode | optional | Sets Http Sink request submission mode. Two modes are available to select, `single` and `batch` which is the default mode if option is not specified. |
| gid.connector.http.sink.request.batch.size | optional | Applicable only for `gid.connector.http.sink.writer.request.mode = batch`. Sets number of individual events/requests that will be submitted as one HTTP request by HTTP sink. The default value is 500 which is same as HTTP Sink `maxBatchSize` |
+
+## Lookup Cache
+The HTTP Client connector can be used in lookup join as a lookup source (also known as a dimension table).
+
+By default, the lookup cache is not enabled. You can enable it by setting `lookup.cache` to `PARTIAL`.
+The scope of the cache is per job, so long-running jobs can benefit from this caching.
+
+The lookup cache is used to improve the performance of temporal joins. By default, the lookup cache is not enabled,
+so all the API requests are sent on the network. When the lookup cache is enabled, Flink looks in the cache first,
+and only sends requests on the network when there is no cached value, then the cache is updated with the returned rows.
+The oldest rows in this cache are expired when the cache hits the max cached rows `lookup.partial-cache.max-rows`
+or when the row exceeds the max time to live specified by `lookup.partial-cache.expire-after-write`
+or `lookup.partial-cache.expire-after-access`.
+
+By default, flink caches the empty query result for the primary key. You can toggle this behaviour by setting
+`lookup.partial-cache.cache-missing-key` to false.
+
+
## Build and deployment
To build the project locally you need to have `maven 3` and Java 11+.
@@ -545,7 +570,6 @@ The mapping from Http Json Response to SQL table schema is done via Flink's Json
## TODO
### HTTP TableLookup Source
-- Implement caches.
- Think about Retry Policy for Http Request
- Check other `//TODO`'s.
diff --git a/pom.xml b/pom.xml
index 4170de3e..bea9d81e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -68,6 +68,7 @@ under the License.
+
1.16.3
11
@@ -296,7 +297,6 @@ under the License.
maven-surefire-plugin
3.0.0-M5
-
diff --git a/src/main/java/com/getindata/connectors/http/internal/table/lookup/AsyncHttpTableLookupFunction.java b/src/main/java/com/getindata/connectors/http/internal/table/lookup/AsyncHttpTableLookupFunction.java
index 0dccc7d5..720186ad 100644
--- a/src/main/java/com/getindata/connectors/http/internal/table/lookup/AsyncHttpTableLookupFunction.java
+++ b/src/main/java/com/getindata/connectors/http/internal/table/lookup/AsyncHttpTableLookupFunction.java
@@ -1,8 +1,6 @@
package com.getindata.connectors.http.internal.table.lookup;
import java.util.Collection;
-import java.util.Collections;
-import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@@ -10,7 +8,7 @@
import lombok.RequiredArgsConstructor;
import lombok.extern.slf4j.Slf4j;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.functions.AsyncTableFunction;
+import org.apache.flink.table.functions.AsyncLookupFunction;
import org.apache.flink.table.functions.FunctionContext;
import org.apache.flink.util.concurrent.ExecutorThreadFactory;
@@ -19,7 +17,7 @@
@Slf4j
@RequiredArgsConstructor
-public class AsyncHttpTableLookupFunction extends AsyncTableFunction {
+public class AsyncHttpTableLookupFunction extends AsyncLookupFunction {
private static final String PULLING_THREAD_POOL_SIZE = "8";
@@ -73,29 +71,27 @@ public void open(FunctionContext context) throws Exception {
);
}
- public void eval(CompletableFuture> resultFuture, Object... keys) {
-
- CompletableFuture> future = new CompletableFuture<>();
- future.completeAsync(() -> decorate.lookupByKeys(keys), pullingThreadPool);
+ @Override
+ public CompletableFuture> asyncLookup(RowData keyRow) {
+ CompletableFuture> future = new CompletableFuture<>();
+ future.completeAsync(() -> decorate.lookup(keyRow), pullingThreadPool);
// We don't want to use ForkJoinPool at all. We are using a different thread pool
// for publishing here intentionally to avoid thread starvation.
+ CompletableFuture> resultFuture = new CompletableFuture<>();
future.whenCompleteAsync(
- (optionalResult, throwable) -> {
+ (result, throwable) -> {
if (throwable != null) {
log.error("Exception while processing Http Async request", throwable);
resultFuture.completeExceptionally(
new RuntimeException("Exception while processing Http Async request",
throwable));
} else {
- if (optionalResult.isPresent()) {
- resultFuture.complete(Collections.singleton(optionalResult.get()));
- } else {
- resultFuture.complete(Collections.emptyList());
- }
+ resultFuture.complete(result);
}
},
publishingThreadPool);
+ return resultFuture;
}
public LookupRow getLookupRow() {
diff --git a/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSource.java b/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSource.java
index 5e688ed5..c077f21c 100644
--- a/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSource.java
+++ b/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSource.java
@@ -2,6 +2,7 @@
import java.util.ArrayList;
import java.util.List;
+import javax.annotation.Nullable;
import lombok.extern.slf4j.Slf4j;
import org.apache.flink.api.common.serialization.DeserializationSchema;
@@ -10,15 +11,19 @@
import org.apache.flink.table.api.DataTypes.Field;
import org.apache.flink.table.connector.Projection;
import org.apache.flink.table.connector.format.DecodingFormat;
-import org.apache.flink.table.connector.source.AsyncTableFunctionProvider;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.LookupTableSource;
-import org.apache.flink.table.connector.source.TableFunctionProvider;
import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
+import org.apache.flink.table.connector.source.lookup.AsyncLookupFunctionProvider ;
+import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
+import org.apache.flink.table.connector.source.lookup.PartialCachingAsyncLookupProvider;
+import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.factories.DynamicTableFactory;
import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.functions.AsyncLookupFunction;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
@@ -46,17 +51,20 @@ public class HttpLookupTableSource
private final DynamicTableFactory.Context dynamicTableFactoryContext;
private final DecodingFormat> decodingFormat;
+ @Nullable
+ private final LookupCache cache;
public HttpLookupTableSource(
DataType physicalRowDataType,
HttpLookupConfig lookupConfig,
DecodingFormat> decodingFormat,
- DynamicTableFactory.Context dynamicTablecontext) {
-
+ DynamicTableFactory.Context dynamicTablecontext,
+ @Nullable LookupCache cache) {
this.physicalRowDataType = physicalRowDataType;
this.lookupConfig = lookupConfig;
this.decodingFormat = decodingFormat;
this.dynamicTableFactoryContext = dynamicTablecontext;
+ this.cache = cache;
}
@Override
@@ -66,6 +74,7 @@ public void applyProjection(int[][] projectedFields, DataType producedDataType)
@Override
public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext lookupContext) {
+ log.debug("getLookupRuntimeProvider Entry");
LookupRow lookupRow = extractLookupRow(lookupContext.getKeys());
@@ -94,21 +103,38 @@ public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext lookupContex
PollingClientFactory pollingClientFactory =
createPollingClientFactory(lookupQueryCreator, lookupConfig);
- HttpTableLookupFunction dataLookupFunction =
- new HttpTableLookupFunction(
- pollingClientFactory,
- responseSchemaDecoder,
- lookupRow,
- lookupConfig
- );
+ return getLookupRuntimeProvider(lookupRow, responseSchemaDecoder, pollingClientFactory);
+ }
+
+ protected LookupRuntimeProvider getLookupRuntimeProvider(LookupRow lookupRow,
+ DeserializationSchema responseSchemaDecoder,
+ PollingClientFactory pollingClientFactory) {
+ HttpTableLookupFunction dataLookupFunction =
+ new HttpTableLookupFunction(
+ pollingClientFactory,
+ responseSchemaDecoder,
+ lookupRow,
+ lookupConfig
+ );
if (lookupConfig.isUseAsync()) {
- log.info("Using Async version of HttpLookupTable.");
- return AsyncTableFunctionProvider.of(
- new AsyncHttpTableLookupFunction(dataLookupFunction));
+ AsyncLookupFunction asyncLookupFunction =
+ new AsyncHttpTableLookupFunction(dataLookupFunction);
+ if (cache != null) {
+ log.info("Using async version of HttpLookupTable with cache.");
+ return PartialCachingAsyncLookupProvider.of(asyncLookupFunction, cache);
+ } else {
+ log.info("Using async version of HttpLookupTable without cache.");
+ return AsyncLookupFunctionProvider.of(asyncLookupFunction);
+ }
} else {
- log.info("Using blocking version of HttpLookupTable.");
- return TableFunctionProvider.of(dataLookupFunction);
+ if (cache != null) {
+ log.info("Using blocking version of HttpLookupTable with cache.");
+ return PartialCachingLookupProvider.of(dataLookupFunction, cache);
+ } else {
+ log.info("Using blocking version of HttpLookupTable without cache.");
+ return LookupFunctionProvider.of(dataLookupFunction);
+ }
}
}
@@ -118,7 +144,8 @@ public DynamicTableSource copy() {
physicalRowDataType,
lookupConfig,
decodingFormat,
- dynamicTableFactoryContext
+ dynamicTableFactoryContext,
+ cache
);
}
diff --git a/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceFactory.java b/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceFactory.java
index 6822e91b..c9f8f8c2 100644
--- a/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceFactory.java
+++ b/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceFactory.java
@@ -5,6 +5,7 @@
import java.util.Set;
import java.util.function.Predicate;
import java.util.stream.Collectors;
+import javax.annotation.Nullable;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.configuration.ConfigOption;
@@ -15,6 +16,9 @@
import org.apache.flink.table.catalog.ResolvedSchema;
import org.apache.flink.table.connector.format.DecodingFormat;
import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.factories.DeserializationFormatFactory;
import org.apache.flink.table.factories.DynamicTableSourceFactory;
@@ -48,7 +52,7 @@ public DynamicTableSource createDynamicTableSource(Context dynamicTableContext)
FactoryUtil.TableFactoryHelper helper =
FactoryUtil.createTableFactoryHelper(this, dynamicTableContext);
- ReadableConfig readableConfig = helper.getOptions();
+ ReadableConfig readable = helper.getOptions();
helper.validateExcept(
// properties coming from org.apache.flink.table.api.config.ExecutionConfigOptions
"table.",
@@ -62,7 +66,7 @@ public DynamicTableSource createDynamicTableSource(Context dynamicTableContext)
FactoryUtil.FORMAT
);
- HttpLookupConfig lookupConfig = getHttpLookupOptions(dynamicTableContext, readableConfig);
+ HttpLookupConfig lookupConfig = getHttpLookupOptions(dynamicTableContext, readable);
ResolvedSchema resolvedSchema = dynamicTableContext.getCatalogTable().getResolvedSchema();
@@ -73,7 +77,8 @@ public DynamicTableSource createDynamicTableSource(Context dynamicTableContext)
physicalRowDataType,
lookupConfig,
decodingFormat,
- dynamicTableContext
+ dynamicTableContext,
+ getLookupCache(readable)
);
}
@@ -89,7 +94,18 @@ public Set> requiredOptions() {
@Override
public Set> optionalOptions() {
- return Set.of(URL_ARGS, ASYNC_POLLING, LOOKUP_METHOD, REQUEST_CALLBACK_IDENTIFIER);
+
+ return Set.of(
+ URL_ARGS,
+ ASYNC_POLLING,
+ LOOKUP_METHOD,
+ REQUEST_CALLBACK_IDENTIFIER,
+ LookupOptions.CACHE_TYPE,
+ LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_ACCESS,
+ LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_WRITE,
+ LookupOptions.PARTIAL_CACHE_MAX_ROWS,
+ LookupOptions.PARTIAL_CACHE_CACHE_MISSING_KEY,
+ LookupOptions.MAX_RETRIES);
}
private HttpLookupConfig getHttpLookupOptions(Context context, ReadableConfig readableConfig) {
@@ -115,6 +131,18 @@ private HttpLookupConfig getHttpLookupOptions(Context context, ReadableConfig re
.build();
}
+ @Nullable
+ private LookupCache getLookupCache(ReadableConfig tableOptions) {
+ LookupCache cache = null;
+ // Do not support legacy cache options
+ if (tableOptions
+ .get(LookupOptions.CACHE_TYPE)
+ .equals(LookupOptions.LookupCacheType.PARTIAL)) {
+ cache = DefaultLookupCache.fromConfig(tableOptions);
+ }
+ return cache;
+ }
+
// TODO verify this since we are on 1.15 now.
// Backport from Flink 1.15-Master
private DataType toRowDataType(List columns, Predicate columnPredicate) {
diff --git a/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpTableLookupFunction.java b/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpTableLookupFunction.java
index 655669cd..8579a02f 100644
--- a/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpTableLookupFunction.java
+++ b/src/main/java/com/getindata/connectors/http/internal/table/lookup/HttpTableLookupFunction.java
@@ -1,5 +1,7 @@
package com.getindata.connectors.http.internal.table.lookup;
+import java.util.Collection;
+import java.util.Collections;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicInteger;
@@ -8,17 +10,16 @@
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.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.functions.FunctionContext;
-import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.functions.LookupFunction;
import com.getindata.connectors.http.internal.PollingClient;
import com.getindata.connectors.http.internal.PollingClientFactory;
import com.getindata.connectors.http.internal.utils.SerializationSchemaUtils;
@Slf4j
-public class HttpTableLookupFunction extends TableFunction {
+public class HttpTableLookupFunction extends LookupFunction {
private final PollingClientFactory pollingClientFactory;
@@ -50,32 +51,22 @@ public HttpTableLookupFunction(
@Override
public void open(FunctionContext context) throws Exception {
- super.open(context);
-
this.responseSchemaDecoder.open(
SerializationSchemaUtils
.createDeserializationInitContext(HttpTableLookupFunction.class));
this.localHttpCallCounter = new AtomicInteger(0);
this.client = pollingClientFactory
- .createPollClient(options, responseSchemaDecoder);
-
- context
- .getMetricGroup()
- .gauge("http-table-lookup-call-counter", () -> localHttpCallCounter.intValue());
- }
+ .createPollClient(options, responseSchemaDecoder);
- /**
- * This is a lookup method which is called by Flink framework in a runtime.
- */
- public void eval(Object... keys) {
- lookupByKeys(keys)
- .ifPresent(this::collect);
+ context.getMetricGroup()
+ .gauge("http-table-lookup-call-counter", () -> localHttpCallCounter.intValue());
}
- public Optional lookupByKeys(Object[] keys) {
- RowData rowData = GenericRowData.of(keys);
+ @Override
+ public Collection lookup(RowData keyRow) {
localHttpCallCounter.incrementAndGet();
- return client.pull(rowData);
+ Optional result = client.pull(keyRow);
+ return result.map(Collections::singletonList).orElse(Collections.emptyList());
}
}
diff --git a/src/main/java/com/getindata/connectors/http/internal/table/lookup/JavaNetHttpPollingClient.java b/src/main/java/com/getindata/connectors/http/internal/table/lookup/JavaNetHttpPollingClient.java
index 44202341..ce3a31cc 100644
--- a/src/main/java/com/getindata/connectors/http/internal/table/lookup/JavaNetHttpPollingClient.java
+++ b/src/main/java/com/getindata/connectors/http/internal/table/lookup/JavaNetHttpPollingClient.java
@@ -66,6 +66,7 @@ public JavaNetHttpPollingClient(
@Override
public Optional pull(RowData lookupRow) {
try {
+ log.debug("Optional pull with Rowdata={}.", lookupRow);
return queryAndProcess(lookupRow);
} catch (Exception e) {
log.error("Exception during HTTP request.", e);
diff --git a/src/test/java/com/getindata/connectors/http/internal/table/lookup/AsyncHttpTableLookupFunctionTest.java b/src/test/java/com/getindata/connectors/http/internal/table/lookup/AsyncHttpTableLookupFunctionTest.java
index d66070bf..fb627480 100644
--- a/src/test/java/com/getindata/connectors/http/internal/table/lookup/AsyncHttpTableLookupFunctionTest.java
+++ b/src/test/java/com/getindata/connectors/http/internal/table/lookup/AsyncHttpTableLookupFunctionTest.java
@@ -5,7 +5,6 @@
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
-import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
@@ -13,8 +12,11 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
+import static java.util.Collections.emptyList;
+import static java.util.Collections.singletonList;
import lombok.extern.slf4j.Slf4j;
+import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.functions.FunctionContext;
import org.junit.jupiter.api.BeforeEach;
@@ -82,7 +84,7 @@ void shouldEvaluateInAsyncWay() throws InterruptedException {
assertThat(result.size()).isEqualTo(rowKeys.length);
assertThat(threadNames.size()).isEqualTo(rowKeys.length);
- verify(decorate, times(rowKeys.length)).lookupByKeys(any());
+ verify(decorate, times(rowKeys.length)).lookup(any());
}
@Test
@@ -111,7 +113,7 @@ void shouldHandleExceptionOnOneThread() throws InterruptedException {
throwable -> {
wasException.set(true);
latch.countDown();
- return Collections.emptyList();
+ return emptyList();
});
}
@@ -126,12 +128,12 @@ void shouldHandleExceptionOnOneThread() throws InterruptedException {
// -1 since one will have an exception
assertThat(result.size()).isEqualTo(rowKeys.length - 1);
assertThat(threadNames.size()).isEqualTo(rowKeys.length);
- verify(decorate, times(rowKeys.length)).lookupByKeys(any());
+ verify(decorate, times(rowKeys.length)).lookup(any());
}
@Test
- void shouldHandleEmptyOptionalResult() throws InterruptedException {
- mockPollingWithEmptyOptional();
+ void shouldHandleEmptyCollectionResult() throws InterruptedException {
+ mockPollingWithEmptyList();
final List result = Collections.synchronizedList(new ArrayList<>());
@@ -169,48 +171,48 @@ void shouldHandleEmptyOptionalResult() throws InterruptedException {
// -1 since one will have one empty result.
assertThat(result.size()).isEqualTo(rowKeys.length - 1);
assertThat(threadNames.size()).isEqualTo(rowKeys.length);
- verify(decorate, times(rowKeys.length)).lookupByKeys(any());
+ verify(decorate, times(rowKeys.length)).lookup(any());
}
private void mockPolling() {
- when(decorate.lookupByKeys(any()))
+ when(decorate.lookup(any()))
.thenAnswer(
invocationOnMock -> {
threadNames.add(Thread.currentThread().getName());
// make sure we pile up all keyRows on threads
barrier.await();
- return Optional.of(buildGenericRowData(
- Collections.singletonList(invocationOnMock.getArgument(0))));
+ return singletonList(buildGenericRowData(
+ singletonList(invocationOnMock.getArgument(0))));
});
}
private void mockPollingWithException() {
- when(decorate.lookupByKeys(any()))
+ when(decorate.lookup(any()))
.thenAnswer(
invocationOnMock -> {
threadNames.add(Thread.currentThread().getName());
// make sure we pile up all keyRows on threads
barrier.await();
- Integer argument = (Integer) ((Object[]) invocationOnMock.getArgument(0))[0];
+ Integer argument = ((GenericRowData) invocationOnMock.getArgument(0)).getInt(0);
if (argument == 12) {
throw new RuntimeException("Exception On problematic item");
}
- return Optional.of(buildGenericRowData(Collections.singletonList(argument)));
+ return singletonList(buildGenericRowData(singletonList(argument)));
});
}
- private void mockPollingWithEmptyOptional() {
- when(decorate.lookupByKeys(any()))
+ private void mockPollingWithEmptyList() {
+ when(decorate.lookup(any()))
.thenAnswer(
invocationOnMock -> {
threadNames.add(Thread.currentThread().getName());
// make sure we pile up all keyRows on threads
barrier.await();
- Integer argument = (Integer) ((Object[]) invocationOnMock.getArgument(0))[0];
+ Integer argument = ((GenericRowData) invocationOnMock.getArgument(0)).getInt(0);
if (argument == 12) {
- return Optional.empty();
+ return emptyList();
}
- return Optional.of(buildGenericRowData(Collections.singletonList(argument)));
+ return singletonList(buildGenericRowData(singletonList(argument)));
});
}
}
diff --git a/src/test/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceITCaseTest.java b/src/test/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceITCaseTest.java
index 1d7c43e8..3b2f0b29 100644
--- a/src/test/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceITCaseTest.java
+++ b/src/test/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceITCaseTest.java
@@ -4,6 +4,7 @@
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
+import java.util.Map;
import java.util.Objects;
import java.util.SortedSet;
import java.util.TreeSet;
@@ -24,6 +25,11 @@
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.TableResult;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.runtime.functions.table.lookup.LookupCacheManager;
+import org.apache.flink.table.test.lookup.cache.LookupCacheAssert;
import org.apache.flink.types.Row;
import org.apache.flink.util.CloseableIterator;
import org.apache.flink.util.StringUtils;
@@ -772,6 +778,67 @@ public void testNestedLookupJoinWithoutCast() throws Exception {
assertThat(collectedRows.size()).isEqualTo(5);
}
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void testHttpLookupJoinWithCache(boolean isAsync) throws Exception {
+ // GIVEN
+ LookupCacheManager.keepCacheOnRelease(true);
+
+ setupServerStub(wireMockServer);
+
+ String lookupTable =
+ "CREATE TABLE Customers ("
+ + "id STRING,"
+ + "id2 STRING,"
+ + "msg STRING,"
+ + "uuid STRING,"
+ + "details ROW<"
+ + "isActive BOOLEAN,"
+ + "nestedDetails ROW<"
+ + "balance STRING"
+ + ">"
+ + ">"
+ + ") WITH ("
+ + "'format' = 'json',"
+ + "'connector' = 'rest-lookup',"
+ + "'lookup-method' = 'GET',"
+ + "'url' = 'http://localhost:9090/client',"
+ + "'gid.connector.http.source.lookup.header.Content-Type' = 'application/json',"
+ + (isAsync ? "'asyncPolling' = 'true'," : "")
+ + "'lookup.cache' = 'partial',"
+ + "'lookup.partial-cache.max-rows' = '100'"
+ + ")";
+
+ // WHEN
+ SortedSet rows = testLookupJoin(lookupTable);
+
+ // THEN
+ try {
+ assertEnrichedRows(rows);
+
+ LookupCacheAssert.assertThat(getCache()).hasSize(4)
+ .containsKey(GenericRowData.of(
+ BinaryStringData.fromString("3"), BinaryStringData.fromString("4")))
+ .containsKey(GenericRowData.of(
+ BinaryStringData.fromString("4"), BinaryStringData.fromString("5")))
+ .containsKey(GenericRowData.of(
+ BinaryStringData.fromString("1"), BinaryStringData.fromString("2")))
+ .containsKey(GenericRowData.of(
+ BinaryStringData.fromString("2"), BinaryStringData.fromString("3")));
+ } finally {
+ LookupCacheManager.getInstance().checkAllReleased();
+ LookupCacheManager.getInstance().clear();
+ LookupCacheManager.keepCacheOnRelease(false);
+ }
+ }
+
+ private LookupCache getCache() {
+ Map managedCaches =
+ LookupCacheManager.getInstance().getManagedCaches();
+ assertThat(managedCaches).as("There should be only 1 shared cache registered").hasSize(1);
+ return managedCaches.get(managedCaches.keySet().iterator().next()).getCache();
+ }
+
private @NotNull SortedSet testLookupJoin(String lookupTable) throws Exception {
String sourceTable =
diff --git a/src/test/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceTest.java b/src/test/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceTest.java
index 2861491f..86e81e32 100644
--- a/src/test/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceTest.java
+++ b/src/test/java/com/getindata/connectors/http/internal/table/lookup/HttpLookupTableSourceTest.java
@@ -1,28 +1,41 @@
package com.getindata.connectors.http.internal.table.lookup;
import java.util.Arrays;
+import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.metrics.groups.CacheMetricGroup;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.catalog.Column;
import org.apache.flink.table.catalog.ResolvedSchema;
import org.apache.flink.table.catalog.UniqueConstraint;
-import org.apache.flink.table.connector.source.AsyncTableFunctionProvider;
-import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.lookup.AsyncLookupFunctionProvider;
+import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
+import org.apache.flink.table.connector.source.lookup.PartialCachingAsyncLookupProvider;
+import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.runtime.connector.source.LookupRuntimeProviderContext;
import org.apache.flink.table.types.DataType;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableList;
import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
import static com.getindata.connectors.http.internal.table.lookup.HttpLookupTableSourceFactory.row;
+
class HttpLookupTableSourceTest {
public static final DataType PHYSICAL_ROW_DATA_TYPE =
@@ -68,12 +81,10 @@ void shouldCreateTableSourceWithParams() {
HttpLookupTableSource tableSource =
(HttpLookupTableSource) createTableSource(SCHEMA, getOptions());
- TableFunctionProvider lookupProvider =
- (TableFunctionProvider)
- tableSource.getLookupRuntimeProvider(new LookupRuntimeProviderContext(lookupKey));
-
- HttpTableLookupFunction tableFunction =
- (HttpTableLookupFunction) lookupProvider.createTableFunction();
+ LookupTableSource.LookupRuntimeProvider lookupProvider =
+ tableSource.getLookupRuntimeProvider(new LookupRuntimeProviderContext(lookupKey));
+ HttpTableLookupFunction tableFunction = (HttpTableLookupFunction)
+ ((LookupFunctionProvider) lookupProvider).createLookupFunction();
LookupRow actualLookupRow = tableFunction.getLookupRow();
assertThat(actualLookupRow).isNotNull();
@@ -100,12 +111,13 @@ void shouldCreateAsyncTableSourceWithParams() {
HttpLookupTableSource tableSource =
(HttpLookupTableSource) createTableSource(SCHEMA, options);
- AsyncTableFunctionProvider lookupProvider =
- (AsyncTableFunctionProvider)
- tableSource.getLookupRuntimeProvider(new LookupRuntimeProviderContext(lookupKey));
+ AsyncLookupFunctionProvider lookupProvider =
+ (AsyncLookupFunctionProvider)
+ tableSource.getLookupRuntimeProvider(
+ new LookupRuntimeProviderContext(lookupKey));
AsyncHttpTableLookupFunction tableFunction =
- (AsyncHttpTableLookupFunction) lookupProvider.createAsyncTableFunction();
+ (AsyncHttpTableLookupFunction) lookupProvider.createAsyncLookupFunction();
LookupRow actualLookupRow = tableFunction.getLookupRow();
assertThat(actualLookupRow).isNotNull();
@@ -120,10 +132,111 @@ void shouldCreateAsyncTableSourceWithParams() {
actualLookupConfig.getReadableConfig().get(HttpLookupConnectorOptions.ASYNC_POLLING)
)
.withFailMessage(
- "Readable config probably was not passed from Table Factory or it is empty.")
+ "Readable config probably was not passed" +
+ " from Table Factory or it is empty.")
.isTrue();
}
+ @ParameterizedTest
+ @MethodSource("configProvider")
+ void testGetLookupRuntimeProvider(TestSpec testSpec) {
+ LookupCache cache = new LookupCache() {
+ @Override
+ public void open(CacheMetricGroup cacheMetricGroup) {
+
+ }
+
+ @Nullable
+ @Override
+ public Collection getIfPresent(RowData rowData) {
+ return null;
+ }
+
+ @Override
+ public Collection put(RowData rowData, Collection collection) {
+ return null;
+ }
+
+ @Override
+ public void invalidate(RowData rowData) {
+
+ }
+
+ @Override
+ public long size() {
+ return 0;
+ }
+
+ @Override
+ public void close() throws Exception {
+
+ }
+ };
+
+ HttpLookupConfig options = HttpLookupConfig.builder()
+ .useAsync(testSpec.isAsync)
+ .build();
+ LookupTableSource.LookupRuntimeProvider lookupRuntimeProvider =
+ getLookupRuntimeProvider(testSpec.hasCache ? cache : null, options);
+ assertTrue(testSpec.expected.isInstance(lookupRuntimeProvider));
+
+ }
+
+ private static class TestSpec {
+
+ boolean hasCache;
+ boolean isAsync;
+
+ Class expected;
+
+ private TestSpec(boolean hasCache,
+ boolean isAsync,
+ Class expected) {
+ this.hasCache = hasCache;
+ this.isAsync = isAsync;
+ this.expected = expected;
+ }
+
+ @Override
+ public String toString() {
+ return "TestSpec{"
+ + "hasCache="
+ + hasCache
+ + ", isAsync="
+ + isAsync
+ + ", expected="
+ + expected
+ + '}';
+ }
+ }
+
+ static Collection configProvider() {
+ return ImmutableList.builder()
+ .addAll(getTestSpecs())
+ .build();
+ }
+
+ @NotNull
+ private static ImmutableList getTestSpecs() {
+ return ImmutableList.of(
+ new TestSpec(false, false, LookupFunctionProvider.class),
+ new TestSpec(true, false, PartialCachingLookupProvider.class),
+ new TestSpec(false, true, AsyncLookupFunctionProvider.class),
+ new TestSpec(true, true, PartialCachingAsyncLookupProvider.class)
+ );
+ }
+
+ private static LookupTableSource.LookupRuntimeProvider
+ getLookupRuntimeProvider(LookupCache cache, HttpLookupConfig options) {
+ HttpLookupTableSource tableSource = new HttpLookupTableSource(
+ null, options,
+ null, null, cache);
+ int[][] lookupKeys = {{1, 2}};
+ LookupTableSource.LookupContext lookupContext =
+ new LookupRuntimeProviderContext(lookupKeys);
+ return tableSource.getLookupRuntimeProvider(null, null, null);
+ }
+
private Map getOptionsWithAsync() {
Map options = getOptions();
options = new HashMap<>(options);