Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Status

...

Page properties


Discussion thread

...

...

JIRA: TBD

...

Vote thread
JIRA

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyFLINK-28415

Release1.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).


Table of Contents

Motivation

As a widely-used feature in Flink SQL jobs, the performance of lookup table source is essential not only for users but also source developers for tuning their implementations. Most lookup table sources use cache to achieve better performance, but there are some features missing in the current design of cache:

...

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 strategies: LRU cache and all cache.

LRU 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 cachingLRU is the most common caching strategy, which dynamically evicts entries in the cache according to the given configuration. For supporting LRU cache in lookup table, we propose several new interfaces to simplify the work for developers to implement lookup table functions and enable cache:

  • 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, as the API interacting with table source to get LookupFunction and LookupCache.

The cache serves as a component in LookupJoinRunner, and would be pluggable by specifying in 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. 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

The LRU 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, pass it to the LookupJoinRunner, and the cache will be instantiated during the runtime execution.

All Cache

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 "all 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 new interface RescanRuntimeProvider in order to reuse several new 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
languagejava
titleLookupFunction
/**
 * 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
languagejava
titleAsyncLookupFunction
/**
 * 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) {
            asyncLookupasyncLookup(GenericRowData.of(keys))
                .whenCompleteAsync(
                        (result, exception) -> {
                            if (exception != null) {
                                future.completeExceptionally(exception);
                                return;
                            }
                            future.complete(result);
                        });
    }
}

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.

LookupFunctionProvider

Code Block
Code Block
languagejava
titleLookupCacheLookupFunctionProvider
/**
 * A semi-persistentprovider mappingfor fromcreating {@link LookupFunction}. */
@PublicEvolving
public interface LookupFunctionProvider extends LookupTableSource.LookupRuntimeProvider {

    static LookupFunctionProvider of(LookupFunction lookupFunction) {
        return () -> lookupFunction;
    }

    LookupFunction createLookupFunction();
}

AsyncLookupFunctionProvider

Code Block
languagejava
titleAsyncLookupFunctionProvider
/** A provider for creating {@link AsyncLookupFunction}. */
@PublicEvolving
public interface AsyncLookupFunctionProvider extends LookupTableSource.LookupRuntimeProvider {

    static AsyncLookupFunctionProvider of(AsyncLookupFunction asyncLookupFunction) {
        return () -> asyncLookupFunction;
    }

    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
languagejava
titleLookupCache
/**
 * 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 LookupCache extends AutoClosable, Serializable {

    /**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 LookupCache {

    /**
     * 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
     * specifiedInitialize the valuecache.
        *
     * @return@param metricGroup the previousmetric valuegroup rowsto associatedregister withcache key, or null if there was no mapping for keyrelated metrics.
     */
 @param key - key row with whichvoid open(CacheMetricGroup metricGroup);

    /**
     * Returns the specified value isassociated towith bekey associated
in this cache, or null *if there @paramis valueno cached value rowsfor
 to be associated with the specified* key.
     */
    @Nullable
    Collection<RowData> putgetIfPresent(RowData key, Collection<RowData> value);

    /**
 Discards any cached value for * Associates the specified value rows with the specified key. */
row in the cache. voidIf invalidate(RowData key);

the cache
     /** Returns the number of key-value mappings in the cache. */
    long size();
}

LookupCacheFactory

As the cache should be instantiated during runtime execution to avoid serialization / deserialization, a factory is required for creating the cache. 

Code Block
languagejava
titleLookupCacheFactory

/** Factory for creating an instance of {@link LookupCache}. */
@PublicEvolving
public interface LookupCacheFactory extends Serializable {

    /**
     * Create a {@link LookupCache}.
     ** 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 metricGroupvalue - Thevalue lookuprows cacheto metricbe groupassociated in whichwith the cachespecified registerkey
 predefined and
   */
  *  Collection<RowData> put(RowData key, custom metrics.Collection<RowData> value);

     /**/
 Discards any cached LookupCachevalue createCache(LookupCacheMetricGroup metricGroup);
}

DefaultLookupCacheFactory

In order to simplify the usage of developer, we provide a default factory for building a default cache. 

Code Block
languagejava
titleDefaultLookupCacheFactory
/** Factory for creating instance of {@link DefaultLookupCache}for the specified key. */
@PublicEvolving
public  class DefaultLookupCacheFactory implementsvoid LookupCacheFactory {invalidate(RowData key);

