ben-manes commented on a change in pull request #16130: URL: https://github.com/apache/beam/pull/16130#discussion_r765411720
########## File path: sdks/java/harness/src/main/java/org/apache/beam/fn/harness/Caches.java ########## @@ -0,0 +1,285 @@ +/* + * 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.beam.fn.harness; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.SdkHarnessOptions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors; +import org.cache2k.Cache2kBuilder; +import org.cache2k.operation.Weigher; +import org.github.jamm.MemoryMeter; + +/** Utility methods used to instantiate and operate over cache instances. */ +@SuppressWarnings("nullness") +public final class Caches { + + /** A cache that never stores any values. */ + public static <K, V> Cache<K, V> noop() { + return new SubCacheable<K, V>() { + @Override + public V peek(K key) { + return null; + } + + @Override + public V computeIfAbsent(K key, Function<K, V> loadingFunction) { + return loadingFunction.apply(key); + } + + @Override + public void put(K key, V value) {} + + @Override + public void clear() {} + + @Override + public void remove(K key) {} + + @Override + public Set<K> keys() { + return Collections.emptySet(); + } + }; + } + + /** + * Uses the specified {@link PipelineOptions} to configure and return a cache instance based upon + * parameters within {@link SdkHarnessOptions}. + */ + public static <K, V> Cache<K, V> fromOptions(PipelineOptions options) { + // We specifically use cache2k since it allows for recursive computeIfAbsent calls + // preventing deadlock from occurring when a loading function mutates the underlying cache + org.cache2k.Cache<Object, Object> cache = + Cache2kBuilder.forUnknownTypes() + .maximumWeight( + options.as(SdkHarnessOptions.class).getMaxCacheMemoryUsageMb() * 1024L * 1024L) + .weigher( + new Weigher<Object, Object>() { + private final MemoryMeter memoryMeter = MemoryMeter.builder().build(); + + @Override + public int weigh(Object key, Object value) { + long size = memoryMeter.measureDeep(key) + memoryMeter.measureDeep(value); + return size > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) size; + } + }) + .storeByReference(true) + .executor(MoreExecutors.directExecutor()) + .build(); + + return (Cache<K, V>) forCache(cache); + } + + /** + * Returns a view of a cache that operates on keys with a specified key prefix. + * + * <p>All lookups, insertions, and removals into the parent {@link Cache} will be prefixed by the + * specified prefixes. + * + * <p>Operations which operate over the entire caches contents such as {@link Cache#clear} only + * operate over keys with the specified prefixes. + */ + public static <K, V> Cache<K, V> subCache( + Cache<?, ?> cache, Object keyPrefix, Object... additionalKeyPrefix) { + if (cache instanceof SubCache) { + return new SubCache<>( + ((SubCache<?, ?>) cache).cache, + ((SubCache<?, ?>) cache).keyPrefix.subKey(keyPrefix, additionalKeyPrefix)); + } else if (cache instanceof SubCacheable) { + return new SubCache<>((SubCacheable<?, ?>) cache, CompositeKey.ROOT.subKey(keyPrefix)); + } + throw new UnsupportedOperationException("An unsupported type of cache was passed."); + } + + /** A cache that never evicts any values. */ + public static <K, V> Cache<K, V> eternal() { + // We specifically use cache2k since it allows for recursive computeIfAbsent calls Review comment: fwiw, Caffeine handles this through `AsyncCache` where you do not need a separate thread to execute on. See FAQ's [example](https://github.com/ben-manes/caffeine/wiki/Faq#recursive-computations), which is similar to what Apache Solr does in their usage. It is a simple workaround for these rare cases and makes it more explicit about what is happening, as often recursive computes are surprising and can be error prone. I am not sure if one can expect per-key linearizability with recursive writes, so the plus/minus of this behavior depends on the use-case. One very minor topic to be aware of is that the `Cache#computeIfAbsent` used here does **not** follow the specification for Map's. In addition, until recently the Map's implementation was broken and caused the cache to be in an invalid state. See this [bug report](https://github.com/cache2k/cache2k/issues/174) where I found these issues for the simple single threaded cases. Since these types of mistakes are sadly common and only ironed by usage, you might want to spend a little extra time validating the behavior and avoiding regressions. Caffeine has had its share of bugs and it takes usage to wrangle them out, so early adopters should be a little more proactive to help the library authors catch their oversights. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
