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/28 15:35:13 UTC

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

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


##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/cache/DefaultLookupCache.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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 Ticker ticker;
+
+    // For tracking cache metrics
+    private Counter hitCounter;
+    private 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) {

Review Comment:
   The `open()` method can be called concurrently and multiple times. So the implementation should be thread-safe, and we shouldn't initialize the cache again. Please add concurrent tests for `open()` and other methods. 



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/cache/DefaultLookupCache.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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 Ticker ticker;
+
+    // For tracking cache metrics
+    private Counter hitCounter;
+    private Counter missCounter;

Review Comment:
   Should be `transient`. Please add a serialization test for this. 



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/cache/DefaultLookupCache.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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 Ticker ticker;
+
+    // For tracking cache metrics
+    private Counter hitCounter;
+    private 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));
+    }

Review Comment:
   It is easy to lead OOM if we don't set the TTL and maxRows. I would suggest validating it has been set or provide a default value.



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/LookupOptions.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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 java.time.Duration;
+
+/** 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("The caching strategy for this lookup table");

Review Comment:
   Add description for the available values. 



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