    private/** finalReturns Durationthe expireAfterAccessDuration;
number of key-value mappings privatein finalthe Duration expireAfterWriteDuration;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
languagejava
titleDefaultLookupCache
/** Default implementation of {@link LookupCache}. */
@PublicEvolving
public class DefaultLookupCache implements LookupCache {
    private final Duration expireAfterAccessDuration;
    private final Duration expireAfterWriteDurationprivate final Integer initialCapacity;
    private final Long maximumSize;

    public static DefaultLookupCacheFactory.Builder newBuilder() {
        return new Builder();
    }

private final Long maximumSize;
    private DefaultLookupCacheFactory(
final boolean cacheMissingKey;
    
      Duration expireAfterAccessDuration,private DefaultLookupCache(
            Duration expireAfterWriteDurationexpireAfterAccessDuration,
            IntegerDuration initialCapacityexpireAfterWriteDuration,
            Long maximumSize,
			boolean cacheMissingKey) {
        // Validation
        this.expireAfterAccessDuration this.expireAfterAccessDuration = expireAfterAccessDuration;
        this.expireAfterWriteDuration = expireAfterWriteDuration;
        this.initialCapacity = initialCapacity;
        this.maximumSize = maximumSize;
		this.cacheMissingKey = cacheMissingKey;
    }

    @Override
    public static publicBuilder LookupCache createCache(LookupCacheMetricGroup metricGroupnewBuilder() {
        //return Create instance of DefaultLookupCachenew Builder();
    } 

   public static /**class Builder of {@link DefaultLookupCacheFactory}. */
    public static class Builder {        
        private Duration expireAfterAccessDuration;
        private Duration expireAfterWriteDuration;
        private IntegerLong initialCapacitymaximumSize;
        private LongBoolean maximumSizecacheMissingKey;

        public DefaultLookupCacheFactory.Builder expireAfterAccess(Duration duration) {
            expireAfterAccessDuration = duration;
            return this;
        }

        public DefaultLookupCacheFactory.Builder expireAfterWrite(Duration duration) {
            expireAfterWriteDuration = duration;
            return this;
        }

        public DefaultLookupCacheFactory.Builder initialCapacitymaximumSize(intlong initialCapacitymaximumSize) {
            this.initialCapacitymaximumSize = initialCapacitymaximumSize;
            return this;
        }

        public DefaultLookupCacheFactory.Builder maximumSizecacheMissingKey(longboolean maximumSizecacheMissingKey) {
            this.maximumSizecacheMissingKey = maximumSizecacheMissingKey;
            return this;
        }          

        public DefaultLookupCacheFactoryDefaultLookupCache build() {
            return new DefaultLookupCacheFactoryDefaultLookupCache(
                    expireAfterAccessDuration,
                    expireAfterWriteDuration,
                    initialCapacity,
                    maximumSizemaximumSize,
					cacheMissingKey);
        }
    }     
}

...

CacheMetricGroup

An interface defining all cache related metric:

Code Block
languagejava
titleCacheMetricGroup
/**
 * Pre-defined metrics for {@code LookupCache}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 LookupCacheMetricGroupCacheMetricGroup 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. */
    void loadCounter(Counter loadCounter);

    /** The number of load failures. */
    void numLoadFailuresCounter(Counter numLoadFailuresCounter);

    /** The time spent for the latest load operation. */
    void latestLoadTimeGauge(Gauge<Long> latestLoadTimeGauge);

    /** The number of records in cache. */
    void numCachedRecordsGauge(Gauge<Long> numCachedRecordsGauge);

    /** The number of bytes used by cache. */
    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
languagejava
titleLookupFunctionProviderPartialCachingLookupProvider
/**
 * Provider for creating {@link LookupFunction} and {@link LookupCacheFactoryLookupCache} if caching should be
 * enabled for thestoring lookup tableentries.
 */
@PublicEvolving
public interface LookupFunctionProviderPartialCachingLookupProvider extends LookupTableSource.LookupRuntimeProviderLookupFunctionProvider {

    /**
       /** CreatesBuild a builder of{@link PartialCachingLookupProvider} from the specified {@link LookupFunctionProviderLookupFunction}. */and
    static Builder newBuilder() {* {@link LookupCache}.
     */
   return newstatic PartialCachingLookupProvider Builderof();
LookupFunction lookupFunction, LookupCache cache) }{

     /** Creates an {@linkreturn LookupFunction} instance. */new PartialCachingLookupProvider() {

    LookupFunction createLookupFunction();

    /**
   @Override
  * Gets the {@link LookupCacheFactory} for creating lookup cache.
  public LookupCache getCache() *{
     *  <p>This factory will be used for creating an instance ofreturn cache during;
  runtime execution for
     * optimizing the access}

 to external lookup table.
     *
   @Override
  * @return an {@link Optional} of {@link LookupCacheFactory}, or an emptypublic {@link Optional} ifLookupFunction createLookupFunction() {
     *     caching shouldn't be applies to the lookupreturn table.lookupFunction;
     */
    Optional<LookupCacheFactory> getCacheFactory();

  }
  /**
     * Whether};
  the missing key}

 (key fields without any/** matchingGet valuea rows)new shouldinstance beof stored in the{@link LookupCache}. */
    LookupCache * cache.
     *
     * <p>Please note that this option is required if {@link #getCacheFactory()} returns a non-emptygetCache();
}

PartialCachingAsyncLookupProvider

Code Block
languagejava
titlePartialCachingAsyncLookupProvider
/**
 * Provider for creating {@link AsyncLookupFunction} and {@link LookupCache} for storing lookup entries.
 */
@PublicEvolving
public interface PartialCachingAsyncLookupProvider extends AsyncLookupFunctionProvider {

    /**
     * instance. If the cache factory is empty, the return value of this function will be ignoredBuild a {@link PartialCachingLookupProvider} from the specified {@link AsyncLookupFunction} and
     * {@link LookupCache}.
     */
    static *PartialCachingLookupProvider @return true if a null or empty value should be stored in the cache.of(AsyncLookupFunction asyncLookupFunction, LookupCache cache) {
        return new PartialCachingAsyncLookupProvider() {

     */
      Optional<Boolean> cacheMissingKey();

    /** Builder class for {@link LookupFunctionProvider}. */
@Override
            public classLookupCache BuildergetCache() {

        private LookupFunction lookupFunction;
       return privatecache;
 LookupCacheFactory cacheFactory;
        private Boolean enableCacheMissingKey;}

        /** Sets lookup function. */
    @Override
            public BuilderAsyncLookupFunction withLookupFunctioncreateAsyncLookupFunction(LookupFunction lookupFunction) {
            this.lookupFunction = lookupFunction    return asyncLookupFunction;
            return this;}
        }
;
    }

    /** EnablesGet cachinga andnew setsinstance theof cache{@link factoryLookupCache}. */
    LookupCache getCache();
}

FullCachingLookupProvider

This interface is for supporting full cache strategy. It reuses ScanRuntimeProvider and defines reload time. 

Code Block
languagejava
titleFullCachingLookupProvider
/**
 * 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 CacheReloadTrigger}.
 */
@PublicEvolving
public interface FullCachingLookupProvider extends LookupFunctionProvider {
    static FullCachingLookupProvider of(
    public Builder withCacheFactory(LookupCacheFactory cacheFactory) {
            this.cacheFactory = cacheFactory;
            return this;
        }

        /**
         * Enables storing missing key in the cache.
         *
         * <p>See {@link LookupFunctionProvider#cacheMissingKey()} for more detailsScanTableSource.ScanRuntimeProvider scanRuntimeProvider,
         */
   CacheReloadTrigger cacheReloadTrigger) {
   public Builder enableCacheMissingKey() {
   return new FullCachingLookupProvider() {
            @Override
    this.enableCacheMissingKey = true;
        public ScanTableSource.ScanRuntimeProvider getScanRuntimeProvider() {
                return thisscanRuntimeProvider;
            }

            @Override
            public LookupFunctionProviderCacheReloadTrigger buildgetCacheReloadTrigger() {
            // Build LookupFunctionProvider    return cacheReloadTrigger;
        }
    }
}

AsyncLookupFunctionProvider

Code Block
languagejava
titleAsyncLookupFunctionProvider
@PublicEvolving
public interface AsyncLookupFunctionProvider extends LookupTableSource.LookupRuntimeProvider {


            /**@Override
     *    Creates {@link AsyncLookupFunctionProvider} withpublic theLookupFunction givencreateLookupFunction() {@link AsyncLookupFunction} and

          * disable lookup table caching.
  return keyRow -> */null;
    static AsyncLookupFunctionProvider of(AsyncLookupFunction asyncLookupFunction) {
    }
    return new AsyncLookupFunctionProvider() {  };
    }

    /**
    @Override
 * Get a {@link ScanTableSource.ScanRuntimeProvider} for scanning all entries from  public AsyncLookupFunction createAsyncLookupFunction() {the external
     * lookup table and load into the cache.
    return asyncLookupFunction;*/
    ScanTableSource.ScanRuntimeProvider getScanRuntimeProvider();

    /** Get a {@link CacheReloadTrigger}

 for triggering the reload operation. */
      @Override
            public Optional<LookupCacheFactory> getCacheFactory() {CacheReloadTrigger getCacheReloadTrigger();
}

CacheReloadTrigger

A trigger defining custom logic for triggering full cache reloading.

Code Block
languagejava
titleCacheReloadTrigger
/** 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)  return Optional.empty()throws Exception;

    /**
     * Context of {@link CacheReloadTrigger}

 for getting information about times and
     * triggering @Overridereload.
     */
    interface Context {

 public Optional<Boolean> cacheMissingKey() {
    /** Get current processing time. */
        returnlong Optional.emptycurrentProcessingTime();

        /** Get current watermark }
on the main stream. */
    };
    long }currentWatermark();

    /**
     /** Creates {@link AsyncLookupFunctionProvider} withTrigger a reload operation on the givenfull {@link AsyncLookupFunction} andcache. */
     * enable caching with specified {@link LookupCacheFactory}.CompletableFuture<Void> triggerReload();
    }
}

PeriodicCacheReloadTrigger

An implementation of FullCachingReloadTrigger that triggers reload with a specified interval.

Code Block
languagejava
titlePeriodicCacheReloadTrigger
/** A trigger that reloads all entries periodically with specified interval or delay. */
public class PeriodicCacheReloadTrigger implements CacheReloadTrigger {
 */
    static AsyncLookupFunctionProvider of(
            AsyncLookupFunction asyncLookupFunction,
            LookupCacheFactory cacheBuilder,
    private final Duration reloadInterval;
    private booleanfinal cacheMissingKey) {ScheduleMode scheduleMode;

    public PeriodicCacheReloadTrigger(Duration reloadInterval,  return new AsyncLookupFunctionProvider(ScheduleMode scheduleMode) {
        this.reloadInterval =   @OverridereloadInterval;
        this.scheduleMode = scheduleMode;
  public AsyncLookupFunction createAsyncLookupFunction() { }

    @Override
    public void open(CacheReloadTrigger.Context context) {
    return asyncLookupFunction;
   // Register periodic reload task
     }

    @Override
    public void close() throws Exception @Override{
        // Dispose resources
  public Optional<LookupCacheFactory> getCacheFactory() { }

    public enum ScheduleMode {
         return Optional.of(cacheBuilder);FIXED_DELAY,
        FIXED_RATE
    }

            @Override
            public Optional<Boolean> cacheMissingKey() {
                return Optional.of(cacheMissingKey);}

TimedCacheReloadTrigger

Code Block
languagejava
titleTimedCacheReloadTrigger
/** 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;

    public TimedCacheReloadTrigger(LocalTime reloadTime, int reloadIntervalInDays) {
        this.reloadTime = reloadTime;
  }
      this.reloadIntervalInDays = }reloadIntervalInDays;
    }

    /** Creates an {@link AsyncLookupFunction} instance. */
    AsyncLookupFunction createAsyncLookupFunction();

    /**@Override
    public void open(Context context) {
		// Register periodic reload task
    }

    @Override
 * Gets the {@linkpublic LookupCacheFactory} for creating lookup cache.
     *void close() throws Exception {
		// Dispose resources
     * <p>This factory will be used for creating an instance of cache during runtime execution for
     * optimizing the access to external lookup table.
     *
     * @return an {@link Optional} of {@link LookupCacheFactory}, or an empty {@link Optional} if
     *     caching shouldn't be applies to the lookup table.
     */
    Optional<LookupCacheFactory> getCacheFactory();

    /**
     * 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 AsyncLookupFunctionProvider}. */
    class Builder {

        private AsyncLookupFunction asyncLookupFunction;
        private LookupCacheFactory cacheFactory;
        private Boolean enableCacheMissingKey;

        /** Sets lookup function. */
        public Builder withAsyncLookupFunction(AsyncLookupFunction asyncLookupFunction) {
            this.asyncLookupFunction = asyncLookupFunction;
            return this;
        }

        /** Enables caching and sets the cache factory. */
        public Builder withCacheFactory(LookupCacheFactory cacheFactory) {
            this.cacheFactory = cacheFactory;
            return this;
        }

        /**
         * Enables storing missing key in the cache.
         *
         * <p>See {@link AsyncLookupFunctionProvider#cacheMissingKey()} for more details.
         */
        public Builder enableCacheMissingKey() {
            this.enableCacheMissingKey = true;
            return this;
        }

        public AsyncLookupFunctionProvider build() {
            // Build AsyncLookupFunctionProvider
        }
    }
}

RescanRuntimeProvider

This interface is for supporting all cache strategy. It reuses ScanRuntimeProvider and defines interval of re-scan. 

Code Block
languagejava
titleRescanRuntimeProvider
/**
 * Runtime provider for periodically re-scanning all entries of the lookup table and storing the
 * table locally for lookup.
 */
@PublicEvolving
public interface RescanRuntimeProvider extends LookupTableSource.LookupRuntimeProvider {

    /**
     * Gets the {@link org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider}
     * for executing the periodically re-scan.
     */
    ScanTableSource.ScanRuntimeProvider getScanRuntimeProvider();

    /** Gets the interval between two re-scans. */
    Duration getRescanInterval();
}

TableFunctionProvider / AsyncTableFunctionProvider

...

}
}

TableFunctionProvider / AsyncTableFunctionProvider

We'd like to deprecate these two interfaces and let developers to switch to the new LookupFunctionProvider / AsyncLookupFunctionProvider / FullCachingLookupProvider instead.

Table Options for Lookup Cache

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. 

OptionTypeDescriptions
lookup.cacheEnum 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.max-retriesIntegerThe maximum allowed retries if a lookup operation fails
lookup.partial-cache.expire-after-accessDurationDuration to expire an entry in the cache after accessing
lookup.partial-cache.expire-after-writeDurationDuration to expire an entry in the cache after writing
lookup.partial-cache.cache-missing-keyBooleanWhether to store an empty value into the cache if the lookup key doesn't match any rows in the table
lookup.partial-cache.max-rowsLongThe maximum number of rows to store in the cache
lookup.full-cache.reload-strategyEnum of PERIODIC and TIMED

The reload strategy for the full cache scenario.

PERIODIC: Use PeriodicCacheReloadTrigger

TIMED: Use TimedCacheReloadTrigger

lookup.full-cache.periodic-reload.intervalDurationDuration to trigger reload in the PeriodicCacheReloadTrigger
lookup.full-cache.periodic-reload.schedule-modeEnum of FIXED_DELAY and FIXED_RATEThe periodically schedule mode of reloading in the PeriodicCacheReloadTrigger
lookup.full-cache.timed-reload.iso-timeString

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-in-daysInteger

The interval 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.

NameTypeUnitDescription
numCachedRecordGaugeRecordsThe number of records in cache.
numCachedBytesGaugeBytesThe number of bytes used by cache.
hitCountCounter
The number of cache hits
missCountCounter
The number of cache misses, which might leads to loading operations
loadCountCounter

The number of times to load data into cache from external system.

For

LRU

partial cache the load count should be equal to miss count, but for all cache this would be different.

numLoadFailureCounter
The number of load failures
latestLoadTimeGaugemsThe time spent for the latest load operation

Here we just define fundamental metrics and let the external metric system make the aggregation to get more descriptive values such as hitRate = hitCount / (hitCount + missCount).

Scope

The metric group for the cache would be a sub-group of the OperatorMetricGroup where the table function belongs to.

Future Works

In order to reduce network I/O with external systems and the usage of cache further, some optimizations implemented on scan source could be also applied on the lookup table, such as projection and filter pushdown. These features will be introduced separately in another FLIP. 

Compatibility, Deprecation, and Migration Plan

Currently we have JDBC, Hive and HBase connector implemented lookup table source. All existing implementations will be migrated to the current design and the migration will be transparent to end users. Table options related to caching defined by these connectors will be migrated to new table options defined in this FLIP above

Test Plan

We will use unit and integration test for validating the functionality of cache implementations.

Rejected Alternatives

Add cache in TableFunction implementations

Compared with this design, adding cache in TableFunction implementations might lead to inconsistency between sync and async table function, and not suitable for applying optimizations. 

...