Skip to content

Commit

Permalink
Improve implementation
Browse files Browse the repository at this point in the history
 * Use LookupFunction and AsyncLookupFunction interfaces
 * Add integration tests
  • Loading branch information
Grzegorz Kołakowski committed Jul 29, 2024
1 parent c4acdb3 commit 24ae7d0
Show file tree
Hide file tree
Showing 10 changed files with 186 additions and 223 deletions.
6 changes: 4 additions & 2 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -16,8 +20,6 @@
[Slf4JHttpLookupPostRequestCallback](https://github.com/getindata/flink-http-connector/blob/main/src/main/java/com/getindata/connectors/http/internal/table/lookup/Slf4JHttpLookupPostRequestCallback.java)
is used instead.

- Added support for caching of synchronous lookup joins.

## [0.13.0] - 2024-04-03

### Added
Expand Down
33 changes: 17 additions & 16 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -424,12 +424,12 @@ is set to `'true'`, it will be used as header value as is, without any extra mod
| 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 <a href="#lookup-cache">Lookup Cache</a> 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 <a href="#lookup-cache">Lookup Cache</a> 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 <a href="#lookup-cache">Lookup Cache</a> 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 <a href="#lookup-cache">Lookup Cache</a> section for more details. |
| lookup.max-retries | optional | The max retry times if the lookup failed; default is 3. See the following <a href="#lookup-cache">Lookup Cache</a> section for more details. |
| 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 <a href="#lookup-cache">Lookup Cache</a> 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 <a href="#lookup-cache">Lookup Cache</a> 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 <a href="#lookup-cache">Lookup Cache</a> 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 <a href="#lookup-cache">Lookup Cache</a> section for more details. |
| lookup.max-retries | optional | The max retry times if the lookup failed; default is 3. See the following <a href="#lookup-cache">Lookup Cache</a> 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. |
Expand Down Expand Up @@ -469,18 +469,20 @@ is set to `'true'`, it will be used as header value as is, without any extra mod


## Lookup Cache
The HTTP Client connector can be used in temporal join as a lookup source (also known as a dimension table).
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`. Caching is only enabled if `asyncPolling` = false.
The scope of the cache is per job, so long running jobs can benefit from this caching.
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`.
The cached rows might not be the latest, but users can tune expiration options to a smaller value to have fresher data, but this may increase the number of API requests sent. So this is a balance between throughput and correctness.
A good use case for enabling this cache, is when the API responses are very slowly changing; for example master or reference data.
There are many cases when caching is not appropriate, for example calling an API to get the latest stock price.
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 a Primary key. You can toggle this behaviour by setting `lookup.partial-cache.cache-missing-key` to false.
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
Expand Down Expand Up @@ -568,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 for async.
- Think about Retry Policy for Http Request
- Check other `//TODO`'s.

Expand Down
Original file line number Diff line number Diff line change
@@ -1,16 +1,14 @@
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;

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;

Expand All @@ -19,7 +17,7 @@

@Slf4j
@RequiredArgsConstructor
public class AsyncHttpTableLookupFunction extends AsyncTableFunction<RowData> {
public class AsyncHttpTableLookupFunction extends AsyncLookupFunction {

private static final String PULLING_THREAD_POOL_SIZE = "8";

Expand Down Expand Up @@ -73,29 +71,27 @@ public void open(FunctionContext context) throws Exception {
);
}

public void eval(CompletableFuture<Collection<RowData>> resultFuture, Object... keys) {

CompletableFuture<Optional<RowData>> future = new CompletableFuture<>();
future.completeAsync(() -> decorate.lookupByKeys(keys), pullingThreadPool);
@Override
public CompletableFuture<Collection<RowData>> asyncLookup(RowData keyRow) {
CompletableFuture<Collection<RowData>> 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<Collection<RowData>> 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() {
Expand Down

This file was deleted.

Loading

0 comments on commit 24ae7d0

Please sign in to comment.