You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/29 16:38:21 UTC

[GitHub] [flink] SmirAlex commented on a diff in pull request #20196: [FLINK-28417][table] Add interfaces for LookupCache and default implementation

SmirAlex commented on code in PR #20196:
URL: https://github.com/apache/flink/pull/20196#discussion_r933430045


##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/LookupOptions.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.connector.source.lookup;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.description.Description;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.description.TextElement.code;
+
+/** Predefined options for lookup cache. */
+public class LookupOptions {
+    public static final ConfigOption<LookupCacheType> CACHE_TYPE =
+            ConfigOptions.key("lookup.cache")
+                    .enumType(LookupCacheType.class)
+                    .defaultValue(LookupCacheType.NONE)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "The caching strategy for this lookup table, including %s, %s and %s",
+                                            code(LookupCacheType.NONE.toString()),
+                                            code(LookupCacheType.PARTIAL.toString()),
+                                            code(LookupCacheType.FULL.toString()))
+                                    .build());
+
+    public static final ConfigOption<Integer> MAX_RETRIES =
+            ConfigOptions.key("lookup.max-retries")
+                    .intType()
+                    .defaultValue(0)

Review Comment:
   Currently in HBase and JDBC connector default value is 3. Maybe leave this value like it was?



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/LookupOptions.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.connector.source.lookup;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.description.Description;
+
+import java.time.Duration;
+
+import static org.apache.flink.configuration.description.TextElement.code;
+
+/** Predefined options for lookup cache. */

Review Comment:
   Maybe change 'lookup cache' to 'lookup table' (option 'max-retries' is not related with caching)?



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/cache/DefaultLookupCache.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.connector.source.lookup.cache;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.ThreadSafeSimpleCounter;
+import org.apache.flink.metrics.groups.CacheMetricGroup;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.clock.Clock;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Ticker;
+import org.apache.flink.shaded.guava30.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava30.com.google.common.cache.CacheBuilder;
+
+import javax.annotation.Nullable;
+
+import java.time.Duration;
+import java.util.Collection;
+
+import static org.apache.flink.table.connector.source.lookup.LookupOptions.CACHE_TYPE;
+import static org.apache.flink.table.connector.source.lookup.LookupOptions.LookupCacheType.PARTIAL;
+import static org.apache.flink.table.connector.source.lookup.LookupOptions.PARTIAL_CACHE_CACHE_MISSING_KEY;
+import static org.apache.flink.table.connector.source.lookup.LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_ACCESS;
+import static org.apache.flink.table.connector.source.lookup.LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_WRITE;
+import static org.apache.flink.table.connector.source.lookup.LookupOptions.PARTIAL_CACHE_MAX_ROWS;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Default implementation of {@link LookupCache}. */
+@PublicEvolving
+public class DefaultLookupCache implements LookupCache {
+    private static final long serialVersionUID = 6839408984141411172L;
+
+    // Configurations of the cache
+    private final Duration expireAfterAccessDuration;
+    private final Duration expireAfterWriteDuration;
+    private final Long maximumSize;
+    private final boolean cacheMissingKey;
+
+    // The underlying Guava cache implementation
+    private transient Cache<RowData, Collection<RowData>> guavaCache;
+
+    // Guava Ticker for testing expiration
+    private transient Ticker ticker;
+
+    // For tracking cache metrics
+    private transient Counter hitCounter;
+    private transient Counter missCounter;
+
+    private DefaultLookupCache(
+            Duration expireAfterAccessDuration,
+            Duration expireAfterWriteDuration,
+            Long maximumSize,
+            boolean cacheMissingKey) {
+        this.expireAfterAccessDuration = expireAfterAccessDuration;
+        this.expireAfterWriteDuration = expireAfterWriteDuration;
+        this.maximumSize = maximumSize;
+        this.cacheMissingKey = cacheMissingKey;
+    }
+
+    /** Creates a builder for the cache. */
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    public static DefaultLookupCache fromConfig(ReadableConfig config) {
+        checkArgument(
+                config.get(CACHE_TYPE).equals(PARTIAL),
+                "'%s' should be '%s' in order to build a default lookup cache",
+                CACHE_TYPE.key(),
+                PARTIAL);
+        return new DefaultLookupCache(
+                config.get(PARTIAL_CACHE_EXPIRE_AFTER_ACCESS),
+                config.get(PARTIAL_CACHE_EXPIRE_AFTER_WRITE),
+                config.get(PARTIAL_CACHE_MAX_ROWS),
+                config.get(PARTIAL_CACHE_CACHE_MISSING_KEY));
+    }
+
+    @Override
+    public void open(CacheMetricGroup metricGroup) {
+        synchronized (this) {
+            // The cache should already been opened if guava cache is not null
+            if (guavaCache != null) {
+                return;
+            }
+            // Initialize Guava cache
+            CacheBuilder<Object, Object> guavaCacheBuilder = CacheBuilder.newBuilder();
+            if (expireAfterAccessDuration != null) {
+                guavaCacheBuilder.expireAfterAccess(expireAfterAccessDuration);
+            }
+            if (expireAfterWriteDuration != null) {
+                guavaCacheBuilder.expireAfterWrite(expireAfterWriteDuration);
+            }
+            if (maximumSize != null) {
+                guavaCacheBuilder.maximumSize(maximumSize);
+            }
+            if (ticker != null) {
+                guavaCacheBuilder.ticker(ticker);
+            }
+            guavaCache = guavaCacheBuilder.build();
+
+            // Initialize and register metrics
+            // Here we can't reuse Guava cache statistics because guavaCache#getIfPresent is not
+            // counted in the stat
+            hitCounter = new ThreadSafeSimpleCounter();
+            missCounter = new ThreadSafeSimpleCounter();
+            metricGroup.hitCounter(hitCounter);
+            metricGroup.missCounter(missCounter);
+            metricGroup.numCachedRecordsGauge(() -> guavaCache.size());
+        }
+    }
+
+    @Nullable
+    @Override
+    public Collection<RowData> getIfPresent(RowData key) {
+        Collection<RowData> value = guavaCache.getIfPresent(key);
+        if (value != null) {
+            hitCounter.inc();
+        } else {
+            missCounter.inc();
+        }
+        return value;
+    }
+
+    @Override
+    public Collection<RowData> put(RowData key, Collection<RowData> value) {
+        checkNotNull(key, "Cannot put an entry with null key into the cache");
+        checkNotNull(value, "Cannot put an entry with null value into the cache");
+        if (!value.isEmpty()) {

Review Comment:
   Maybe simplify this to `if (!value.isEmpty() || cacheMissingKey)`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org