Status
...
...
...
JIRA: TBD
...
| Vote thread |
|
---|
JIRA | Jira |
---|
server | ASF JIRA |
---|
serverId | 5aa69414-a9e9-3523-82ec-879b028fb15b |
---|
key | FLINK-28415 |
---|
|
|
---|
Release | 1.16 |
---|
|
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
This FLIP is a joint work of initially proposed by Yuan Zhu (zstraw@163.com), and finished by Qingsheng Ren (partial caching part) and Alexander Smirnov (full caching part).
Motivation
...
In order to address the issues above, we propose here to define a unified abstraction for lookup source cache and its related metrics.
Proposed Changes
We'd like to split the proposal into two kinds of caching mode: partial caching and full caching.
Partial caching
Partial caching loads data into the cache along with the access to the external system. If the key to lookup does not exist in the cache, a lookup action to the external system will be triggered and the lookup result will be stored in the cache for further loading. Users and lookup table developers are able to config the eviction policy and maximum size of the cache.
In order to support partial caching, we propose to introduce several new interfaces to simplify the work for developers to implement lookup table functions and enable cache:
Top-level APIs
In order to clarify the semantic of lookup, we'd like to introduce some top-level APIs for general lookup operations without caching:
LookupFunction
/ AsyncLookupFunction
, an extended version of TableFunction to make the API more straight forward.LookupFunctionProvider
/ AsyncLookupProvider
, serve as the creator of LookupFunction / AsyncLookupFunction in table source
And APIs related to the cache:
LookupCache
, defining the cache used in lookup table.DefaultLookupCache
a default implementation of a cache that suitable for most use cases.CacheMetricGroup
, defining metrics should be reported by the cache.
Partial and Full Caching
More specifically, we'd like to provide public interfaces for the most 2 common cases to lookup source developers, which are named as partial and full caching.
Partial caching
Partial caching loads data into the cache along with the access to the external system. If the key to lookup does not exist in the cache, a lookup action to the external system will be triggered and the lookup result will be stored in the cache for further loading. Users and lookup table developers are able to config the eviction policy and maximum size of the cache.
In order to support partial caching, we propose to introduce 2 new interfaces:
PartialCachingLookupProvider
/ AsyncPartialCachingLookupProvider
LookupFunction
/ AsyncLookupFunction
, an extended version of TableFunction to clarify the semantic of lookup.LookupCache
/ LookupCacheFactory
, defining the cache and its factory used in lookup table.DefaultLookupCacheFactory
, a default implementation of a cache that suitable for most use cases.LookupCacheMetricGroup
, defining metrics should be reported by the lookup cache.LookupFunctionProvider
/ AsyncLookupFunctionProvider
, as the API interacting with table source to get LookupFunction and LookupCacheFactoryLookupCache.
The cache serves as a component in LookupJoinRunner, and would be pluggable by specifying LookupCacheFactory in LookupFunctionProvider. The developer of a lookup table needs to define a LookupFunctionProvider / AsyncLookupProvider in their implementation of LookupTableSource to specify the LookupFunction and the factory of the cache, then the planner will take over the cache factory, the constructor of the provider. The planner will take over the lookup function and the cache created from the provider and pass it to the LookupJoinRunner, and the . The cache will be instantiated during the runtime execution and loading operations via lookup function if there's a cache miss.
Full Caching
If the size of lookup table is relatively small to fit into the memory, and the lookup table doesn't change frequently, it'll be more efficient to load all entries of the lookup table into the cache to reduce network I/O, and refresh the table periodically. We'd like to name this use case as "full cache". Logically the reload operation is a kind of scan, so we'd like to reuse the ScanRuntimeProvider so that developers could reuse the scanning logic implemented in Source / SourceFunction / InputFormat. Considering the complexity of Source API, we'd like to support SourceFunction and InputFormat API first. Supporting Source API might require new topology and will be discussed later in another FLIP.
We propose to introduce a several new interface FullCachingLookupProvider
in order to reuse interfaces:
FullCachingLookupProvider
, for reusing the ability of scanning.
...
CacheReloadTrigger
, for customizing reloading strategies of all entries in the full cache.
Also we'd like to provide two default implementations of CacheReloadTrigger:
PeriodicCacheReloadTrigger
, for triggering reload periodically with a specific interval- TimedCacheReloadTrigger, for triggering reload at the specific time and repeat with the interval in days
...
Public Interfaces
Lookup Functions
As the usage of TableFunction interface is not quite straight forward to lookup table developers, we'd like to introduce a new interface for sync and async lookup tables. Caching will be only supported on LookupFunction / AsyncLookupFunction.
Code Block |
---|
language | java |
---|
title | LookupFunction |
---|
|
/**
* A wrapper class of {@link TableFunction} for synchronously lookup rows matching the lookup keys
* from external system.
*
* <p>The output type of this table function is fixed as {@link RowData}.
*/
@PublicEvolving
public abstract class LookupFunction extends TableFunction<RowData> {
/**
* Synchronously lookup rows matching the lookup keys.
*
* @param keyRow - A {@link RowData} that wraps keys to lookup.
* @return A collections of all matching rows in the lookup table.
*/
public abstract Collection<RowData> lookup(RowData keyRow) throws IOException;
/** Invoke {@link #lookup} and handle exceptions. */
public final void eval(Object... keys) {
try {
lookup(GenericRowData.of(keys)).forEach(this::collect);
} catch (IOException e) {
throw new RuntimeException("Failed to lookup values with given key", e);
}
}
} |
Code Block |
---|
language | java |
---|
title | AsyncLookupFunction |
---|
|
/**
* A wrapper class of {@link AsyncTableFunction} for asynchronously lookup rows matching the lookup
* keys from external system.
*
* <p>The output type of this table function is fixed as {@link RowData}.
*/
@PublicEvolving
public abstract class AsyncLookupFunction extends AsyncTableFunction<RowData> {
/**
* Asynchronously lookup rows matching the lookup keys.
*
* @param keyRow - A {@link RowData} that wraps keys to lookup.
* @return A collections of all matching rows in the lookup table.
*/
public abstract CompletableFuture<Collection<RowData>> asyncLookup(RowData keyRow);
/** Invokes {@link #asyncLookup} and chains futures. */
public final void eval(CompletableFuture<Collection<RowData>> future, Object... keys) {
asyncLookup(GenericRowData.of(keys))
.whenCompleteAsync(
(result, exception) -> {
if (exception != null) {
future.completeExceptionally(exception);
return;
}
future.complete(result);
});
}
} |
PartialCache
Considering there might be custom caching strategies and optimizations, we'd like to expose the cache interface as public API for developers to make the cache pluggable.
Code Block |
---|
language | java |
---|
title | PartialCache |
---|
|
/**
* A semi-persistent mapping from keys to values for storing entries of lookup table.
*
* <p>The type of the caching key is a {@link RowData} with lookup key fields packed inside. The
* type of value is a {@link Collection} of {@link RowData}, which are rows matching lookup key
* fields.
*
* <p>Cache entries are manually added using {@link #put}, and are stored in the cache until either
* evicted or manually invalidated.
*
* <p>Implementations of this interface are expected to be thread-safe, and can be safely accessed
* by multiple concurrent threads.
*/
@PublicEvolving
public interface PartialCache {
void open(LookupCacheMetricGroup metricGroup);
/**
* Returns the value associated with key in this cache, or null if there is no cached value for
* key.
*/
@Nullable
Collection<RowData> getIfPresent(RowData key);
/**
* Associates the specified value rows with the specified key row in the cache. If the cache
* previously contained value associated with the key, the old value is replaced by the
* specified value.
*
* @return the previous value rows associated with key, or null if there was no mapping for key.
* @param key - key row with which the specified value is to be associated
* @param value – value rows to be associated with the specified key
*/
Collection<RowData> put(RowData key, Collection<RowData> value);
/** Discards any cached value for the specified key. */
void invalidate(RowData key);
/** Returns the number of key-value mappings in the cache. */
long size();
void close();
} |
DefaultPartialCache
As the cache should be instantiated during runtime execution to avoid serialization / deserialization, a factory is required for creating the cache.
AsyncLookupFunction.
Code Block |
---|
language | java |
---|
title | LookupFunction |
---|
|
/**
* A wrapper class of {@link TableFunction} for synchronously lookup rows matching the lookup keys
* from external system.
*
* <p>The output type of this table function is fixed as {@link RowData}.
*/
@PublicEvolving
public abstract class LookupFunction extends TableFunction<RowData> {
/**
* Synchronously lookup rows matching the lookup keys.
*
* @param keyRow - A {@link RowData} that wraps keys to lookup.
* @return A collections of all matching rows in the lookup table.
*/
public abstract Collection<RowData> lookup(RowData keyRow) throws IOException;
/** Invoke {@link #lookup} and handle exceptions. */
public final void eval(Object... keys) {
try {
lookup(GenericRowData.of(keys)).forEach(this::collect);
} catch (IOException e) {
throw new RuntimeException("Failed to lookup values with given key", e);
}
}
} |
Code Block |
---|
language | java |
---|
title | AsyncLookupFunction |
---|
|
/**
* A wrapper class of {@link AsyncTableFunction} for asynchronously lookup rows matching the lookup
* keys from external system.
*
* <p>The output type of this table function is fixed as {@link RowData}.
*/
@PublicEvolving
public abstract class AsyncLookupFunction extends AsyncTableFunction<RowData> {
/**
* Asynchronously lookup rows matching the lookup keys.
*
* @param keyRow - A {@link RowData} that wraps keys to lookup.
* @return A collections of all matching rows in the lookup table.
*/
public abstract CompletableFuture<Collection<RowData>> asyncLookup(RowData keyRow);
/** Invokes {@link #asyncLookup} and chains futures. */
public final void eval(CompletableFuture<Collection<RowData>> future, Object... keys) {
asyncLookup(GenericRowData.of(keys))
.whenCompleteAsync(
|
Code Block |
---|
language | java |
---|
title | DefaultPartialCache |
---|
|
/** Factory for creating an instance of {@link LookupCache}. */
@PublicEvolving
public class DefaultPartialCache implements PartialCache {
private final Duration expireAfterAccessDuration;
private final Duration expireAfterWriteDuration;
private final Long maximumSize;
private DefaultPartialCache(
Duration expireAfterAccessDuration,
(result, exception) -> {
Duration expireAfterWriteDuration,
Long maximumSize) {
if (exception != null) {
this.expireAfterAccessDuration = expireAfterAccessDuration;
this.expireAfterWriteDuration = expireAfterWriteDuration;
this.initialCapacity = initialCapacity;
this.maximumSize = maximumSizefuture.completeExceptionally(exception);
}
// overide
public static Builder newBuilder() {
return new Builder();
}
public static class Builder {return;
private Duration expireAfterAccessDuration;
private Duration expireAfterWriteDuration;
private}
Long maximumSize;
public DefaultPartialCache.Builder expireAfterAccess(Duration duration) {
expireAfterAccessDuration = duration future.complete(result);
return this;
});
}
} |
LookupFunctionProvider
Code Block |
---|
language | java |
---|
title | LookupFunctionProvider |
---|
|
/** A provider for public DefaultPartialCache.Builder expireAfterWrite(Duration duration) {
creating {@link LookupFunction}. */
@PublicEvolving
public interface LookupFunctionProvider extends LookupTableSource.LookupRuntimeProvider {
static LookupFunctionProvider of(LookupFunction expireAfterWriteDuration = duration;lookupFunction) {
return () return this-> lookupFunction;
}
LookupFunction createLookupFunction();
} |
AsyncLookupFunctionProvider
Code Block |
---|
language | java |
---|
title | AsyncLookupFunctionProvider |
---|
|
/** A provider publicfor DefaultPartialCache.Builder maximumSize(long maximumSize) {
creating {@link AsyncLookupFunction}. */
@PublicEvolving
public interface AsyncLookupFunctionProvider extends LookupTableSource.LookupRuntimeProvider {
static AsyncLookupFunctionProvider this.maximumSize = maximumSize;of(AsyncLookupFunction asyncLookupFunction) {
return () return this-> asyncLookupFunction;
}
public DefaultPartialCache build() {
return new DefaultPartialCache(
expireAfterAccessDuration,
expireAfterWriteDuration,
maximumSize);
}
}
} |
LookupCacheMetricGroup
An interface defining all cache related metric:
AsyncLookupFunction createAsyncLookupFunction();
} |
LookupCache
Considering there might be custom caching strategies and optimizations, we'd like to expose the cache interface as public API for developers to make the cache pluggable.
Code Block |
---|
language | java |
---|
title | LookupCache |
---|
|
/**
* A semi-persistent mapping from keys to values for storing entries of lookup table.
*
* <p>The type of the caching key is a {@link RowData} with lookup key fields packed inside. The
* type of value is a {@link Collection} of {@link RowData}, which are rows matching lookup key
* fields.
*
* <p>Cache entries are manually added using {@link #put}, and are stored in the cache until either
* evicted or manually invalidated |
Code Block |
---|
language | java |
---|
title | LookupCacheMetricGroup |
---|
|
/**
* Pre-defined metrics for {@code LookupCache}.
*
* <p>Please note that these methods should only be invoked once. Registering a metric with same
* name for multiple times would lead to an undefined behavior.
*/
@PublicEvolving
public interface LookupCacheMetricGroup extends MetricGroup {
/** The number of cache hits. <p>Implementations of this interface are expected to be thread-safe, and can be safely accessed
* by multiple concurrent threads.
*/
@PublicEvolving
public interface LookupCache extends AutoClosable, Serializable {
/**
* Initialize the cache.
*
* @param metricGroup the metric group to register cache related metrics.
*/
void hitCounteropen(CounterCacheMetricGroup hitCountermetricGroup);
/**
/** The number of cache misses. */
void missCounter(Counter missCounter);
/** The number of times to load data into cache from external system. */ * Returns the value associated with key in this cache, or null if there is no cached value for
* key.
*/
@Nullable
voidCollection<RowData> loadCountergetIfPresent(CounterRowData loadCounterkey);
/**
The number of load failures. */
Associates the specified voidvalue numLoadFailuresCounter(Counter numLoadFailuresCounter);
/** The time spent for the latest load operation. */rows with the specified key row in the cache. If the cache
void latestLoadTimeGauge(Gauge<Long> latestLoadTimeGauge);
/** The number of records in cache. */
* previously contained value associated with the key, the old value is replaced by the
void* numCachedRecordsGauge(Gauge<Long> numCachedRecordsGauge);
specified value.
/ **
The number of bytes used by cache. */
void numCachedBytesGauge(Gauge<Long> numCachedBytesGauge);
} |
LookupFunctionProvider
This is the API between table framework and user's table source. Implementation should define how to create a lookup function and whether to use cache.
Code Block |
---|
language | java |
---|
title | LookupFunctionProvider |
---|
|
/**
* Provider for creating {@link LookupFunction} and {@link LookupCacheFactory} if caching should be
* enabled for the lookup table.
*/
@PublicEvolving
public interface LookupFunctionProvider extends LookupTableSource.LookupRuntimeProvider {
/** Creates a builder of {@link LookupFunctionProvider}. */
static Builder newBuilder() {
return new Builder();
}* @return the previous value rows associated with key, or null if there was no mapping for key.
* @param key - key row with which the specified value is to be associated
* @param value – value rows to be associated with the specified key
*/
Collection<RowData> put(RowData key, Collection<RowData> value);
/** Creates an {@link LookupFunction} instance Discards any cached value for the specified key. */
LookupFunctionvoid createLookupFunctioninvalidate(RowData key);
/**
Returns the number of key-value *mappings Getsin the {@link LookupCacheFactory} for creating lookup cache.
*/
long size();
} |
DefaultLookupCache
As the cache should be instantiated during runtime execution to avoid serialization / deserialization, a factory is required for creating the cache.
Code Block |
---|
language | java |
---|
title | DefaultLookupCache |
---|
|
/** Default implementation of {@link LookupCache}. */
@PublicEvolving
public class DefaultLookupCache implements LookupCache {
private final Duration expireAfterAccessDuration;
private final Duration expireAfterWriteDuration;
private final Long maximumSize;
private final boolean cacheMissingKey;
private DefaultLookupCache(
Duration expireAfterAccessDuration,
Duration expireAfterWriteDuration,
Long maximumSize,
boolean cacheMissingKey) {
this.expireAfterAccessDuration = expireAfterAccessDuration;
this.expireAfterWriteDuration = expireAfterWriteDuration;
this.initialCapacity = initialCapacity;
this.maximumSize = maximumSize;
this.cacheMissingKey = cacheMissingKey;
}
public static Builder newBuilder() {
return new Builder();
}
public static class Builder { * <p>This cache will be used during runtime execution for
* optimizing the access to external lookup table.
*
* @return an {@link Optional} of {@link LookupCache}, or an empty {@link Optional} if
* caching shouldn't be applies to the lookup table.
*/
Optional<PartialCache> getCache();
/**
* Whether the missing key (key fields without any matching value rows) should be stored in the
* cache.
*
* <p>Please note that this option is required if {@link #getCacheFactory()} returns a non-empty
* instance. If the cache factory is empty, the return value of this function will be ignored.
*
* @return true if a null or empty value should be stored in the cache.
*/
Optional<Boolean> cacheMissingKey();
/** Builder class for {@link LookupFunctionProvider}. */
class Builder {
private LookupFunctionDuration lookupFunctionexpireAfterAccessDuration;
private PartialCacheDuration partialCacheexpireAfterWriteDuration;
private BooleanLong enableCacheMissingKeymaximumSize;
/**private Sets lookup function. */Boolean cacheMissingKey;
public Builder withLookupFunctionexpireAfterAccess(LookupFunctionDuration lookupFunctionduration) {
this.lookupFunctionexpireAfterAccessDuration = lookupFunctionduration;
return this;
}
/** Enables caching and sets the cache factorEnables storing missing key in the cache. */
public Builder withCache(PartialCache partialCache , Boolean enableCacheMissingKey expireAfterWrite(Duration duration) {
this.partialCacheexpireAfterWriteDuration = partialCacheduration;
this.enableCacheMissingKey = true;
returnreturn this;
}
public LookupFunctionProviderBuilder buildmaximumSize(long maximumSize) {
// Build LookupFunctionProvider
}
}
} |
AsyncLookupFunctionProvider
Code Block |
---|
language | java |
---|
title | AsyncLookupFunctionProvider |
---|
|
@PublicEvolving
public interface AsyncLookupFunctionProvider extends LookupTableSource.LookupRuntimeProvider {
/** Creates a builder of {@link LookupFunctionProvider}. */
static Builder newBuilder() {this.maximumSize = maximumSize;
return this;
}
returnpublic newBuilder BuildercacheMissingKey(boolean cacheMissingKey); {
}
/** Creates an {@link AsyncLookupFunction} instance. */ this.cacheMissingKey = cacheMissingKey;
AsyncLookupFunction createAsyncLookupFunction();
return /**this;
* Gets the {@link} LookupCacheFactory} for creating lookup cache.
*
public DefaultLookupCache * <p>This factory will be used for creating an instance of cache during runtime execution forbuild() {
return new DefaultLookupCache(
* optimizing the access to external lookup table.
*
expireAfterAccessDuration,
* @return an {@link Optional} of {@link LookupCacheFactory}, or an empty {@link Optional} if
* expireAfterWriteDuration,
caching shouldn't be applies to the lookup table.
*/
Optional<LookupCacheFactory> getCacheFactory( maximumSize,
cacheMissingKey);
/**
}
* Whether the} missing key (key fields without any matching value rows) should be stored in the
* cache.
*
* <p>Please note that this option is required if {@link #getCacheFactory()} returns a non-empty
* instance. If the cache factory is empty, the return value of this function will be ignored.
*
* @return true if a null or empty value should be stored in the cache.
} |
CacheMetricGroup
An interface defining all cache related metric:
Code Block |
---|
language | java |
---|
title | CacheMetricGroup |
---|
|
/**
* Pre-defined metrics for cache.
*
* <p>Please note that these methods should only be invoked once. Registering a metric with same
* name for multiple times would lead to an undefined behavior.
*/
@PublicEvolving
public interface CacheMetricGroup extends MetricGroup {
/** The number of cache hits. */
void hitCounter(Counter hitCounter);
/** The number of cache misses. */
void missCounter(Counter missCounter);
/** The number of times to load data into cache from external system. */
Optional<Boolean>void cacheMissingKeyloadCounter(Counter loadCounter);
/** BuilderThe classnumber forof {@linkload AsyncLookupFunctionProvider}failures. */
classvoid Builder {numLoadFailuresCounter(Counter numLoadFailuresCounter);
/** The time private AsyncLookupFunction asyncLookupFunction;spent for the latest load operation. */
void latestLoadTimeGauge(Gauge<Long> latestLoadTimeGauge);
private LookupCacheFactory cacheFactory;
/** The number of records in cache. */
privatevoid Boolean enableCacheMissingKeynumCachedRecordsGauge(Gauge<Long> numCachedRecordsGauge);
/** The number of /**bytes Setsused lookupby functioncache. */
void numCachedBytesGauge(Gauge<Long> numCachedBytesGauge);
} |
PartialCachingLookupProvider
This is the API between table framework and user's table source. Implementation should define how to create a lookup function and whether to use cache.
Code Block |
---|
language | java |
---|
title | PartialCachingLookupProvider |
---|
|
/**
* Provider for creating {@link LookupFunction} and {@link LookupCache} for storing lookup entries.
*/
@PublicEvolving
public interface PartialCachingLookupProvider extends LookupFunctionProvider {
/**
* Build a {@link PartialCachingLookupProvider} from the specified {@link LookupFunction} and public Builder withAsyncLookupFunction(AsyncLookupFunction asyncLookupFunction) {
this.asyncLookupFunction = asyncLookupFunction;
return this;
}
/** Enables caching and sets the cache factory. */
* {@link LookupCache}.
public Builder withCacheFactory(LookupCacheFactory cacheFactory) {*/
static PartialCachingLookupProvider of(LookupFunction lookupFunction, LookupCache cache) {
this.cacheFactory = cacheFactory;
return new PartialCachingLookupProvider() {
return this;
}
@Override
/**
public LookupCache getCache() {
* Enables storing missing key in the cache.
return *cache;
* <p>See {@link AsyncLookupFunctionProvider#cacheMissingKey()}
for more details.
*/
@Override
public BuilderLookupFunction enableCacheMissingKeycreateLookupFunction() {
this.enableCacheMissingKey =return truelookupFunction;
}
return this;
};
}
/** Get a new instance of {@link LookupCache}. */
public AsyncLookupFunctionProvider build() {
// Build AsyncLookupFunctionProvider
}
}
} |
FullCachingLookupProvider
This interface is for supporting full cache strategy. It reuses ScanRuntimeProvider and defines reload time.
LookupCache getCache();
} |
PartialCachingAsyncLookupProvider
Code Block |
---|
language | java |
---|
title | PartialCachingAsyncLookupProvider |
---|
|
/**
* Provider for creating {@link AsyncLookupFunction} and {@link LookupCache} for storing lookup entries.
*/
@PublicEvolving
public interface PartialCachingAsyncLookupProvider extends AsyncLookupFunctionProvider {
/**
* Build a {@link PartialCachingLookupProvider} from the specified {@link AsyncLookupFunction} and
* {@link LookupCache}.
*/
static PartialCachingLookupProvider of(AsyncLookupFunction asyncLookupFunction, LookupCache cache) {
return new PartialCachingAsyncLookupProvider() {
@Override
public LookupCache getCache() {
return cache; |
Code Block |
---|
language | java |
---|
title | FullCachingLookupProvider |
---|
|
/**
* Runtime provider for fully loading and periodically reloading all entries of the lookup table and
* storing the table locally for lookup.
*
* <p>Implementations should provide a {@link ScanTableSource.ScanRuntimeProvider} in order to reuse
* the ability of scanning for loading all entries from the lookup table.
*/
@PublicEvolving
public interface FullCachingLookupProvider extends LookupTableSource.LookupRuntimeProvider {
/**
* Creates {@link FullCachingLookupProvider} with provided scan runtime provider and reload
* time.
*/
static FullCachingLookupProvider of(
ScanTableSource.ScanRuntimeProvider scanRuntimeProvider,}
ReloadStrategy reloadStrategy) {@Override
return new FullCachingLookupProvider() {
public AsyncLookupFunction createAsyncLookupFunction() {
@Override
return public ScanTableSource.ScanRuntimeProvider getScanRuntimeProvider() {
asyncLookupFunction;
}
return scanRuntimeProvider};
}
/** Get a new }
instance of {@link LookupCache}. */
@Override
public ReloadStrategy getReloadStrategy() {
return reloadStrategy;
}
};
}
/**
* GetsLookupCache getCache();
} |
FullCachingLookupProvider
This interface is for supporting full cache strategy. It reuses ScanRuntimeProvider and defines reload time.
Code Block |
---|
language | java |
---|
title | FullCachingLookupProvider |
---|
|
/**
* A {@link CachingLookupProvider} that never lookup in external system on cache miss and provides a
* cache for holding all entries in the external system. The cache will be fully reloaded from the
* external system by the {@link ScanTableSource.ScanRuntimeProvider} and reload operations will be
* triggered by the {@link org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider}
* for executing the periodically reload.
CacheReloadTrigger}.
*/
@PublicEvolving
public interface FullCachingLookupProvider extends LookupFunctionProvider {
static FullCachingLookupProvider of(
*/
ScanTableSource.ScanRuntimeProvider getScanRuntimeProvider(); scanRuntimeProvider,
CacheReloadTrigger cacheReloadTrigger) /**{
* Getsreturn thenew strategy when to reload cache. See {@link ReloadStrategy} and default implementation
* {@link FixedDelayReloadStrategy}.
FullCachingLookupProvider() {
@Override
*/
public ReloadStrategyScanTableSource.ScanRuntimeProvider getReloadStrategygetScanRuntimeProvider(); {
/** Defines strategy when to reload cache. */
interfacereturn ReloadStrategyscanRuntimeProvider;
extends Serializable {
}
/**
* Schedules task that@Override
reloads cache. This method is called just once, so implementation
public CacheReloadTrigger getCacheReloadTrigger() {
* must create the separate thread that will run {@code reloadTask} by itsreturn specificcacheReloadTrigger;
* triggers.
}
*/ @Override
void scheduleReload(Runnable reloadTask);
/** Stopspublic taskLookupFunction that reloads cache. */
createLookupFunction() {
void stopReloading();
}
} |
FixedDelayReloadStrategy
This class is default implementation of FullCachingLookupProvider.ReloadStrategy that schedules reloads with fixed delay, using ScheduledExecutorService#scheduleWithFixedDelay.
Code Block |
---|
language | java |
---|
title | FixedDelayReloadStrategy |
---|
|
/**
* Default {@link FullCachingLookupProvider.ReloadStrategy} that schedules reloads with fixed delay.
*/
public class FixedDelayReloadStrategy implements FullCachingLookupProvider.ReloadStrategy {
private final Duration reloadInterval return keyRow -> null;
}
};
@Nullable}
private final LocalTime reloadStartTime;/**
private ScheduledExecutorService executorService;
private FixedDelayReloadStrategy(Duration reloadInterval, @Nullable LocalTime reloadStartTime) {* Get a {@link ScanTableSource.ScanRuntimeProvider} for scanning all entries from the external
* lookup this.reloadInterval = reloadInterval;
table and load into the cache.
*/
thisScanTableSource.reloadStartTime = reloadStartTimeScanRuntimeProvider getScanRuntimeProvider();
}
@Override/** Get a {@link CacheReloadTrigger} for triggering the reload operation. */
publicCacheReloadTrigger void scheduleReload(Runnable reloadTask) {
long delay = reloadInterval.toMillis();
long initialDelay = 0;getCacheReloadTrigger();
} |
CacheReloadTrigger
A trigger defining custom logic for triggering full cache reloading.
Code Block |
---|
language | java |
---|
title | CacheReloadTrigger |
---|
|
/** Customized trigger for reloading all lookup table entries in full caching mode. */
@PublicEvolving
public interface CachingReloadTrigger extends AutoCloseable, Serializable {
/** Open the trigger. */
void open(Context context) throws if (reloadStartTime != null) {Exception;
/**
* Context of {@link CacheReloadTrigger} for getting LocalTimeinformation nowabout = LocalTime.now(ZoneOffset.UTC);times and
* triggering reload.
Duration*/
initialDelayDuration = Duration.between(now, reloadStartTime);interface Context {
/** Get current processing if (initialDelayDuration.isNegative()) {time. */
long currentProcessingTime();
// in case when reloadStartTime less than current time, reload will happen next day
** Get current watermark on the main stream. */
long currentWatermark();
/** Trigger a reload operation initialDelayDurationon = initialDelayDuration.plus(1, ChronoUnit.DAYS);the full cache. */
CompletableFuture<Void> triggerReload();
}
}
|
PeriodicCacheReloadTrigger
An implementation of FullCachingReloadTrigger that triggers reload with a specified interval.
Code Block |
---|
language | java |
---|
title | PeriodicCacheReloadTrigger |
---|
|
/** A trigger that reloads all entries periodically with specified interval or initialDelay = initialDelayDuration.toMillis();
delay. */
public class PeriodicCacheReloadTrigger implements CacheReloadTrigger {
}
private final Duration reloadInterval;
private executorServicefinal = Executors.newSingleThreadScheduledExecutor();ScheduleMode scheduleMode;
public PeriodicCacheReloadTrigger(Duration reloadInterval, executorService.scheduleWithFixedDelay(ScheduleMode scheduleMode) {
this.reloadInterval = reloadInterval;
reloadTask, initialDelay, delay, TimeUnit.MILLISECONDS) this.scheduleMode = scheduleMode;
}
@Override
public void stopReloadingopen(CacheReloadTrigger.Context context) {
// if (executorService != null) {Register periodic reload task
}
@Override
public void close() throws Exception {
executorService.shutdown();
// Dispose }resources
}
/**public Createsenum reload strategy with periodic intervals. */ScheduleMode {
public static FixedDelayReloadStrategy withInterval(Duration reloadInterval) {
FIXED_DELAY,
FIXED_RATE
return new FixedDelayReloadStrategy(reloadInterval, null);
}
/**
* Creates reload strategy with periodic intervals after initial delay up to }
} |
TimedCacheReloadTrigger
Code Block |
---|
language | java |
---|
title | TimedCacheReloadTrigger |
---|
|
/** A trigger that reloads at a specific local time and repeat for the given interval in days. */
public class TimedCacheReloadTrigger implements CacheReloadTrigger {
private final LocalTime reloadTime;
private final int reloadIntervalInDays;
{@code
* reloadStartTime}.
*/
public static FixedDelayReloadStrategy withIntervalAfterDelay(
TimedCacheReloadTrigger(LocalTime reloadTime, int reloadIntervalInDays) {
Duration reloadInterval, LocalTime reloadStartTime) {this.reloadTime = reloadTime;
returnthis.reloadIntervalInDays new FixedDelayReloadStrategy(reloadInterval, reloadStartTime)= reloadIntervalInDays;
}
/**@Override
Creates strategy with dailypublic reloadvoid at specified {@code reloadStartTime}. */open(Context context) {
// Register periodic reload task
public}
static FixedDelayReloadStrategy dailyAtSpecifiedTime(LocalTime reloadStartTime) { @Override
public void close() throws return new FixedDelayReloadStrategy(Duration.ofDays(1), reloadStartTime);Exception {
// Dispose resources
}
} |
TableFunctionProvider / AsyncTableFunctionProvider
...
In order to unify the usage of caching across all connectors, we'd like to introduce some common table options, which are defined under class LookupOptions
. Note that these options are not required to implement by all connectors.
Option | Type | Descriptions |
---|
lookup.cache | Enum of NONE, PARTIAL and FULL | The caching strategy for this lookup table. NONE: Do not use cache Partial: Use partial caching mode FULL: Use full caching mode |
lookup.async |
lookup.max-retries | Integer | The maximum allowed retries if a lookup operation fails |
lookup.partial-cache.expire-after-access | Duration | Duration to expire an entry in the cache after accessing |
lookup.partial-cache.expire-after-write | Duration | Duration to expire an entry in the cache after writing |
lookup.partial-cache.cache-missing-key | Boolean | Whether to |
use asynchronous mode for the lookup store an empty value into the cache if the lookup key doesn't match any rows in the table |
lookup.partial-cache.max- |
retriesInteger allowed retries if a lookup operation failsnumber of rows to store in the cache |
lookup. |
partialexpire-after-accessreload-strategy | Enum of PERIODIC and TIMED | The reload strategy for the full cache scenario. PERIODIC: Use PeriodicCacheReloadTrigger TIMED: Use TimedCacheReloadTrigger |
lookup.full-cache.periodic-reload.interval |
Duration | Duration to expire an entry in the cache after accessing | lookup.partial-cache.expire-after-write expire an entry cache after writingPeriodicCacheReloadTrigger |
lookup. |
partialcache-missing-keyBoolean | Whether to store an empty value into the cache if the lookup key doesn't match any rows in the table | lookup.partial-cache.max-rows | Long | The maximum number of rows to store in the cacheperiodic-reload.schedule-mode | Enum of FIXED_DELAY and FIXED_RATE | The periodically schedule mode of reloading in the PeriodicCacheReloadTrigger |
lookup.full-cache.timed-reload.iso- |
intervalDuration | Interval of reloading all entries from the lookup table into cacheString | Time in ISO-8601 format when cache needs to be reloaded. Time can be specified either with timezone or without timezone (target JVM local timezone will be used). See formatter ISO_TIME. |
lookup.full-cache.timed-reload.interval- |
starttimeString start time of the cache reload operation in UTCinterval in days to trigger the reload at the specified time |
Cache Metrics
It is important to mention that a cache implementation does not have to report all the defined metrics. But if a cache reports a metric of the same semantic defined below, the implementation should follow the convention.
...