|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.beam.sdk.io.gcp.bigquery; |
| 19 | + |
| 20 | +import java.util.concurrent.Callable; |
| 21 | +import java.util.concurrent.ExecutorService; |
| 22 | +import java.util.concurrent.Executors; |
| 23 | +import java.util.concurrent.TimeUnit; |
| 24 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; |
| 25 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; |
| 26 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.RemovalNotification; |
| 27 | +import org.checkerframework.checker.nullness.qual.NonNull; |
| 28 | +import org.checkerframework.checker.nullness.qual.Nullable; |
| 29 | +import org.joda.time.Duration; |
| 30 | +import org.slf4j.Logger; |
| 31 | +import org.slf4j.LoggerFactory; |
| 32 | + |
| 33 | +/** |
| 34 | + * Encapsulates the cache of {@link AppendClientInfo} objects and the synchronization protocol |
| 35 | + * required to use them safely. The Guava cache object is thread-safe. However our protocol requires |
| 36 | + * that client pin the StreamAppendClient after looking up the cache, and we must ensure that the |
| 37 | + * cache is not accessed in between the lookup and the pin (any access of the cache could trigger |
| 38 | + * element expiration). |
| 39 | + */ |
| 40 | +class AppendClientCache<KeyT extends @NonNull Object> { |
| 41 | + private static final Logger LOG = LoggerFactory.getLogger(AppendClientCache.class); |
| 42 | + private final ExecutorService closeWriterExecutor = Executors.newCachedThreadPool(); |
| 43 | + |
| 44 | + private final Cache<KeyT, AppendClientInfo> appendCache; |
| 45 | + |
| 46 | + @SuppressWarnings({"FutureReturnValueIgnored"}) |
| 47 | + AppendClientCache(Duration expireAfterAccess) { |
| 48 | + this.appendCache = |
| 49 | + CacheBuilder.newBuilder() |
| 50 | + .expireAfterAccess(expireAfterAccess.getMillis(), TimeUnit.MILLISECONDS) |
| 51 | + .removalListener( |
| 52 | + (RemovalNotification<KeyT, AppendClientInfo> removal) -> { |
| 53 | + LOG.info("Expiring append client for {}", removal.getKey()); |
| 54 | + final @Nullable AppendClientInfo appendClientInfo = removal.getValue(); |
| 55 | + if (appendClientInfo != null) { |
| 56 | + // Remove the pin owned by the cache itself. Since the client has not been |
| 57 | + // marked as closed, we |
| 58 | + // can call unpin in this thread without worrying about blocking the thread. |
| 59 | + appendClientInfo.unpinAppendClient(null); |
| 60 | + // Close the client in another thread to avoid blocking the main thread. |
| 61 | + closeWriterExecutor.submit(appendClientInfo::close); |
| 62 | + } |
| 63 | + }) |
| 64 | + .build(); |
| 65 | + } |
| 66 | + |
| 67 | + // The cache itself always own one pin on the object. This Callable is always used to ensure that |
| 68 | + // the cache |
| 69 | + // adds a pin before loading a value. |
| 70 | + private static Callable<AppendClientInfo> wrapWithPin(Callable<AppendClientInfo> loader) { |
| 71 | + return () -> { |
| 72 | + AppendClientInfo client = loader.call(); |
| 73 | + client.pinAppendClient(); |
| 74 | + return client; |
| 75 | + }; |
| 76 | + } |
| 77 | + |
| 78 | + /** |
| 79 | + * Atomically get an append client from the cache and add a pin. This pin is owned by the client, |
| 80 | + * which has the responsibility of removing it. If the client is not in the cache, loader will be |
| 81 | + * used to load the client; in this case an additional pin will be added owned by the cache, |
| 82 | + * removed when the item is evicted. |
| 83 | + */ |
| 84 | + public AppendClientInfo getAndPin(KeyT key, Callable<AppendClientInfo> loader) throws Exception { |
| 85 | + synchronized (this) { |
| 86 | + AppendClientInfo info = appendCache.get(key, wrapWithPin(loader)); |
| 87 | + info.pinAppendClient(); |
| 88 | + return info; |
| 89 | + } |
| 90 | + } |
| 91 | + |
| 92 | + public AppendClientInfo get(KeyT key, Callable<AppendClientInfo> loader) throws Exception { |
| 93 | + return appendCache.get(key, wrapWithPin(loader)); |
| 94 | + } |
| 95 | + |
| 96 | + public AppendClientInfo putAndPin(KeyT key, Callable<AppendClientInfo> loader) throws Exception { |
| 97 | + synchronized (this) { |
| 98 | + AppendClientInfo info = wrapWithPin(loader).call(); |
| 99 | + appendCache.put(key, info); |
| 100 | + info.pinAppendClient(); |
| 101 | + return info; |
| 102 | + } |
| 103 | + } |
| 104 | + |
| 105 | + public AppendClientInfo put(KeyT key, Callable<AppendClientInfo> loader) throws Exception { |
| 106 | + AppendClientInfo info = wrapWithPin(loader).call(); |
| 107 | + appendCache.put(key, info); |
| 108 | + return info; |
| 109 | + } |
| 110 | + |
| 111 | + public void invalidate(KeyT key, AppendClientInfo expectedClient) { |
| 112 | + // The default stream is cached across multiple different DoFns. If they all try |
| 113 | + // and |
| 114 | + // invalidate, then we can get races between threads invalidating and recreating |
| 115 | + // streams. For this reason, |
| 116 | + // we check to see that the cache still contains the object we created before |
| 117 | + // invalidating (in case another |
| 118 | + // thread has already invalidated and recreated the stream). |
| 119 | + synchronized (this) { |
| 120 | + AppendClientInfo cachedAppendClient = appendCache.getIfPresent(key); |
| 121 | + if (cachedAppendClient != null |
| 122 | + && System.identityHashCode(cachedAppendClient) |
| 123 | + == System.identityHashCode(expectedClient)) { |
| 124 | + appendCache.invalidate(key); |
| 125 | + } |
| 126 | + } |
| 127 | + } |
| 128 | + |
| 129 | + public void invalidate(KeyT key) { |
| 130 | + synchronized (this) { |
| 131 | + appendCache.invalidate(key); |
| 132 | + } |
| 133 | + } |
| 134 | + |
| 135 | + public void tickle(KeyT key) { |
| 136 | + appendCache.getIfPresent(key); |
| 137 | + } |
| 138 | + |
| 139 | + public void clear() { |
| 140 | + synchronized (this) { |
| 141 | + appendCache.invalidateAll(); |
| 142 | + } |
| 143 | + } |
| 144 | + |
| 145 | + public void unpinAsync(AppendClientInfo appendClientInfo) { |
| 146 | + appendClientInfo.unpinAppendClient(closeWriterExecutor); |
| 147 | + } |
| 148 | +} |
0 commit comments