From 6aa71e22e85e5681f22bb75c0b761006f8803535 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 27 Sep 2017 12:23:23 -0500 Subject: [PATCH 1/4] STORM-2759: Let users indicate if a blob should restart a worker --- log4j2/cluster.xml | 6 +- storm-client/pom.xml | 2 +- .../src/jvm/org/apache/storm/Config.java | 9 +- .../org/apache/storm/utils/CuratorUtils.java | 2 +- .../org/apache/storm/utils/NimbusClient.java | 46 +- .../src/jvm/org/apache/storm/utils/Utils.java | 31 +- .../org/apache/storm/testing4j_test.clj | 71 -- storm-server/pom.xml | 2 +- .../storm/daemon/supervisor/Supervisor.java | 2 +- .../daemon/supervisor/SupervisorUtils.java | 16 +- .../storm/localizer/AsyncLocalizer.java | 893 +++++------------- .../localizer/LocalDownloadedResource.java | 86 -- .../apache/storm/localizer/LocalResource.java | 47 +- .../storm/localizer/LocalizedResource.java | 533 ++++++++--- .../LocalizedResourceRetentionSet.java | 333 ++----- .../storm/localizer/LocalizedResourceSet.java | 101 -- .../storm/localizer/LocallyCachedBlob.java | 105 +- .../localizer/LocallyCachedTopologyBlob.java | 58 +- .../storm/localizer/PortAndAssignment.java | 8 +- .../org/apache/storm/utils/ServerUtils.java | 19 - .../java/org/apache/storm/TestingTest.java | 139 +++ .../storm/localizer/AsyncLocalizerTest.java | 753 ++++++++------- .../LocalizedResourceRetentionSetTest.java | 133 +-- .../localizer/LocalizedResourceSetTest.java | 74 -- 24 files changed, 1563 insertions(+), 1906 deletions(-) delete mode 100644 storm-server/src/main/java/org/apache/storm/localizer/LocalDownloadedResource.java delete mode 100644 storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceSet.java create mode 100644 storm-server/src/test/java/org/apache/storm/TestingTest.java delete mode 100644 storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceSetTest.java diff --git a/log4j2/cluster.xml b/log4j2/cluster.xml index 35ef2576091..2a12fbac478 100644 --- a/log4j2/cluster.xml +++ b/log4j2/cluster.xml @@ -21,7 +21,7 @@ %d{yyyy-MM-dd HH:mm:ss.SSS} %c{1.} %t [%p] %msg%n - @@ -32,7 +32,7 @@ - @@ -43,7 +43,7 @@ - diff --git a/storm-client/pom.xml b/storm-client/pom.xml index b37f458797c..6ecb04da17d 100644 --- a/storm-client/pom.xml +++ b/storm-client/pom.xml @@ -251,7 +251,7 @@ **/generated/** - 10785 + 10473 diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java index 6be0c216729..a73e7914875 100644 --- a/storm-client/src/jvm/org/apache/storm/Config.java +++ b/storm-client/src/jvm/org/apache/storm/Config.java @@ -91,11 +91,12 @@ public class Config extends HashMap { /** * A map with blobstore keys mapped to each filename the worker will have access to in the - * launch directory to the blob by local file name and uncompress flag. Both localname and - * uncompress flag are optional. It uses the key is localname is not specified. Each topology - * will have different map of blobs. Example: topology.blobstore.map: {"blobstorekey" : + * launch directory to the blob by local file name, uncompress flag, and if the worker + * should restart when the blob is updated. localname, workerRestart, and + * uncompress are optional. If localname is not specified the name of the key is used instead. + * Each topologywill have different map of blobs. Example: topology.blobstore.map: {"blobstorekey" : * {"localname": "myblob", "uncompress": false}, "blobstorearchivekey" : - * {"localname": "myarchive", "uncompress": true}} + * {"localname": "myarchive", "uncompress": true, "workerRestart": true}} */ @CustomValidator(validatorClass = MapOfStringToMapOfStringToObjectValidator.class) public static final String TOPOLOGY_BLOBSTORE_MAP = "topology.blobstore.map"; diff --git a/storm-client/src/jvm/org/apache/storm/utils/CuratorUtils.java b/storm-client/src/jvm/org/apache/storm/utils/CuratorUtils.java index f36003ddf20..a3c255868fe 100644 --- a/storm-client/src/jvm/org/apache/storm/utils/CuratorUtils.java +++ b/storm-client/src/jvm/org/apache/storm/utils/CuratorUtils.java @@ -45,7 +45,7 @@ public static CuratorFramework newCurator(Map conf, List } public static CuratorFramework newCurator(Map conf, List servers, Object port, String root, ZookeeperAuthInfo auth) { - List serverPorts = new ArrayList(); + List serverPorts = new ArrayList<>(); for (String zkServer : servers) { serverPorts.add(zkServer + ":" + ObjectReader.getInt(port)); } diff --git a/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java b/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java index c6194002359..8589d65f9de 100644 --- a/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java +++ b/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java @@ -15,24 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.storm.utils; +import com.google.common.collect.Lists; +import java.security.Principal; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.storm.Config; import org.apache.storm.generated.Nimbus; import org.apache.storm.generated.NimbusSummary; import org.apache.storm.security.auth.ReqContext; import org.apache.storm.security.auth.ThriftClient; import org.apache.storm.security.auth.ThriftConnectionType; -import com.google.common.collect.Lists; import org.apache.thrift.transport.TTransportException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.security.Principal; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - public class NimbusClient extends ThriftClient { private static volatile Nimbus.Iface _localOverrideClient = null; @@ -49,7 +49,7 @@ public void close() throws Exception { /** * @return true of new clients will be overridden to connect to a local cluster - * and not the configured remote cluster. + * and not the configured remote cluster. */ public static boolean isLocalOverride() { return _localOverrideClient != null; @@ -86,7 +86,18 @@ public static NimbusClient getConfiguredClient(Map conf) { public static NimbusClient getConfiguredClient(Map conf, Integer timeout) { return getConfiguredClientAs(conf, null, timeout); } - + + private static String oldLeader = ""; + + private static synchronized boolean shouldLogLeader(String leader) { + //Only log if the leader has changed. It is not interesting otherwise. + if (oldLeader.equals(leader)) { + return false; + } + oldLeader = leader; + return true; + } + public static NimbusClient getConfiguredClientAs(Map conf, String asUser) { return getConfiguredClientAs(conf, asUser, null); } @@ -102,14 +113,14 @@ public static NimbusClient getConfiguredClientAs(Map conf, Strin conf = fullConf; if (conf.containsKey(Config.STORM_DO_AS_USER)) { if (asUser != null && !asUser.isEmpty()) { - LOG.warn("You have specified a doAsUser as param {} and a doAsParam as config, config will take precedence." - , asUser, conf.get(Config.STORM_DO_AS_USER)); + LOG.warn("You have specified a doAsUser as param {} and a doAsParam as config, config will take precedence.", + asUser, conf.get(Config.STORM_DO_AS_USER)); } asUser = (String) conf.get(Config.STORM_DO_AS_USER); } List seeds; - if(conf.containsKey(Config.NIMBUS_HOST)) { + if (conf.containsKey(Config.NIMBUS_HOST)) { LOG.warn("Using deprecated config {} for backward compatibility. Please update your storm.yaml so it only has config {}", Config.NIMBUS_HOST, Config.NIMBUS_SEEDS); seeds = Lists.newArrayList(conf.get(Config.NIMBUS_HOST).toString()); @@ -126,7 +137,9 @@ public static NimbusClient getConfiguredClientAs(Map conf, Strin nimbusSummary = client.getClient().getLeader(); if (nimbusSummary != null) { String leaderNimbus = nimbusSummary.get_host() + ":" + nimbusSummary.get_port(); - LOG.info("Found leader nimbus : {}", leaderNimbus); + if (LOG.isDebugEnabled() || shouldLogLeader(leaderNimbus)) { + LOG.info("Found leader nimbus : {}", leaderNimbus); + } if (nimbusSummary.get_host().equals(host) && nimbusSummary.get_port() == port) { NimbusClient ret = client; client = null; @@ -147,13 +160,12 @@ public static NimbusClient getConfiguredClientAs(Map conf, Strin client.close(); } } - throw new NimbusLeaderNotFoundException("Could not find a nimbus leader, please try " + - "again after some time."); + throw new NimbusLeaderNotFoundException("Could not find a nimbus leader, please try again after some time."); } throw new NimbusLeaderNotFoundException( - "Could not find leader nimbus from seed hosts " + seeds + ". " + - "Did you specify a valid list of nimbus hosts for config " + - Config.NIMBUS_SEEDS + "?"); + "Could not find leader nimbus from seed hosts " + seeds + ". " + + "Did you specify a valid list of nimbus hosts for config " + + Config.NIMBUS_SEEDS + "?"); } public NimbusClient(Map conf, String host, int port) throws TTransportException { diff --git a/storm-client/src/jvm/org/apache/storm/utils/Utils.java b/storm-client/src/jvm/org/apache/storm/utils/Utils.java index 37c29630c79..4ad2ee2f7b8 100644 --- a/storm-client/src/jvm/org/apache/storm/utils/Utils.java +++ b/storm-client/src/jvm/org/apache/storm/utils/Utils.java @@ -402,14 +402,24 @@ public static SmartThread asyncLoop(final Callable afn) { * @return true if throwable is instance of klass, false otherwise. */ public static boolean exceptionCauseIsInstanceOf(Class klass, Throwable throwable) { - Throwable t = throwable; + return unwrapTo(klass, throwable) != null; + } + + public static T unwrapTo(Class klass, Throwable t) { while (t != null) { if (klass.isInstance(t)) { - return true; + return (T)t; } t = t.getCause(); } - return false; + return null; + } + + public static void unwrapAndThrow(Class klass, Throwable t) throws T { + T ret = unwrapTo(klass, t); + if (ret != null) { + throw ret; + } } public static RuntimeException wrapInRuntime(Exception e){ @@ -1100,21 +1110,6 @@ public static String threadDump() { return dump.toString(); } - public static long getVersionFromBlobVersionFile(File versionFile) { - long currentVersion = 0; - if (versionFile.exists() && !(versionFile.isDirectory())) { - try (BufferedReader br = new BufferedReader(new FileReader(versionFile))) { - String line = br.readLine(); - currentVersion = Long.parseLong(line); - } catch (IOException e) { - throw new RuntimeException(e); - } - return currentVersion; - } else { - return -1; - } - } - public static boolean checkDirExists(String dir) { File file = new File(dir); return file.isDirectory(); diff --git a/storm-core/test/clj/integration/org/apache/storm/testing4j_test.clj b/storm-core/test/clj/integration/org/apache/storm/testing4j_test.clj index 1b12928c682..c1ba39e771d 100644 --- a/storm-core/test/clj/integration/org/apache/storm/testing4j_test.clj +++ b/storm-core/test/clj/integration/org/apache/storm/testing4j_test.clj @@ -61,77 +61,6 @@ (is (Time/isSimulating))))) (is (not (Time/isSimulating))))) -(def complete-topology-testjob - (reify TestJob - (^void run [this ^ILocalCluster cluster] - (let [topology (Thrift/buildTopology - {"spout" (Thrift/prepareSpoutDetails (TestWordSpout. true) (Integer. 3))} - {"2" (Thrift/prepareBoltDetails - {(GlobalStreamId. "spout" Utils/DEFAULT_STREAM_ID) - (Thrift/prepareFieldsGrouping ["word"])} - (TestWordCounter.) (Integer. 4)) - "3" (Thrift/prepareBoltDetails - {(GlobalStreamId. "spout" Utils/DEFAULT_STREAM_ID) - (Thrift/prepareGlobalGrouping)} - (TestGlobalCount.)) - "4" (Thrift/prepareBoltDetails - {(GlobalStreamId. "2" Utils/DEFAULT_STREAM_ID) - (Thrift/prepareGlobalGrouping)} - (TestAggregatesCounter.))}) - mocked-sources (doto (MockedSources.) - (.addMockData "spout" (into-array Values [(Values. (into-array ["nathan"])) - (Values. (into-array ["bob"])) - (Values. (into-array ["joey"])) - (Values. (into-array ["nathan"]))]) - )) - storm-conf (doto (Config.) - (.setNumWorkers 2)) - complete-topology-param (doto (CompleteTopologyParam.) - (.setMockedSources mocked-sources) - (.setStormConf storm-conf)) - results (Testing/completeTopology cluster - topology - complete-topology-param)] - (is (Testing/multiseteq [["nathan"] ["bob"] ["joey"] ["nathan"]] - (Testing/readTuples results "spout"))) - (is (Testing/multiseteq [["nathan" (int 1)] ["nathan" (int 2)] ["bob" (int 1)] ["joey" (int 1)]] - (Testing/readTuples results "2"))) - (is (= [[1] [2] [3] [4]] - (Testing/readTuples results "3"))) - (is (= [[1] [2] [3] [4]] - (Testing/readTuples results "4"))) - )))) - -(deftest test-complete-topology-netty-simulated - (let [daemon-conf (doto (Config.) - (.put STORM-LOCAL-MODE-ZMQ true)) - mk-cluster-param (doto (MkClusterParam.) - (.setSupervisors (int 4)) - (.setDaemonConf daemon-conf))] - (Testing/withSimulatedTimeLocalCluster - mk-cluster-param complete-topology-testjob))) - -(deftest test-complete-topology-netty - (let [daemon-conf (doto (Config.) - (.put STORM-LOCAL-MODE-ZMQ true)) - mk-cluster-param (doto (MkClusterParam.) - (.setSupervisors (int 4)) - (.setDaemonConf daemon-conf))] - (Testing/withLocalCluster - mk-cluster-param complete-topology-testjob))) - -(deftest test-complete-topology-local - (let [mk-cluster-param (doto (MkClusterParam.) - (.setSupervisors (int 4)))] - (Testing/withLocalCluster - mk-cluster-param complete-topology-testjob))) - -(deftest test-complete-topology-local-simulated - (let [mk-cluster-param (doto (MkClusterParam.) - (.setSupervisors (int 4)))] - (Testing/withSimulatedTimeLocalCluster - mk-cluster-param complete-topology-testjob))) - (deftest test-with-tracked-cluster (Testing/withTrackedCluster (reify TestJob diff --git a/storm-server/pom.xml b/storm-server/pom.xml index 02104beea01..9a1b4c0c0a2 100644 --- a/storm-server/pom.xml +++ b/storm-server/pom.xml @@ -130,7 +130,7 @@ maven-checkstyle-plugin - 3590 + 2699 diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java index 08d32f14a0d..e11ee4b0590 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java @@ -108,7 +108,7 @@ public Supervisor(Map conf, IContext sharedContext, ISupervisor try { this.localState = ServerConfigUtils.supervisorState(conf); - this.asyncLocalizer = new AsyncLocalizer(conf, currAssignment, localState.getLocalAssignmentsMap()); + this.asyncLocalizer = new AsyncLocalizer(conf); } catch (IOException e) { throw Utils.wrapInRuntime(e); } diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java index 33574c3d28a..30031f65326 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java @@ -69,10 +69,21 @@ public static void rmrAsUser(Map conf, String id, String path) t * @param blobInfo * @return */ - public static Boolean shouldUncompressBlob(Map blobInfo) { + public static boolean shouldUncompressBlob(Map blobInfo) { return ObjectReader.getBoolean(blobInfo.get("uncompress"), false); } + /** + * Given the blob information returns the value of the workerRestart field, handling it either being a string or a boolean value, or + * if it's not specified then returns false + * + * @param blobInfo + * @return + */ + public static boolean needsCallback(Map blobInfo) { + return ObjectReader.getBoolean(blobInfo.get("workerRestart"), false); + } + /** * Returns a list of LocalResources based on the blobstore-map passed in * @@ -83,7 +94,8 @@ public static List blobstoreMapToLocalresources(Map localResourceList = new ArrayList<>(); if (blobstoreMap != null) { for (Map.Entry> map : blobstoreMap.entrySet()) { - LocalResource localResource = new LocalResource(map.getKey(), shouldUncompressBlob(map.getValue())); + Map blobConf = map.getValue(); + LocalResource localResource = new LocalResource(map.getKey(), shouldUncompressBlob(blobConf), needsCallback(blobConf)); localResourceList.add(localResource); } } diff --git a/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java b/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java index 4b475024626..71f3495e744 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java @@ -20,13 +20,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import java.io.BufferedWriter; import java.io.File; -import java.io.FileOutputStream; -import java.io.FileWriter; import java.io.IOException; -import java.io.PrintWriter; -import java.net.URLDecoder; import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; @@ -39,100 +34,73 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; -import java.util.stream.Collectors; -import org.apache.commons.io.FileUtils; import org.apache.storm.Config; import org.apache.storm.DaemonConfig; import org.apache.storm.blobstore.ClientBlobStore; -import org.apache.storm.blobstore.InputStreamWithMeta; import org.apache.storm.daemon.supervisor.AdvancedFSOps; import org.apache.storm.daemon.supervisor.SupervisorUtils; import org.apache.storm.generated.AuthorizationException; import org.apache.storm.generated.KeyNotFoundException; import org.apache.storm.generated.LocalAssignment; import org.apache.storm.generated.StormTopology; -import org.apache.storm.streams.Pair; import org.apache.storm.utils.ConfigUtils; import org.apache.storm.utils.NimbusLeaderNotFoundException; import org.apache.storm.utils.ObjectReader; import org.apache.storm.utils.ServerUtils; -import org.apache.storm.utils.ShellUtils; import org.apache.storm.utils.Utils; import org.apache.thrift.transport.TTransportException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; - /** * Downloads and caches blobs locally. */ public class AsyncLocalizer implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class); - public static final String FILECACHE = "filecache"; - public static final String USERCACHE = "usercache"; - // sub directories to store either files or uncompressed archives respectively - public static final String FILESDIR = "files"; - public static final String ARCHIVESDIR = "archives"; - private static final String TO_UNCOMPRESS = "_tmp_"; private static final CompletableFuture ALL_DONE_FUTURE = new CompletableFuture<>(); + static { ALL_DONE_FUTURE.complete(null); } - private static Set readDownloadedTopologyIds(Map conf) throws IOException { - Set stormIds = new HashSet<>(); - String path = ConfigUtils.supervisorStormDistRoot(conf); - Collection rets = ConfigUtils.readDirContents(path); - for (String ret : rets) { - stormIds.add(URLDecoder.decode(ret, "UTF-8")); - } - return stormIds; - } - - private final AtomicReference> currAssignment; private final boolean isLocalMode; - private final Map blobPending; + // track resources - user to resourceSet + protected final ConcurrentMap> userFiles = new ConcurrentHashMap<>(); + protected final ConcurrentMap> userArchives = new ConcurrentHashMap<>(); + // topology to tracking of topology dir and resources + private final Map> blobPending; private final Map conf; private final AdvancedFSOps fsOps; private final boolean symlinksDisabled; private final ConcurrentHashMap topologyBlobs = new ConcurrentHashMap<>(); private final ConcurrentHashMap> topologyBasicDownloaded = new ConcurrentHashMap<>(); - // track resources - user to resourceSet - private final ConcurrentMap userRsrc = new ConcurrentHashMap<>(); - - private final String localBaseDir; - + private final Path localBaseDir; private final int blobDownloadRetries; private final ScheduledExecutorService execService; // cleanup - private long cacheTargetSize; + @VisibleForTesting + protected long cacheTargetSize; private final long cacheCleanupPeriod; @VisibleForTesting - AsyncLocalizer(Map conf, AdvancedFSOps ops, String baseDir, - AtomicReference> currAssignment, - Map portToAssignments) throws IOException { + AsyncLocalizer(Map conf, AdvancedFSOps ops, String baseDir) throws IOException { this.conf = conf; isLocalMode = ConfigUtils.isLocalMode(conf); fsOps = ops; - localBaseDir = baseDir; + localBaseDir = Paths.get(baseDir); // default cache size 10GB, converted to Bytes cacheTargetSize = ObjectReader.getInt(conf.get(DaemonConfig.SUPERVISOR_LOCALIZER_CACHE_TARGET_SIZE_MB), 10 * 1024).longValue() << 20; @@ -151,14 +119,10 @@ private static Set readDownloadedTopologyIds(Map conf) t symlinksDisabled = (boolean)conf.getOrDefault(Config.DISABLE_SYMLINKS, false); blobPending = new HashMap<>(); - this.currAssignment = currAssignment; - - recoverBlobReferences(portToAssignments); } - public AsyncLocalizer(Map conf, AtomicReference> currAssignment, - Map portToAssignments) throws IOException { - this(conf, AdvancedFSOps.make(conf), ConfigUtils.supervisorLocalDir(conf), currAssignment, portToAssignments); + public AsyncLocalizer(Map conf) throws IOException { + this(conf, AdvancedFSOps.make(conf), ConfigUtils.supervisorLocalDir(conf)); } @VisibleForTesting @@ -197,29 +161,54 @@ LocallyCachedBlob getTopoConf(final String topologyId) throws IOException { return topoConf; } + private LocalizedResource getUserArchive(String user, String key) throws IOException { + assert user != null : "All user archives require a user present"; + ConcurrentMap keyToResource = userArchives.computeIfAbsent(user, (u) -> new ConcurrentHashMap<>()); + return keyToResource.computeIfAbsent(key, (k) -> new LocalizedResource(key, localBaseDir, true, fsOps, conf, user)); + } + + private LocalizedResource getUserFile(String user, String key) throws IOException { + assert user != null : "All user archives require a user present"; + ConcurrentMap keyToResource = userFiles.computeIfAbsent(user, (u) -> new ConcurrentHashMap<>()); + return keyToResource.computeIfAbsent(key, (k) -> new LocalizedResource(key, localBaseDir, false, fsOps, conf, user)); + } + + /** + * Request that all of the blobs necessary for this topology be downloaded. + * @param assignment the assignment that needs the blobs + * @param port the port the assignment is a part of + * @param cb a callback for when the blobs change. This is only for blobs that are tied to the lifetime of the worker. + * @return a Future that indicates when they are all downloaded. + * @throws IOException if there was an error while trying doing it. + */ public synchronized CompletableFuture requestDownloadTopologyBlobs(final LocalAssignment assignment, final int port, final BlobChangingCallback cb) throws IOException { - final String topologyId = assignment.get_topology_id(); + final PortAndAssignment pna = new PortAndAssignment(port, assignment); + final String topologyId = pna.getToplogyId(); - CompletableFuture baseBlobs = requestDownloadBaseTopologyBlobs(assignment, port, cb); + CompletableFuture baseBlobs = requestDownloadBaseTopologyBlobs(pna, cb); return baseBlobs.thenComposeAsync((v) -> { - LocalDownloadedResource localResource = blobPending.get(topologyId); + CompletableFuture localResource = blobPending.get(topologyId); if (localResource == null) { - Supplier supplier = new DownloadBlobs(topologyId, assignment.get_owner()); - localResource = new LocalDownloadedResource(CompletableFuture.supplyAsync(supplier, execService)); + Supplier supplier = new DownloadBlobs(pna, cb); + localResource = CompletableFuture.supplyAsync(supplier, execService); blobPending.put(topologyId, localResource); + } else { + try { + addReferencesToBlobs(pna, cb); + } catch (Exception e) { + throw new RuntimeException(e); + } } - CompletableFuture r = localResource.reserve(port, assignment); LOG.debug("Reserved blobs {} {}", topologyId, localResource); - return r; + return localResource; }); } @VisibleForTesting - synchronized CompletableFuture requestDownloadBaseTopologyBlobs(final LocalAssignment assignment, final int port, - BlobChangingCallback cb) throws IOException { - PortAndAssignment pna = new PortAndAssignment(port, assignment); - final String topologyId = assignment.get_topology_id(); + synchronized CompletableFuture requestDownloadBaseTopologyBlobs(PortAndAssignment pna, BlobChangingCallback cb) + throws IOException { + final String topologyId = pna.getToplogyId(); LocallyCachedBlob topoJar = getTopoJar(topologyId); topoJar.addReference(pna, cb); @@ -233,6 +222,7 @@ synchronized CompletableFuture requestDownloadBaseTopologyBlobs(final Loca CompletableFuture ret = topologyBasicDownloaded.get(topologyId); if (ret == null) { ret = downloadOrUpdate(topoJar, topoCode, topoConf); + topologyBasicDownloaded.put(topologyId, ret); } return ret; } @@ -240,12 +230,16 @@ synchronized CompletableFuture requestDownloadBaseTopologyBlobs(final Loca private static final int ATTEMPTS_INTERVAL_TIME = 100; private CompletableFuture downloadOrUpdate(LocallyCachedBlob ... blobs) { - CompletableFuture [] all = new CompletableFuture[blobs.length]; - for (int i = 0; i < blobs.length; i++) { - final LocallyCachedBlob blob = blobs[i]; + return downloadOrUpdate(Arrays.asList(blobs)); + } + + private CompletableFuture downloadOrUpdate(Collection blobs) { + CompletableFuture [] all = new CompletableFuture[blobs.size()]; + int i = 0; + for (final LocallyCachedBlob blob: blobs) { all[i] = CompletableFuture.runAsync(() -> { LOG.debug("STARTING download of {}", blob); - try (ClientBlobStore blobStore = ServerUtils.getClientBlobStoreForSupervisor(conf)) { + try (ClientBlobStore blobStore = getClientBlobStore()) { boolean done = false; long failures = 0; while (!done) { @@ -253,7 +247,7 @@ private CompletableFuture downloadOrUpdate(LocallyCachedBlob ... blobs) { synchronized (blob) { long localVersion = blob.getLocalVersion(); long remoteVersion = blob.getRemoteVersion(blobStore); - if (localVersion != remoteVersion) { + if (localVersion != remoteVersion || !blob.isFullyDownloaded()) { try { long newVersion = blob.downloadToTempLocation(blobStore); blob.informAllOfChangeAndWaitForConsensus(); @@ -277,104 +271,42 @@ private CompletableFuture downloadOrUpdate(LocallyCachedBlob ... blobs) { } LOG.debug("FINISHED download of {}", blob); }, execService); + i++; } return CompletableFuture.allOf(all); } - /** - * For each of the downloaded topologies, adds references to the blobs that the topologies are using. This is used to reconstruct the - * cache on restart. - * @param topoId the topology id - * @param user the User that owns this topology - */ - private void addBlobReferences(String topoId, String user) throws IOException { - Map topoConf = ConfigUtils.readSupervisorStormConf(conf, topoId); - @SuppressWarnings("unchecked") - Map> blobstoreMap = (Map>) topoConf.get(Config.TOPOLOGY_BLOBSTORE_MAP); - String topoName = (String) topoConf.get(Config.TOPOLOGY_NAME); - List localresources = SupervisorUtils.blobstoreMapToLocalresources(blobstoreMap); - if (blobstoreMap != null) { - addReferences(localresources, user, topoName); - } - } - - /** - * Pick up where we left off last time. - * @param portToAssignments the current set of assignments for the supervisor. - */ - private void recoverBlobReferences(Map portToAssignments) throws IOException { - Set downloadedTopoIds = readDownloadedTopologyIds(conf); - if (portToAssignments != null) { - Map assignments = new HashMap<>(); - for (LocalAssignment la : portToAssignments.values()) { - assignments.put(la.get_topology_id(), la); - } - for (String topoId : downloadedTopoIds) { - LocalAssignment la = assignments.get(topoId); - if (la != null) { - addBlobReferences(topoId, la.get_owner()); - } else { - LOG.warn("Could not find an owner for topo {}", topoId); - } - } - } - } - /** * Downloads all blobs listed in the topology configuration for all topologies assigned to this supervisor, and creates version files * with a suffix. The runnable is intended to be run periodically by a timer, created elsewhere. */ - private void updateBlobs() { - for (LocallyCachedBlob blob : topologyBlobs.values()) { - if (blob.isUsed()) { - try { - downloadOrUpdate(blob); - } catch (Exception e) { - LOG.error("Could not update {}", blob, e); - } + @VisibleForTesting + void updateBlobs() { + List> futures = new ArrayList<>(); + futures.add(downloadOrUpdate(topologyBlobs.values())); + if (symlinksDisabled) { + LOG.warn("symlinks are disabled so blobs cannot be downloaded."); + } else { + for (ConcurrentMap map : userArchives.values()) { + futures.add(downloadOrUpdate(map.values())); + } + + for (ConcurrentMap map : userFiles.values()) { + futures.add(downloadOrUpdate(map.values())); } } - try { - Map topoIdToOwner = currAssignment.get().values().stream() - .map((la) -> Pair.of(la.get_topology_id(), la.get_owner())) - .distinct() - .collect(Collectors.toMap(Pair::getFirst, Pair::getSecond)); - for (String topoId : readDownloadedTopologyIds(conf)) { - String owner = topoIdToOwner.get(topoId); - if (owner == null) { - //We got a case where the local assignment is not up to date, no point in going on... - LOG.warn("The blobs will not be updated for {} until the local assignment is updated...", topoId); + for (CompletableFuture f: futures) { + try { + f.get(); + } catch (Exception e) { + if (Utils.exceptionCauseIsInstanceOf(TTransportException.class, e)) { + LOG.error("Network error while updating blobs, will retry again later", e); + } else if (Utils.exceptionCauseIsInstanceOf(NimbusLeaderNotFoundException.class, e)) { + LOG.error("Nimbus unavailable to update blobs, will retry again later", e); } else { - String stormRoot = ConfigUtils.supervisorStormDistRoot(conf, topoId); - LOG.debug("Checking Blob updates for storm topology id {} With target_dir: {}", topoId, stormRoot); - updateBlobsForTopology(conf, topoId, owner); + LOG.error("Could not update blob, will retry again later", e); } } - } catch (Exception e) { - if (Utils.exceptionCauseIsInstanceOf(TTransportException.class, e)) { - LOG.error("Network error while updating blobs, will retry again later", e); - } else if (Utils.exceptionCauseIsInstanceOf(NimbusLeaderNotFoundException.class, e)) { - LOG.error("Nimbus unavailable to update blobs, will retry again later", e); - } else { - throw Utils.wrapInRuntime(e); - } - } - } - - /** - * Update each blob listed in the topology configuration if the latest version of the blob has not been downloaded. - */ - private void updateBlobsForTopology(Map conf, String stormId, String user) - throws IOException { - Map topoConf = ConfigUtils.readSupervisorStormConf(conf, stormId); - Map> blobstoreMap = (Map>) topoConf.get(Config.TOPOLOGY_BLOBSTORE_MAP); - List localresources = SupervisorUtils.blobstoreMapToLocalresources(blobstoreMap); - try { - updateBlobs(localresources, user); - } catch (AuthorizationException authExp) { - LOG.error("AuthorizationException error", authExp); - } catch (KeyNotFoundException knf) { - LOG.error("KeyNotFoundException error", knf); } } @@ -384,6 +316,7 @@ private void updateBlobsForTopology(Map conf, String stormId, St */ public void start() { execService.scheduleWithFixedDelay(this::updateBlobs, 30, 30, TimeUnit.SECONDS); + LOG.debug("Scheduling cleanup every {} millis", cacheCleanupPeriod); execService.scheduleAtFixedRate(this::cleanup, cacheCleanupPeriod, cacheCleanupPeriod, TimeUnit.MILLISECONDS); } @@ -394,58 +327,78 @@ public void close() throws InterruptedException { } } - //ILocalizer + private List getLocalResources(PortAndAssignment pna) throws IOException { + String topologyId = pna.getToplogyId(); + Map topoConf = ConfigUtils.readSupervisorStormConf(conf, topologyId); + + @SuppressWarnings("unchecked") + Map> blobstoreMap = (Map>) topoConf.get(Config.TOPOLOGY_BLOBSTORE_MAP); + + List ret = new ArrayList<>(); + if (blobstoreMap != null) { + List tmp = SupervisorUtils.blobstoreMapToLocalresources(blobstoreMap); + if (tmp != null) { + ret.addAll(tmp); + } + } + + StormTopology stormCode = ConfigUtils.readSupervisorTopology(conf, topologyId, fsOps); + List dependencies = new ArrayList<>(); + if (stormCode.is_set_dependency_jars()) { + dependencies.addAll(stormCode.get_dependency_jars()); + } + if (stormCode.is_set_dependency_artifacts()) { + dependencies.addAll(stormCode.get_dependency_artifacts()); + } + for (String dependency : dependencies) { + ret.add(new LocalResource(dependency, false, true)); + } + return ret; + } + + @VisibleForTesting + void addReferencesToBlobs(PortAndAssignment pna, BlobChangingCallback cb) + throws IOException, KeyNotFoundException, AuthorizationException { + List localResourceList = getLocalResources(pna); + if (!localResourceList.isEmpty()) { + getBlobs(localResourceList, pna, cb); + } + } + private class DownloadBlobs implements Supplier { - private final String topologyId; - private final String topoOwner; + private final PortAndAssignment pna; + private final BlobChangingCallback cb; - public DownloadBlobs(String topologyId, String topoOwner) { - this.topologyId = topologyId; - this.topoOwner = topoOwner; + public DownloadBlobs(PortAndAssignment pna, BlobChangingCallback cb) { + this.pna = pna; + this.cb = cb; } @Override public Void get() { try { + String topologyId = pna.getToplogyId(); + String topoOwner = pna.getOwner(); String stormroot = ConfigUtils.supervisorStormDistRoot(conf, topologyId); Map topoConf = ConfigUtils.readSupervisorStormConf(conf, topologyId); @SuppressWarnings("unchecked") - Map> blobstoreMap = (Map>) topoConf.get(Config.TOPOLOGY_BLOBSTORE_MAP); - String topoName = (String) topoConf.get(Config.TOPOLOGY_NAME); - - List localResourceList = new ArrayList<>(); - if (blobstoreMap != null) { - List tmp = SupervisorUtils.blobstoreMapToLocalresources(blobstoreMap); - if (tmp != null) { - localResourceList.addAll(tmp); - } - } - - StormTopology stormCode = ConfigUtils.readSupervisorTopology(conf, topologyId, fsOps); - List dependencies = new ArrayList<>(); - if (stormCode.is_set_dependency_jars()) { - dependencies.addAll(stormCode.get_dependency_jars()); - } - if (stormCode.is_set_dependency_artifacts()) { - dependencies.addAll(stormCode.get_dependency_artifacts()); - } - for (String dependency : dependencies) { - localResourceList.add(new LocalResource(dependency, false)); - } + Map> blobstoreMap = + (Map>) topoConf.get(Config.TOPOLOGY_BLOBSTORE_MAP); + List localResourceList = getLocalResources(pna); if (!localResourceList.isEmpty()) { File userDir = getLocalUserFileCacheDir(topoOwner); if (!fsOps.fileExists(userDir)) { fsOps.forceMkdir(userDir); } - List localizedResources = getBlobs(localResourceList, topoOwner, topoName, userDir); + List localizedResources = getBlobs(localResourceList, pna, cb); fsOps.setupBlobPermissions(userDir, topoOwner); if (!symlinksDisabled) { for (LocalizedResource localizedResource : localizedResources) { String keyName = localizedResource.getKey(); //The sym link we are pointing to - File rsrcFilePath = new File(localizedResource.getCurrentSymlinkPath()); + File rsrcFilePath = localizedResource.getCurrentSymlinkPath().toFile(); String symlinkName = null; if (blobstoreMap != null) { @@ -472,10 +425,18 @@ public Void get() { } } - public synchronized void recoverRunningTopology(final LocalAssignment assignment, final int port, + /** + * Do everything needed to recover the state in the AsyncLocalizer for a running topology. + * @param currentAssignment the assignment for the topology. + * @param port the port the assignment is on. + * @param cb a callback for when the blobs are updated. This will only be for blobs that + * indicate that if they change the worker should be restarted. + * @throws IOException on any error trying to recover the state. + */ + public synchronized void recoverRunningTopology(final LocalAssignment currentAssignment, final int port, final BlobChangingCallback cb) throws IOException { - PortAndAssignment pna = new PortAndAssignment(port, assignment); - final String topologyId = assignment.get_topology_id(); + final PortAndAssignment pna = new PortAndAssignment(port, currentAssignment); + final String topologyId = pna.getToplogyId(); LocallyCachedBlob topoJar = getTopoJar(topologyId); topoJar.addReference(pna, cb); @@ -486,12 +447,18 @@ public synchronized void recoverRunningTopology(final LocalAssignment assignment LocallyCachedBlob topoConf = getTopoConf(topologyId); topoConf.addReference(pna, cb); - LocalDownloadedResource localResource = blobPending.get(topologyId); + CompletableFuture localResource = blobPending.get(topologyId); if (localResource == null) { - localResource = new LocalDownloadedResource(ALL_DONE_FUTURE); + localResource = ALL_DONE_FUTURE; blobPending.put(topologyId, localResource); } - localResource.reserve(port, assignment); + + try { + addReferencesToBlobs(pna, cb); + } catch (KeyNotFoundException | AuthorizationException e) { + LOG.error("Could not recover all blob references for {}", pna); + } + LOG.debug("Recovered blobs {} {}", topologyId, localResource); } @@ -526,125 +493,52 @@ public synchronized void releaseSlotFor(LocalAssignment assignment, int port) th topoConfBlob.removeReference(pna); } - LocalDownloadedResource localResource = blobPending.get(topologyId); - if (localResource == null || !localResource.release(port, assignment)) { - LOG.warn("Released blob reference {} {} for something that we didn't have {}", topologyId, port, localResource); - } else if (localResource.isDone()){ - LOG.info("Released blob reference {} {} Cleaning up BLOB references...", topologyId, port); - blobPending.remove(topologyId); - Map topoConf = ConfigUtils.readSupervisorStormConf(conf, topologyId); - @SuppressWarnings("unchecked") - Map> blobstoreMap = (Map>) topoConf.get(Config.TOPOLOGY_BLOBSTORE_MAP); - if (blobstoreMap != null) { - String user = assignment.get_owner(); - String topoName = (String) topoConf.get(Config.TOPOLOGY_NAME); - - for (Map.Entry> entry : blobstoreMap.entrySet()) { - String key = entry.getKey(); - Map blobInfo = entry.getValue(); - try { - removeBlobReference(key, user, topoName, SupervisorUtils.shouldUncompressBlob(blobInfo)); - } catch (Exception e) { - throw new IOException(e); - } - } + for (LocalResource lr : getLocalResources(pna)) { + try { + removeBlobReference(lr.getBlobName(), pna, lr.shouldUncompress()); + } catch (Exception e) { + throw new IOException(e); } - } else { - LOG.debug("Released blob reference {} {} still waiting on {}", topologyId, port, localResource); } } - //From Localizer - - @VisibleForTesting - void setTargetCacheSize(long size) { - cacheTargetSize = size; - } - - // For testing, be careful as it doesn't clone - ConcurrentMap getUserResources() { - return userRsrc; - } - - // baseDir/supervisor/usercache/ - private File getUserCacheDir() { - return new File(localBaseDir, USERCACHE); - } - // baseDir/supervisor/usercache/user1/ + @VisibleForTesting File getLocalUserDir(String userName) { - return new File(getUserCacheDir(), userName); + return LocalizedResource.getLocalUserDir(localBaseDir, userName).toFile(); } // baseDir/supervisor/usercache/user1/filecache + @VisibleForTesting File getLocalUserFileCacheDir(String userName) { - return new File(getLocalUserDir(userName), FILECACHE); + return LocalizedResource.getLocalUserFileCacheDir(localBaseDir, userName).toFile(); } - // baseDir/supervisor/usercache/user1/filecache/files - private File getCacheDirForFiles(File dir) { - return new File(dir, FILESDIR); - } - - // get the directory to put uncompressed archives in - // baseDir/supervisor/usercache/user1/filecache/archives - private File getCacheDirForArchives(File dir) { - return new File(dir, ARCHIVESDIR); - } - - private void addLocalizedResourceInDir(String dir, LocalizedResourceSet lrsrcSet, - boolean uncompress) { - File[] lrsrcs = readCurrentBlobs(dir); - - if (lrsrcs != null) { - for (File rsrc : lrsrcs) { - LOG.info("add localized in dir found: " + rsrc); - /// strip off .suffix - String path = rsrc.getPath(); - int p = path.lastIndexOf('.'); - if (p > 0) { - path = path.substring(0, p); - } - LOG.debug("local file is: {} path is: {}", rsrc.getPath(), path); - LocalizedResource lrsrc = new LocalizedResource(new File(path).getName(), path, - uncompress); - lrsrcSet.add(lrsrc.getKey(), lrsrc, uncompress); - } + private void recoverLocalizedArchivesForUser(String user) throws IOException { + for (String key: LocalizedResource.getLocalizedArchiveKeys(localBaseDir, user)) { + getUserArchive(user, key); } } - // Looks for files in the directory with .current suffix - private File[] readCurrentBlobs(String location) { - File dir = new File(location); - File[] files = null; - if (dir.exists()) { - files = dir.listFiles((d, name) -> name.toLowerCase().endsWith(ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX)); + private void recoverLocalizedFilesForUser(String user) throws IOException { + for (String key: LocalizedResource.getLocalizedFileKeys(localBaseDir, user)) { + getUserFile(user, key); } - return files; } // Check to see if there are any existing files already localized. private void reconstructLocalizedResources() { try { - LOG.info("Reconstruct localized resource: " + getUserCacheDir().getPath()); - Collection users = ConfigUtils.readDirFiles(getUserCacheDir().getPath()); + LOG.info("Reconstruct localized resources"); + Collection users = LocalizedResource.getLocalizedUsers(localBaseDir); if (!(users == null || users.isEmpty())) { - for (File userDir : users) { - String user = userDir.getName(); - LOG.debug("looking in: {} for user: {}", userDir.getPath(), user); - LocalizedResourceSet newSet = new LocalizedResourceSet(user); - LocalizedResourceSet lrsrcSet = userRsrc.putIfAbsent(user, newSet); - if (lrsrcSet == null) { - lrsrcSet = newSet; - } - addLocalizedResourceInDir(getCacheDirForFiles(getLocalUserFileCacheDir(user)).getPath(), - lrsrcSet, false); - addLocalizedResourceInDir( - getCacheDirForArchives(getLocalUserFileCacheDir(user)).getPath(), - lrsrcSet, true); + for (String user : users) { + LOG.debug("reconstructing resources owned by {}", user); + recoverLocalizedFilesForUser(user); + recoverLocalizedArchivesForUser(user); } } else { - LOG.warn("No left over resources found for any user during reconstructing of local resources at: {}", getUserCacheDir().getPath()); + LOG.debug("No left over resources found for any user"); } } catch (Exception e) { LOG.error("ERROR reconstructing localized resources", e); @@ -652,17 +546,19 @@ private void reconstructLocalizedResources() { } // ignores invalid user/topo/key - synchronized void removeBlobReference(String key, String user, String topo, + synchronized void removeBlobReference(String key, PortAndAssignment pna, boolean uncompress) throws AuthorizationException, KeyNotFoundException { - LocalizedResourceSet lrsrcSet = userRsrc.get(user); + String user = pna.getOwner(); + String topo = pna.getToplogyId(); + ConcurrentMap lrsrcSet = uncompress ? userArchives.get(user) : userFiles.get(user); if (lrsrcSet != null) { - LocalizedResource lrsrc = lrsrcSet.get(key, uncompress); + LocalizedResource lrsrc = lrsrcSet.get(key); if (lrsrc != null) { LOG.debug("removing blob reference to: {} for topo: {}", key, topo); - lrsrc.removeReference(topo); + lrsrc.removeReference(pna); } else { - LOG.warn("trying to remove non-existent blob, key: " + key + " for user: " + user + - " topo: " + topo); + LOG.warn("trying to remove non-existent blob, key: " + key + " for user: " + user + + " topo: " + topo); } } else { LOG.warn("trying to remove blob for non-existent resource set for user: " + user + " key: " @@ -670,205 +566,45 @@ synchronized void removeBlobReference(String key, String user, String topo, } } - synchronized void addReferences(List localresource, String user, - String topo) { - LocalizedResourceSet lrsrcSet = userRsrc.get(user); - if (lrsrcSet != null) { - for (LocalResource blob : localresource) { - LocalizedResource lrsrc = lrsrcSet.get(blob.getBlobName(), blob.shouldUncompress()); - if (lrsrc != null) { - lrsrc.addReference(topo); - LOG.debug("added reference for topo: {} key: {}", topo, blob); - } else { - LOG.warn("trying to add reference to non-existent blob, key: " + blob + " topo: " + topo); - } - } - } else { - LOG.warn("trying to add reference to non-existent local resource set, " + - "user: " + user + " topo: " + topo); - } - } - - /** - * This function either returns the blob in the existing cache or if it doesn't exist in the - * cache, it will download the blob and will block until the download is complete. - */ - LocalizedResource getBlob(LocalResource localResource, String user, String topo, - File userFileDir) throws AuthorizationException, KeyNotFoundException, IOException { - ArrayList arr = new ArrayList<>(); - arr.add(localResource); - List results = getBlobs(arr, user, topo, userFileDir); - if (results.isEmpty() || results.size() != 1) { - throw new IOException("Unknown error getting blob: " + localResource + ", for user: " + user + - ", topo: " + topo); - } - return results.get(0); - } - - private boolean isLocalizedResourceDownloaded(LocalizedResource lrsrc) { - File rsrcFileCurrent = new File(lrsrc.getCurrentSymlinkPath()); - File rsrcFileWithVersion = new File(lrsrc.getFilePathWithVersion()); - File versionFile = new File(lrsrc.getVersionFilePath()); - return (rsrcFileWithVersion.exists() && rsrcFileCurrent.exists() && versionFile.exists()); - } - - private boolean isLocalizedResourceUpToDate(LocalizedResource lrsrc, - ClientBlobStore blobstore) throws AuthorizationException, KeyNotFoundException { - String localFile = lrsrc.getFilePath(); - long nimbusBlobVersion = ServerUtils.nimbusVersionOfBlob(lrsrc.getKey(), blobstore); - long currentBlobVersion = ServerUtils.localVersionOfBlob(localFile); - return (nimbusBlobVersion == currentBlobVersion); - } - protected ClientBlobStore getClientBlobStore() { return ServerUtils.getClientBlobStoreForSupervisor(conf); } - /** - * This function updates blobs on the supervisor. It uses a separate thread pool and runs - * asynchronously of the download and delete. - */ - List updateBlobs(List localResources, - String user) throws AuthorizationException, KeyNotFoundException, IOException { - LocalizedResourceSet lrsrcSet = userRsrc.get(user); - ArrayList results = new ArrayList<>(); - ArrayList> updates = new ArrayList<>(); - - if (lrsrcSet == null) { - // resource set must have been removed - return results; - } - ClientBlobStore blobstore = null; - try { - blobstore = getClientBlobStore(); - for (LocalResource localResource: localResources) { - String key = localResource.getBlobName(); - LocalizedResource lrsrc = lrsrcSet.get(key, localResource.shouldUncompress()); - if (lrsrc == null) { - LOG.warn("blob requested for update doesn't exist: {}", key); - } else if ((boolean) conf.getOrDefault(Config.DISABLE_SYMLINKS, false)) { - LOG.warn("symlinks are disabled so blobs cannot be downloaded."); - } else { - // update it if either the version isn't the latest or if any local blob files are missing - if (!isLocalizedResourceUpToDate(lrsrc, blobstore) || - !isLocalizedResourceDownloaded(lrsrc)) { - LOG.debug("updating blob: {}", key); - updates.add(new DownloadBlob(this, conf, key, new File(lrsrc.getFilePath()), user, - lrsrc.isUncompressed(), true)); - } - } - } - } finally { - if(blobstore != null) { - blobstore.shutdown(); - } - } - try { - List> futures = execService.invokeAll(updates); - for (Future futureRsrc : futures) { - try { - LocalizedResource lrsrc = futureRsrc.get(); - // put the resource just in case it was removed at same time by the cleaner - LocalizedResourceSet newSet = new LocalizedResourceSet(user); - LocalizedResourceSet newlrsrcSet = userRsrc.putIfAbsent(user, newSet); - if (newlrsrcSet == null) { - newlrsrcSet = newSet; - } - newlrsrcSet.putIfAbsent(lrsrc.getKey(), lrsrc, lrsrc.isUncompressed()); - results.add(lrsrc); - } - catch (ExecutionException e) { - LOG.error("Error updating blob: ", e); - if (e.getCause() instanceof AuthorizationException) { - throw (AuthorizationException)e.getCause(); - } - if (e.getCause() instanceof KeyNotFoundException) { - throw (KeyNotFoundException)e.getCause(); - } - } - } - } catch (RejectedExecutionException re) { - LOG.error("Error updating blobs : ", re); - } catch (InterruptedException ie) { - throw new IOException("Interrupted Exception", ie); - } - return results; - } - /** * This function either returns the blobs in the existing cache or if they don't exist in the * cache, it downloads them in parallel (up to SUPERVISOR_BLOBSTORE_DOWNLOAD_THREAD_COUNT) - * and will block until all of them have been downloaded + * and will block until all of them have been downloaded. */ - synchronized List getBlobs(List localResources, - String user, String topo, File userFileDir) + synchronized List getBlobs(List localResources, PortAndAssignment pna, BlobChangingCallback cb) throws AuthorizationException, KeyNotFoundException, IOException { if ((boolean) conf.getOrDefault(Config.DISABLE_SYMLINKS, false)) { throw new KeyNotFoundException("symlinks are disabled so blobs cannot be downloaded."); } - LocalizedResourceSet newSet = new LocalizedResourceSet(user); - LocalizedResourceSet lrsrcSet = userRsrc.putIfAbsent(user, newSet); - if (lrsrcSet == null) { - lrsrcSet = newSet; - } + String user = pna.getOwner(); ArrayList results = new ArrayList<>(); - ArrayList> downloads = new ArrayList<>(); + List> futures = new ArrayList<>(); - ClientBlobStore blobstore = null; try { - blobstore = getClientBlobStore(); for (LocalResource localResource: localResources) { String key = localResource.getBlobName(); boolean uncompress = localResource.shouldUncompress(); - LocalizedResource lrsrc = lrsrcSet.get(key, localResource.shouldUncompress()); - boolean isUpdate = false; - if ((lrsrc != null) && (lrsrc.isUncompressed() == localResource.shouldUncompress()) && - (isLocalizedResourceDownloaded(lrsrc))) { - if (isLocalizedResourceUpToDate(lrsrc, blobstore)) { - LOG.debug("blob already exists: {}", key); - lrsrc.addReference(topo); - results.add(lrsrc); - continue; - } - LOG.debug("blob exists but isn't up to date: {}", key); - isUpdate = true; - } + LocalizedResource lrsrc = uncompress ? getUserArchive(user, key) : getUserFile(user, key); // go off to blobstore and get it // assume dir passed in exists and has correct permission LOG.debug("fetching blob: {}", key); - File downloadDir = getCacheDirForFiles(userFileDir); - File localFile = new File(downloadDir, key); - if (uncompress) { - // for compressed file, download to archives dir - downloadDir = getCacheDirForArchives(userFileDir); - localFile = new File(downloadDir, key); - } - downloadDir.mkdir(); - downloads.add(new DownloadBlob(this, conf, key, localFile, user, uncompress, - isUpdate)); - } - } finally { - if(blobstore !=null) { - blobstore.shutdown(); - } - } - try { - List> futures = execService.invokeAll(downloads); - for (Future futureRsrc: futures) { - LocalizedResource lrsrc = futureRsrc.get(); - lrsrc.addReference(topo); - lrsrcSet.add(lrsrc.getKey(), lrsrc, lrsrc.isUncompressed()); + futures.add(downloadOrUpdate(lrsrc)); results.add(lrsrc); + lrsrc.addReference(pna, localResource.needsCallback() ? cb : null); } - } catch (ExecutionException e) { - if (e.getCause() instanceof AuthorizationException) - throw (AuthorizationException)e.getCause(); - else if (e.getCause() instanceof KeyNotFoundException) { - throw (KeyNotFoundException)e.getCause(); - } else { - throw new IOException("Error getting blobs", e); + + for (CompletableFuture futureRsrc: futures) { + futureRsrc.get(); } + } catch (ExecutionException e) { + Utils.unwrapAndThrow(AuthorizationException.class, e); + Utils.unwrapAndThrow(KeyNotFoundException.class, e); + throw new IOException("Error getting blobs", e); } catch (RejectedExecutionException re) { throw new IOException("RejectedExecutionException: ", re); } catch (InterruptedException ie) { @@ -877,201 +613,6 @@ else if (e.getCause() instanceof KeyNotFoundException) { return results; } - static class DownloadBlob implements Callable { - - private AsyncLocalizer localizer; - private Map conf; - private String key; - private File localFile; - private String user; - private boolean uncompress; - private boolean isUpdate; - - DownloadBlob(AsyncLocalizer localizer, Map conf, String key, File localFile, - String user, boolean uncompress, boolean update) { - this.localizer = localizer; - this.conf = conf; - this.key = key; - this.localFile = localFile; - this.user = user; - this.uncompress = uncompress; - isUpdate = update; - } - - @Override - public LocalizedResource call() - throws AuthorizationException, KeyNotFoundException, IOException { - return localizer.downloadBlob(conf, key, localFile, user, uncompress, - isUpdate); - } - } - - private LocalizedResource downloadBlob(Map conf, String key, File localFile, - String user, boolean uncompress, boolean isUpdate) - throws AuthorizationException, KeyNotFoundException, IOException { - ClientBlobStore blobstore = null; - try { - blobstore = getClientBlobStore(); - long nimbusBlobVersion = ServerUtils.nimbusVersionOfBlob(key, blobstore); - long oldVersion = ServerUtils.localVersionOfBlob(localFile.toString()); - FileOutputStream out = null; - PrintWriter writer = null; - int numTries = 0; - String localizedPath = localFile.toString(); - String localFileWithVersion = ServerUtils.constructBlobWithVersionFileName(localFile.toString(), - nimbusBlobVersion); - String localVersionFile = ServerUtils.constructVersionFileName(localFile.toString()); - String downloadFile = localFileWithVersion; - if (uncompress) { - // we need to download to temp file and then unpack into the one requested - downloadFile = new File(localFile.getParent(), TO_UNCOMPRESS + localFile.getName()).toString(); - } - while (numTries < blobDownloadRetries) { - out = new FileOutputStream(downloadFile); - numTries++; - try { - if (!ServerUtils.canUserReadBlob(blobstore.getBlobMeta(key), user, conf)) { - throw new AuthorizationException(user + " does not have READ access to " + key); - } - InputStreamWithMeta in = blobstore.getBlob(key); - byte[] buffer = new byte[1024]; - int len; - while ((len = in.read(buffer)) >= 0) { - out.write(buffer, 0, len); - } - out.close(); - in.close(); - if (uncompress) { - ServerUtils.unpack(new File(downloadFile), new File(localFileWithVersion)); - LOG.debug("uncompressed " + downloadFile + " to: " + localFileWithVersion); - } - - // Next write the version. - LOG.info("Blob: " + key + " updated with new Nimbus-provided version: " + - nimbusBlobVersion + " local version was: " + oldVersion); - // The false parameter ensures overwriting the version file, not appending - writer = new PrintWriter( - new BufferedWriter(new FileWriter(localVersionFile, false))); - writer.println(nimbusBlobVersion); - writer.close(); - - try { - setBlobPermissions(conf, user, localFileWithVersion); - setBlobPermissions(conf, user, localVersionFile); - - // Update the key.current symlink. First create tmp symlink and do - // move of tmp to current so that the operation is atomic. - String tmp_uuid_local = java.util.UUID.randomUUID().toString(); - LOG.debug("Creating a symlink @" + localFile + "." + tmp_uuid_local + " , " + - "linking to: " + localFile + "." + nimbusBlobVersion); - File uuid_symlink = new File(localFile + "." + tmp_uuid_local); - - Files.createSymbolicLink(uuid_symlink.toPath(), - Paths.get(ServerUtils.constructBlobWithVersionFileName(localFile.toString(), - nimbusBlobVersion))); - File current_symlink = new File(ServerUtils.constructBlobCurrentSymlinkName( - localFile.toString())); - Files.move(uuid_symlink.toPath(), current_symlink.toPath(), ATOMIC_MOVE); - } catch (IOException e) { - // if we fail after writing the version file but before we move current link we need to - // restore the old version to the file - try { - PrintWriter restoreWriter = new PrintWriter( - new BufferedWriter(new FileWriter(localVersionFile, false))); - restoreWriter.println(oldVersion); - restoreWriter.close(); - } catch (IOException ignore) {} - throw e; - } - - String oldBlobFile = localFile + "." + oldVersion; - try { - // Remove the old version. Note that if a number of processes have that file open, - // the OS will keep the old blob file around until they all close the handle and only - // then deletes it. No new process will open the old blob, since the users will open the - // blob through the "blob.current" symlink, which always points to the latest version of - // a blob. Remove the old version after the current symlink is updated as to not affect - // anyone trying to read it. - if ((oldVersion != -1) && (oldVersion != nimbusBlobVersion)) { - LOG.info("Removing an old blob file:" + oldBlobFile); - Files.delete(Paths.get(oldBlobFile)); - } - } catch (IOException e) { - // At this point we have downloaded everything and moved symlinks. If the remove of - // old fails just log an error - LOG.error("Exception removing old blob version: " + oldBlobFile); - } - - break; - } catch (AuthorizationException ae) { - // we consider this non-retriable exceptions - if (out != null) { - out.close(); - } - new File(downloadFile).delete(); - throw ae; - } catch (IOException | KeyNotFoundException e) { - if (out != null) { - out.close(); - } - if (writer != null) { - writer.close(); - } - new File(downloadFile).delete(); - if (uncompress) { - try { - FileUtils.deleteDirectory(new File(localFileWithVersion)); - } catch (IOException ignore) {} - } - if (!isUpdate) { - // don't want to remove existing version file if its an update - new File(localVersionFile).delete(); - } - - if (numTries < blobDownloadRetries) { - LOG.error("Failed to download blob, retrying", e); - } else { - throw e; - } - } - } - return new LocalizedResource(key, localizedPath, uncompress); - } finally { - if(blobstore != null) { - blobstore.shutdown(); - } - } - } - - private void setBlobPermissions(Map conf, String user, String path) - throws IOException { - - if (!ObjectReader.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) { - return; - } - String wlCommand = ObjectReader.getString(conf.get(Config.SUPERVISOR_WORKER_LAUNCHER), ""); - if (wlCommand.isEmpty()) { - String stormHome = System.getProperty("storm.home"); - wlCommand = stormHome + "/bin/worker-launcher"; - } - List command = new ArrayList<>(Arrays.asList(wlCommand, user, "blob", path)); - - String[] commandArray = command.toArray(new String[command.size()]); - ShellUtils.ShellCommandExecutor shExec = new ShellUtils.ShellCommandExecutor(commandArray); - LOG.info("Setting blob permissions, command: {}", Arrays.toString(commandArray)); - - try { - shExec.execute(); - LOG.debug("output: {}", shExec.getOutput()); - } catch (ShellUtils.ExitCodeException e) { - int exitCode = shExec.getExitCode(); - LOG.warn("Exit code from worker-launcher is : " + exitCode, e); - LOG.debug("output: {}", shExec.getOutput()); - throw new IOException("Setting blob permissions failed" + - " (exitCode=" + exitCode + ") with output: " + shExec.getOutput(), e); - } - } - private interface ConsumePathAndId { void accept(Path path, String topologyId) throws IOException; } @@ -1094,13 +635,20 @@ private void forEachTopologyDistDir(ConsumePathAndId consumer) throws IOExceptio synchronized void cleanup() { LocalizedResourceRetentionSet toClean = new LocalizedResourceRetentionSet(cacheTargetSize); // need one large set of all and then clean via LRU - for (LocalizedResourceSet t : userRsrc.values()) { - toClean.addResources(t); - LOG.debug("Resources to be cleaned after adding {} : {}", t.getUser(), toClean); + for (Map.Entry> t : userArchives.entrySet()) { + toClean.addResources(t.getValue()); + LOG.debug("Resources to be cleaned after adding {} archives : {}", t.getKey(), toClean); + } + + for (Map.Entry> t : userFiles.entrySet()) { + toClean.addResources(t.getValue()); + LOG.debug("Resources to be cleaned after adding {} files : {}", t.getKey(), toClean); } toClean.addResources(topologyBlobs); - toClean.cleanup(); + try (ClientBlobStore store = getClientBlobStore()) { + toClean.cleanup(store); + } HashSet safeTopologyIds = new HashSet<>(); for (String blobKey : topologyBlobs.keySet()) { @@ -1109,6 +657,7 @@ synchronized void cleanup() { //Deleting this early does not hurt anything topologyBasicDownloaded.keySet().removeIf(topoId -> !safeTopologyIds.contains(topoId)); + blobPending.keySet().removeIf(topoId -> !safeTopologyIds.contains(topoId)); try { forEachTopologyDistDir((p, topologyId) -> { @@ -1121,19 +670,21 @@ synchronized void cleanup() { } LOG.debug("Resource cleanup: {}", toClean); - for (LocalizedResourceSet t : userRsrc.values()) { - if (t.getSize() == 0) { - String user = t.getUser(); + Set allUsers = new HashSet<>(userArchives.keySet()); + allUsers.addAll(userFiles.keySet()); + for (String user: allUsers) { + ConcurrentMap filesForUser = userFiles.get(user); + ConcurrentMap archivesForUser = userArchives.get(user); + if ((filesForUser == null || filesForUser.size() == 0) + && (archivesForUser == null || archivesForUser.size() == 0)) { LOG.debug("removing empty set: {}", user); - File userFileCacheDir = getLocalUserFileCacheDir(user); - getCacheDirForFiles(userFileCacheDir).delete(); - getCacheDirForArchives(userFileCacheDir).delete(); - getLocalUserFileCacheDir(user).delete(); - boolean dirsRemoved = getLocalUserDir(user).delete(); - // to catch race with update thread - if (dirsRemoved) { - userRsrc.remove(user); + try { + LocalizedResource.completelyRemoveUnusedUser(localBaseDir, user); + userFiles.remove(user); + userArchives.remove(user); + } catch (IOException e) { + LOG.error("Error trying to delete cached user files", e); } } } diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocalDownloadedResource.java b/storm-server/src/main/java/org/apache/storm/localizer/LocalDownloadedResource.java deleted file mode 100644 index 2d2e87983b0..00000000000 --- a/storm-server/src/main/java/org/apache/storm/localizer/LocalDownloadedResource.java +++ /dev/null @@ -1,86 +0,0 @@ -/** - * 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.storm.localizer; - -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.CompletableFuture; - -import org.apache.storm.generated.LocalAssignment; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Used for accounting to keep track of who is waiting for specific resources to be downloaded. - */ -public class LocalDownloadedResource { - private static final Logger LOG = LoggerFactory.getLogger(LocalDownloadedResource.class); - private final CompletableFuture pending; - private final Set references; - private boolean isDone; - - - public LocalDownloadedResource(CompletableFuture pending) { - this.pending = pending; - references = new HashSet<>(); - isDone = false; - } - - /** - * Reserve the resources - * @param port the port this is for - * @param la the assignment this is for - * @return a future that can be used to track it being downloaded. - */ - public synchronized CompletableFuture reserve(int port, LocalAssignment la) { - PortAndAssignment pna = new PortAndAssignment(port, la); - if (!references.add(pna)) { - LOG.warn("Resources {} already reserved {} for this topology", pna, references); - } - return pending; - } - - /** - * Release a port from the reference count, and update isDone if all is done. - * @param port the port to release - * @param la the assignment to release - * @return true if the port was being counted else false - */ - public synchronized boolean release(int port, LocalAssignment la) { - PortAndAssignment pna = new PortAndAssignment(port, la); - boolean ret = references.remove(pna); - if (ret && references.isEmpty()) { - isDone = true; - } - return ret; - } - - /** - * Is this has been cleaned up completely. - * @return true if it is done else false - */ - public synchronized boolean isDone() { - return isDone; - } - - @Override - public String toString() { - return references.toString(); - } -} diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocalResource.java b/storm-server/src/main/java/org/apache/storm/localizer/LocalResource.java index f38f6f6768f..862349b664a 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/LocalResource.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/LocalResource.java @@ -15,30 +15,43 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.storm.localizer; /** - * Local Resource requested by the topology + * Local Resource requested by the topology. */ public class LocalResource { - private String _blobKey; - private boolean _uncompress; + private final boolean needsCallback; + private final String blobKey; + private final boolean uncompress; + + /** + * Constructor. + * @param keyname the key of the blob to download. + * @param uncompress should the blob be uncompressed or not. + * @param needsCallback if the blobs changes should a callback happen so the worker is restarted. + */ + public LocalResource(String keyname, boolean uncompress, boolean needsCallback) { + blobKey = keyname; + this.uncompress = uncompress; + this.needsCallback = needsCallback; + } - public LocalResource(String keyname, boolean uncompress) { - _blobKey = keyname; - _uncompress = uncompress; - } + public String getBlobName() { + return blobKey; + } - public String getBlobName() { - return _blobKey; - } + public boolean shouldUncompress() { + return uncompress; + } - public boolean shouldUncompress() { - return _uncompress; - } + public boolean needsCallback() { + return needsCallback; + } - @Override - public String toString() { - return "Key: " + _blobKey + " uncompress: " + _uncompress; - } + @Override + public String toString() { + return "Key: " + blobKey + " uncompress: " + uncompress; + } } diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java index 7241976b24e..4f01c304faf 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java @@ -15,129 +15,434 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.storm.localizer; +import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; + +import com.google.common.annotations.VisibleForTesting; +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.FileWriter; +import java.io.IOException; +import java.io.PrintWriter; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.commons.io.FileUtils; +import org.apache.storm.Config; +import org.apache.storm.blobstore.ClientBlobStore; +import org.apache.storm.blobstore.InputStreamWithMeta; +import org.apache.storm.daemon.supervisor.IAdvancedFSOps; +import org.apache.storm.generated.AuthorizationException; +import org.apache.storm.generated.KeyNotFoundException; +import org.apache.storm.generated.ReadableBlobMeta; +import org.apache.storm.utils.ObjectReader; import org.apache.storm.utils.ServerUtils; +import org.apache.storm.utils.ShellUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; - /** * Represents a resource that is localized on the supervisor. * A localized resource has a .current symlink to the current version file which is named * filename.{current version}. There is also a filename.version which contains the latest version. */ -public class LocalizedResource { - public static final Logger LOG = LoggerFactory.getLogger(LocalizedResource.class); - - // filesystem path to the resource - private final String _localPath; - private final String _versionFilePath; - private final String _symlinkPath; - private final String _key; - private final boolean _uncompressed; - // _size of the resource - private long _size = -1; - // queue of topologies referencing resource - private final Set _ref; - // last access time of the resource -> increment when topology finishes using it - private final AtomicLong _lastAccessTime = new AtomicLong(currentTime()); - - public LocalizedResource(String key, String fileLoc, boolean uncompressed) { - _ref = new HashSet(); - _localPath = fileLoc; - _versionFilePath = ServerUtils.constructVersionFileName(fileLoc); - _symlinkPath = ServerUtils.constructBlobCurrentSymlinkName(fileLoc); - _uncompressed = uncompressed; - _key = key; - // we trust that the file exists - _size = ServerUtils.getDU(new File(getFilePathWithVersion())); - LOG.debug("size of {} is: {}", fileLoc, _size); - } - - // create local resource and add reference - public LocalizedResource(String key, String fileLoc, boolean uncompressed, String topo) { - this(key, fileLoc, uncompressed); - _ref.add(topo); - } - - public boolean isUncompressed() { - return _uncompressed; - } - - public String getKey() { - return _key; - } - - public String getCurrentSymlinkPath() { - return _symlinkPath; - } - - public String getVersionFilePath() { - return _versionFilePath; - } - - public String getFilePathWithVersion() { - long version = ServerUtils.localVersionOfBlob(_localPath); - return ServerUtils.constructBlobWithVersionFileName(_localPath, version); - } - - public String getFilePath() { - return _localPath; - } - - public void addReference(String topo) { - _ref.add(topo); - } - - public void removeReference(String topo) { - if (!_ref.remove(topo)) { - LOG.warn("Tried to remove a reference to a topology that doesn't use this resource"); - } - setTimestamp(); - } - - // The last access time is only valid if the resource doesn't have any references. - public long getLastAccessTime() { - return _lastAccessTime.get(); - } - - // for testing - protected void setSize(long size) { - _size = size; - } - - public long getSize() { - return _size; - } - - private void setTimestamp() { - _lastAccessTime.set(currentTime()); - } - - public int getRefCount() { - return _ref.size(); - } - - private long currentTime() { - return System.nanoTime(); - } - - @Override - public boolean equals(Object other) { - if (other instanceof LocalizedResource) { - LocalizedResource l = (LocalizedResource)other; - return _key.equals(l._key) && _uncompressed == l._uncompressed && _localPath.equals(l._localPath); - } - return false; - } - - @Override - public int hashCode() { - return _key.hashCode() + Boolean.hashCode(_uncompressed) + _localPath.hashCode(); - } +public class LocalizedResource extends LocallyCachedBlob { + private static final Logger LOG = LoggerFactory.getLogger(LocalizedResource.class); + @VisibleForTesting + static final String CURRENT_BLOB_SUFFIX = ".current"; + @VisibleForTesting + static final String BLOB_VERSION_SUFFIX = ".version"; + @VisibleForTesting + static final String FILECACHE = "filecache"; + @VisibleForTesting + static final String USERCACHE = "usercache"; + // sub directories to store either files or uncompressed archives respectively + @VisibleForTesting + static final String FILESDIR = "files"; + @VisibleForTesting + static final String ARCHIVESDIR = "archives"; + private static final String TO_UNCOMPRESS = "_tmp_"; + + private static Path constructVersionFileName(Path baseDir, String key) { + return baseDir.resolve(key + BLOB_VERSION_SUFFIX); + } + + @VisibleForTesting + static long localVersionOfBlob(Path versionFile) { + long currentVersion = -1; + if (Files.exists(versionFile) && !(Files.isDirectory(versionFile))) { + try (BufferedReader br = new BufferedReader(new FileReader(versionFile.toFile()))) { + String line = br.readLine(); + currentVersion = Long.parseLong(line); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return currentVersion; + } + + private static Path constructBlobCurrentSymlinkName(Path baseDir, String key) { + return baseDir.resolve(key + CURRENT_BLOB_SUFFIX); + } + + private static Path constructBlobWithVersionFileName(Path baseDir, String key, long version) { + return baseDir.resolve(key + "." + version); + } + + static Collection getLocalizedUsers(Path localBaseDir) throws IOException { + Path userCacheDir = getUserCacheDir(localBaseDir); + if (!Files.exists(userCacheDir)) { + return Collections.emptyList(); + } + return Files.list(userCacheDir).map((p) -> p.getFileName().toString()).collect(Collectors.toList()); + } + + static void completelyRemoveUnusedUser(Path localBaseDir, String user) throws IOException { + Path userFileCacheDir = getLocalUserFileCacheDir(localBaseDir, user); + // baseDir/supervisor/usercache/user1/filecache/files + Files.deleteIfExists(getCacheDirForFiles(userFileCacheDir)); + // baseDir/supervisor/usercache/user1/filecache/archives + Files.deleteIfExists(getCacheDirForArchives(userFileCacheDir)); + // baseDir/supervisor/usercache/user1/filecache + Files.deleteIfExists(userFileCacheDir); + // baseDir/supervisor/usercache/user1 + Files.deleteIfExists(getLocalUserDir(localBaseDir, user)); + } + + static List getLocalizedArchiveKeys(Path localBaseDir, String user) throws IOException { + Path dir = getCacheDirForArchives(getLocalUserFileCacheDir(localBaseDir, user)); + return readKeysFromDir(dir); + } + + static List getLocalizedFileKeys(Path localBaseDir, String user) throws IOException { + Path dir = getCacheDirForFiles(getLocalUserFileCacheDir(localBaseDir, user)); + return readKeysFromDir(dir); + } + + // Looks for files in the directory with .current suffix + private static List readKeysFromDir(Path dir) throws IOException { + if (!Files.exists(dir)) { + return Collections.emptyList(); + } + return Files.list(dir) + .map((p) -> p.getFileName().toString()) + .filter((name) -> name.toLowerCase().endsWith(CURRENT_BLOB_SUFFIX)) + .map((key) -> { + int p = key.lastIndexOf('.'); + if (p > 0) { + key = key.substring(0, p); + } + return key; + }) + .collect(Collectors.toList()); + } + + // baseDir/supervisor/usercache/ + private static Path getUserCacheDir(Path localBaseDir) { + return localBaseDir.resolve(USERCACHE); + } + + // baseDir/supervisor/usercache/user1/ + static Path getLocalUserDir(Path localBaseDir, String userName) { + return getUserCacheDir(localBaseDir).resolve(userName); + } + + // baseDir/supervisor/usercache/user1/filecache + static Path getLocalUserFileCacheDir(Path localBaseDir, String userName) { + return getLocalUserDir(localBaseDir, userName).resolve(FILECACHE); + } + + // baseDir/supervisor/usercache/user1/filecache/files + private static Path getCacheDirForFiles(Path dir) { + return dir.resolve(FILESDIR); + } + + // get the directory to put uncompressed archives in + // baseDir/supervisor/usercache/user1/filecache/archives + private static Path getCacheDirForArchives(Path dir) { + return dir.resolve(ARCHIVESDIR); + } + + // filesystem path to the resource + private final Path baseDir; + private final Path versionFilePath; + private final Path symlinkPath; + private final boolean uncompressed; + private final IAdvancedFSOps fsOps; + private final String user; + // size of the resource + private long size = -1; + private final Map conf; + + LocalizedResource(String key, Path localBaseDir, boolean uncompressed, IAdvancedFSOps fsOps, Map conf, + String user) { + super(key + (uncompressed ? " archive" : " file"), key); + Path base = getLocalUserFileCacheDir(localBaseDir, user); + this.baseDir = uncompressed ? getCacheDirForArchives(base) : getCacheDirForFiles(base); + this.conf = conf; + this.user = user; + this.fsOps = fsOps; + versionFilePath = constructVersionFileName(baseDir, key); + symlinkPath = constructBlobCurrentSymlinkName(baseDir, key); + this.uncompressed = uncompressed; + //Set the size in case we are recovering an already downloaded object + setSize(); + } + + Path getCurrentSymlinkPath() { + return symlinkPath; + } + + @VisibleForTesting + Path getFilePathWithVersion() { + return constructBlobWithVersionFileName(baseDir, getKey(), getLocalVersion()); + } + + private void setSize() { + // we trust that the file exists + Path withVersion = getFilePathWithVersion(); + size = ServerUtils.getDU(withVersion.toFile()); + LOG.debug("size of {} is: {}", withVersion, size); + } + + @VisibleForTesting + protected void setSize(long size) { + this.size = size; + } + + @Override + public long getLocalVersion() { + return localVersionOfBlob(versionFilePath); + } + + @Override + public long getRemoteVersion(ClientBlobStore store) throws KeyNotFoundException, AuthorizationException { + return ServerUtils.nimbusVersionOfBlob(getKey(), store); + } + + @Override + public long downloadToTempLocation(ClientBlobStore store) throws IOException, KeyNotFoundException, AuthorizationException { + String key = getKey(); + ReadableBlobMeta meta = store.getBlobMeta(key); + if (!ServerUtils.canUserReadBlob(meta, user, conf)) { + throw new AuthorizationException(user + " does not have READ access to " + key); + } + long version; + Path downloadFile; + Path finalLocation; + try (InputStreamWithMeta in = store.getBlob(key)) { + version = in.getVersion(); + finalLocation = constructBlobWithVersionFileName(baseDir, getKey(), version); + if (uncompressed) { + // we need to download to temp file and then unpack into the one requested + downloadFile = tmpOutputLocation(); + } else { + downloadFile = finalLocation; + } + byte[] buffer = new byte[1024]; + int len; + LOG.debug("Downloading {} to {}", key, downloadFile); + Path parent = downloadFile.getParent(); + if (!Files.exists(parent)) { + Files.createDirectory(parent); + } + try (FileOutputStream out = new FileOutputStream(downloadFile.toFile())) { + while ((len = in.read(buffer)) >= 0) { + out.write(buffer, 0, len); + } + } + } + if (uncompressed) { + ServerUtils.unpack(downloadFile.toFile(), finalLocation.toFile()); + LOG.debug("Uncompressed {} to: {}", downloadFile, finalLocation); + } + setBlobPermissions(conf, user, finalLocation); + return version; + } + + @Override + public void commitNewVersion(long version) throws IOException { + String key = getKey(); + LOG.info("Blob: {} updated to version {} from version {}", key, version, getLocalVersion()); + Path localVersionFile = versionFilePath; + // The false parameter ensures overwriting the version file, not appending + try (PrintWriter writer = new PrintWriter( + new BufferedWriter(new FileWriter(localVersionFile.toFile(), false)))) { + writer.println(version); + } + setBlobPermissions(conf, user, localVersionFile); + + // Update the key.current symlink. First create tmp symlink and do + // move of tmp to current so that the operation is atomic. + Path tmpSymlink = tmpSymlinkLocation(); + Path targetOfSymlink = constructBlobWithVersionFileName(baseDir, getKey(), version); + LOG.debug("Creating a symlink @{} linking to: {}", tmpSymlink, targetOfSymlink); + Files.createSymbolicLink(tmpSymlink, targetOfSymlink); + + Path currentSymLink = getCurrentSymlinkPath(); + Files.move(tmpSymlink, currentSymLink, ATOMIC_MOVE); + //Update the size of the objects + setSize(); + } + + private void setBlobPermissions(Map conf, String user, Path path) + throws IOException { + + if (!ObjectReader.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) { + return; + } + String wlCommand = ObjectReader.getString(conf.get(Config.SUPERVISOR_WORKER_LAUNCHER), ""); + if (wlCommand.isEmpty()) { + String stormHome = System.getProperty("storm.home"); + wlCommand = stormHome + "/bin/worker-launcher"; + } + List command = new ArrayList<>(Arrays.asList(wlCommand, user, "blob", path.toString())); + + String[] commandArray = command.toArray(new String[command.size()]); + ShellUtils.ShellCommandExecutor shExec = new ShellUtils.ShellCommandExecutor(commandArray); + LOG.debug("Setting blob permissions, command: {}", Arrays.toString(commandArray)); + + try { + shExec.execute(); + LOG.debug("output: {}", shExec.getOutput()); + } catch (ShellUtils.ExitCodeException e) { + int exitCode = shExec.getExitCode(); + LOG.warn("Exit code from worker-launcher is: {}", exitCode, e); + LOG.debug("output: {}", shExec.getOutput()); + throw new IOException("Setting blob permissions failed" + + " (exitCode=" + exitCode + ") with output: " + shExec.getOutput(), e); + } + } + + private Path tmpOutputLocation() { + return baseDir.resolve(Paths.get(LocalizedResource.TO_UNCOMPRESS + getKey())); + } + + private Path tmpSymlinkLocation() { + return baseDir.resolve(Paths.get(LocalizedResource.TO_UNCOMPRESS + getKey() + CURRENT_BLOB_SUFFIX)); + } + + private static final Pattern VERSION_FILE_PATTERN = Pattern.compile("^(.+)\\.(\\d+)$"); + + @Override + public void cleanupOrphanedData() throws IOException { + //There are a few possible files that we would want to clean up + //baseDir + "/" + "_tmp_" + baseName + //baseDir + "/" + "_tmp_" + baseName + ".current" + //baseDir + "/" + baseName. + //baseDir + "/" + baseName.current + //baseDir + "/" + baseName.version + //In general we always want to delete the _tmp_ files if they are there. + + Path tmpOutput = tmpOutputLocation(); + Files.deleteIfExists(tmpOutput); + Path tmpSym = tmpSymlinkLocation(); + Files.deleteIfExists(tmpSym); + + try { + String baseName = getKey(); + long version = getLocalVersion(); + Path current = getCurrentSymlinkPath(); + + //If .current and .version do not match, we roll back the .version file to match + // what .current is pointing to. + if (Files.exists(current) && Files.isSymbolicLink(current)) { + Path versionFile = Files.readSymbolicLink(current); + Matcher m = VERSION_FILE_PATTERN.matcher(versionFile.getFileName().toString()); + if (m.matches()) { + long foundVersion = Long.valueOf(m.group(2)); + if (foundVersion != version) { + LOG.error("{} does not match the version file so fix the version file", current); + //The versions are different so roll back to whatever current is + try (PrintWriter restoreWriter = new PrintWriter( + new BufferedWriter(new FileWriter(versionFilePath.toFile(), false)))) { + restoreWriter.println(foundVersion); + } + version = foundVersion; + } + } + } + + // Finally delete any baseName. files that are not pointed to by the current version + final long finalVersion = version; + LOG.debug("Looking to clean up after {} in {}", getKey(), baseDir); + try (DirectoryStream ds = fsOps.newDirectoryStream(baseDir, (path) -> { + Matcher m = VERSION_FILE_PATTERN.matcher(path.getFileName().toString()); + if (m.matches()) { + long foundVersion = Long.valueOf(m.group(2)); + return m.group(1).equals(baseName) && foundVersion != finalVersion; + } + return false; + })) { + for (Path p : ds) { + LOG.info("Cleaning up old localized resource file {}", p); + if (Files.isDirectory(p)) { + FileUtils.deleteDirectory(p.toFile()); + } else { + fsOps.deleteIfExists(p.toFile()); + } + } + } + } catch (NoSuchFileException e) { + LOG.warn("Nothing to cleanup with badeDir {} even though we expected there to be something there", baseDir); + } + } + + @Override + public void completelyRemove() throws IOException { + Path fileWithVersion = getFilePathWithVersion(); + Path currentSymLink = getCurrentSymlinkPath(); + + if (uncompressed) { + if (Files.exists(fileWithVersion)) { + // this doesn't follow symlinks, which is what we want + FileUtils.deleteDirectory(fileWithVersion.toFile()); + } + } else { + Files.deleteIfExists(fileWithVersion); + } + Files.deleteIfExists(currentSymLink); + Files.deleteIfExists(versionFilePath); + } + + @Override + public long getSizeOnDisk() { + return size; + } + + @Override + public boolean isFullyDownloaded() { + return Files.exists(getFilePathWithVersion()) + && Files.exists(getCurrentSymlinkPath()) + && Files.exists(versionFilePath); + } + + @Override + public boolean equals(Object other) { + if (other instanceof LocalizedResource) { + LocalizedResource l = (LocalizedResource)other; + return getKey().equals(l.getKey()) && uncompressed == l.uncompressed && baseDir.equals(l.baseDir); + } + return false; + } + + @Override + public int hashCode() { + return getKey().hashCode() + Boolean.hashCode(uncompressed) + baseDir.hashCode(); + } } diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java index 826bf9869ce..936dbc11e4b 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java @@ -15,23 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.storm.localizer; import com.google.common.annotations.VisibleForTesting; -import java.nio.file.Path; -import java.util.concurrent.ConcurrentHashMap; -import org.apache.commons.io.FileUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; import java.util.Comparator; import java.util.Iterator; import java.util.Map; import java.util.SortedMap; import java.util.TreeMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.storm.blobstore.ClientBlobStore; +import org.apache.storm.generated.AuthorizationException; +import org.apache.storm.generated.KeyNotFoundException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A set of resources that we can look at to see which ones we retain and which ones should be @@ -39,24 +37,22 @@ */ public class LocalizedResourceRetentionSet { public static final Logger LOG = LoggerFactory.getLogger(LocalizedResourceRetentionSet.class); - private long delSize; private long currentSize; // targetSize in Bytes private long targetSize; @VisibleForTesting - final SortedMap noReferences; - private int resourceCount = 0; + final SortedMap> noReferences; LocalizedResourceRetentionSet(long targetSize) { this(targetSize, new LRUComparator()); } - LocalizedResourceRetentionSet(long targetSize, Comparator cmp) { + LocalizedResourceRetentionSet(long targetSize, Comparator cmp) { this(targetSize, new TreeMap<>(cmp)); } LocalizedResourceRetentionSet(long targetSize, - SortedMap retain) { + SortedMap> retain) { this.noReferences = retain; this.targetSize = targetSize; } @@ -66,287 +62,88 @@ protected int getSizeWithNoReferences() { return noReferences.size(); } - protected void addResourcesForSet(Iterator setIter, LocalizedResourceSet set) { - CleanableLocalizedResourceSet cleanset = new CleanableLocalizedResourceSet(set); - for (Iterator iter = setIter; setIter.hasNext(); ) { - LocalizedResource lrsrc = iter.next(); - currentSize += lrsrc.getSize(); - resourceCount ++; - if (lrsrc.getRefCount() > 0) { - // always retain resources in use - continue; - } - noReferences.put(new LocalizedBlobComparableResource(lrsrc), cleanset); - } - } - - public void addResources(LocalizedResourceSet set) { - addResourcesForSet(set.getLocalFilesIterator(), set); - addResourcesForSet(set.getLocalArchivesIterator(), set); - } - - public void addResources(ConcurrentHashMap blobs) { - CleanableLocalizedLocallyCachedBlob set = new CleanableLocalizedLocallyCachedBlob(blobs); + /** + * Add blobs to be checked if they can be deleted. + * @param blobs a map of blob name to the blob object. The blobs in this map will be deleted from the map + * if they are deleted on disk too. + */ + public void addResources(ConcurrentMap blobs) { for (LocallyCachedBlob b: blobs.values()) { currentSize += b.getSizeOnDisk(); - resourceCount ++; if (b.isUsed()) { + LOG.debug("NOT going to clean up {}, {} depends on it", b.getKey(), b.getDependencies()); // always retain resources in use continue; } - LocallyCachedBlobComparableResource cb = new LocallyCachedBlobComparableResource(b); - noReferences.put(cb, set); + LOG.debug("Possibly going to clean up {} ts {} size {}", b.getKey(), b.getLastUsed(), b.getSizeOnDisk()); + noReferences.put(b, blobs); } } - public void cleanup() { + /** + * Actually cleanup the blobs to try and get below the target cache size. + * @param store the blobs store client used to check if the blob has been deleted from the blobstore. If it has, the blob will be + * deleted even if the cache is not over the target size. + */ + public void cleanup(ClientBlobStore store) { LOG.debug("cleanup target size: {} current size is: {}", targetSize, currentSize); - for (Iterator> i = - noReferences.entrySet().iterator(); - currentSize - delSize > targetSize && i.hasNext();) { - Map.Entry rsrc = i.next(); - ComparableResource resource = rsrc.getKey(); - CleanableResourceSet set = rsrc.getValue(); - if (resource != null && set.remove(resource)) { - if (set.deleteUnderlyingResource(resource)) { - delSize += resource.getSize(); - LOG.info("deleting: {} with size of: {}", resource.getNameForDebug(), resource.getSize()); + long bytesOver = currentSize - targetSize; + //First delete everything that no longer exists... + for (Iterator>> i = noReferences.entrySet().iterator(); + i.hasNext();) { + Map.Entry> rsrc = i.next(); + LocallyCachedBlob resource = rsrc.getKey(); + try { + resource.getRemoteVersion(store); + } catch (AuthorizationException e) { + //Ignored + } catch (KeyNotFoundException e) { + //The key was removed so we should delete it too. + Map set = rsrc.getValue(); + if (removeBlob(resource, set)) { + bytesOver -= resource.getSizeOnDisk(); + LOG.info("Deleted blob: {} (KEY NOT FOUND).", resource.getKey()); i.remove(); - } else { - // since it failed to delete add it back so it gets retried - set.add(resource.getKey(), resource); } } } - } - - @VisibleForTesting - public boolean deleteResource(CleanableResourceSet set, ComparableResource resource) { - return set.deleteUnderlyingResource(resource); - } - - public long getCurrentSize() { - return currentSize; - } - - public int getResourceCount() { - return resourceCount; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("Cache: ").append(currentSize).append(", "); - sb.append("Deleted: ").append(delSize); - return sb.toString(); - } - - interface ComparableResource { - long getLastAccessTime(); - - long getSize(); - - String getNameForDebug(); - - String getKey(); - } - - interface CleanableResourceSet { - boolean remove(ComparableResource resource); - - void add(String key, ComparableResource resource); - - boolean deleteUnderlyingResource(ComparableResource resource); - } - - public static class LocallyCachedBlobComparableResource implements ComparableResource { - private final LocallyCachedBlob blob; - - public LocallyCachedBlobComparableResource(LocallyCachedBlob blob) { - this.blob = blob; - } - - @Override - public long getLastAccessTime() { - return blob.getLastUsed(); - } - - @Override - public long getSize() { - return blob.getSizeOnDisk(); - } - - @Override - public String getNameForDebug() { - return blob.getKey(); - } - - @Override - public String getKey() { - return blob.getKey(); - } - - @Override - public String toString() { - return blob.toString(); - } - @Override - public boolean equals(Object other) { - if (other instanceof LocallyCachedBlobComparableResource) { - return blob.equals(((LocallyCachedBlobComparableResource) other).blob); + for (Iterator>> i = noReferences.entrySet().iterator(); + bytesOver > 0 && i.hasNext();) { + Map.Entry> rsrc = i.next(); + LocallyCachedBlob resource = rsrc.getKey(); + Map set = rsrc.getValue(); + if (removeBlob(resource, set)) { + bytesOver -= resource.getSizeOnDisk(); + LOG.info("Deleted blob: {} (OVER SIZE LIMIT).", resource.getKey()); + i.remove(); } - return false; - } - - @Override - public int hashCode() { - return blob.hashCode(); } } - private static class CleanableLocalizedLocallyCachedBlob implements CleanableResourceSet { - private final ConcurrentHashMap blobs; - - public CleanableLocalizedLocallyCachedBlob(ConcurrentHashMap blobs) { - this.blobs = blobs; - } - - @Override - public boolean remove(ComparableResource resource) { - if (!(resource instanceof LocallyCachedBlobComparableResource)) { - throw new IllegalStateException(resource + " must be a LocallyCachedBlobComparableResource"); - } - LocallyCachedBlob blob = ((LocallyCachedBlobComparableResource)resource).blob; - synchronized (blob) { - if (!blob.isUsed()) { - try { - blob.completelyRemove(); - } catch (Exception e) { - LOG.warn("Tried to remove {} but failed with", blob, e); - } - blobs.remove(blob.getKey()); - return true; + private boolean removeBlob(LocallyCachedBlob blob, Map blobs) { + synchronized (blob) { + if (!blob.isUsed()) { + try { + blob.completelyRemove(); + } catch (Exception e) { + LOG.warn("Tried to remove {} but failed with", blob, e); } - return false; - } - } - - @Override - public void add(String key, ComparableResource resource) { - ///NOOP not used - } - - @Override - public boolean deleteUnderlyingResource(ComparableResource resource) { - //NOOP not used - return true; - } - } - - private static class LocalizedBlobComparableResource implements ComparableResource { - private final LocalizedResource resource; - - private LocalizedBlobComparableResource(LocalizedResource resource) { - this.resource = resource; - } - - @Override - public long getLastAccessTime() { - return resource.getLastAccessTime(); - } - - @Override - public long getSize() { - return resource.getSize(); - } - - @Override - public String getNameForDebug() { - return resource.getFilePath(); - } - - @Override - public String getKey() { - return resource.getKey(); - } - - @Override - public String toString() { - return resource.getKey() + " at " + resource.getFilePathWithVersion(); - } - - @Override - public boolean equals(Object other) { - if (other instanceof LocalizedBlobComparableResource) { - return resource.equals(((LocalizedBlobComparableResource) other).resource); + blobs.remove(blob.getKey()); + return true; } return false; } - - @Override - public int hashCode() { - return resource.hashCode(); - } } - private static class CleanableLocalizedResourceSet implements CleanableResourceSet { - private final LocalizedResourceSet set; - - public CleanableLocalizedResourceSet(LocalizedResourceSet set) { - this.set = set; - } - - @Override - public boolean remove(ComparableResource resource) { - if (!(resource instanceof LocalizedBlobComparableResource)) { - throw new IllegalStateException(resource + " must be a LocalizedBlobComparableResource"); - } - return set.remove(((LocalizedBlobComparableResource)resource).resource); - } - - @Override - public void add(String key, ComparableResource resource) { - if (!(resource instanceof LocalizedBlobComparableResource)) { - throw new IllegalStateException(resource + " must be a LocalizedBlobComparableResource"); - } - LocalizedResource r = ((LocalizedBlobComparableResource)resource).resource; - set.add(key, r, r.isUncompressed()); - } - - @Override - public boolean deleteUnderlyingResource(ComparableResource resource) { - if (resource instanceof LocalizedBlobComparableResource) { - LocalizedResource lr = ((LocalizedBlobComparableResource) resource).resource; - try { - Path fileWithVersion = new File(lr.getFilePathWithVersion()).toPath(); - Path currentSymLink = new File(lr.getCurrentSymlinkPath()).toPath(); - Path versionFile = new File(lr.getVersionFilePath()).toPath(); - - if (lr.isUncompressed()) { - if (Files.exists(fileWithVersion)) { - // this doesn't follow symlinks, which is what we want - FileUtils.deleteDirectory(fileWithVersion.toFile()); - } - } else { - Files.deleteIfExists(fileWithVersion); - } - Files.deleteIfExists(currentSymLink); - Files.deleteIfExists(versionFile); - return true; - } catch (IOException e) { - LOG.warn("Could not delete: {}", resource.getNameForDebug(), e); - } - return false; - } else { - throw new IllegalArgumentException("Don't know how to handle a " + resource.getClass()); - } - } + @Override + public String toString() { + return "Cache: " + currentSize; } - static class LRUComparator implements Comparator { - public int compare(ComparableResource r1, ComparableResource r2) { - long ret = r1.getLastAccessTime() - r2.getLastAccessTime(); + static class LRUComparator implements Comparator { + public int compare(LocallyCachedBlob r1, LocallyCachedBlob r2) { + long ret = r1.getLastUsed() - r2.getLastUsed(); if (0 == ret) { return System.identityHashCode(r1) - System.identityHashCode(r2); } diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceSet.java b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceSet.java deleted file mode 100644 index 62d5b2fcfc0..00000000000 --- a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceSet.java +++ /dev/null @@ -1,101 +0,0 @@ -/** - * 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.storm.localizer; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Iterator; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -/** - * Set of localized resources for a specific user. - */ -public class LocalizedResourceSet { - - public static final Logger LOG = LoggerFactory.getLogger(LocalizedResourceSet.class); - // Key to LocalizedResource mapping for files - private final ConcurrentMap _localrsrcFiles; - // Key to LocalizedResource mapping for files to be uncompressed - private final ConcurrentMap _localrsrcArchives; - private String _user; - - LocalizedResourceSet(String user) { - this._localrsrcFiles = new ConcurrentHashMap(); - this._localrsrcArchives = new ConcurrentHashMap(); - _user = user; - } - - public String getUser() { - return _user; - } - - public int getSize() { - return _localrsrcFiles.size() + _localrsrcArchives.size(); - } - - public LocalizedResource get(String name, boolean uncompress) { - if (uncompress) { - return _localrsrcArchives.get(name); - } - return _localrsrcFiles.get(name); - } - - public void putIfAbsent(String resourceName, LocalizedResource updatedResource, - boolean uncompress) { - if (uncompress) { - _localrsrcArchives.putIfAbsent(resourceName, updatedResource); - } else { - _localrsrcFiles.putIfAbsent(resourceName, updatedResource); - } - } - - public void add(String resourceName, LocalizedResource newResource, boolean uncompress) { - if (uncompress) { - _localrsrcArchives.put(resourceName, newResource); - } else { - _localrsrcFiles.put(resourceName, newResource); - } - } - - public boolean exists(String resourceName, boolean uncompress) { - if (uncompress) { - return _localrsrcArchives.containsKey(resourceName); - } - return _localrsrcFiles.containsKey(resourceName); - } - - public boolean remove(LocalizedResource resource) { - LocalizedResource lrsrc = null; - if (resource.isUncompressed()) { - lrsrc = _localrsrcArchives.remove(resource.getKey()); - } else { - lrsrc = _localrsrcFiles.remove(resource.getKey()); - } - return (lrsrc != null); - } - - public Iterator getLocalFilesIterator() { - return _localrsrcFiles.values().iterator(); - } - - public Iterator getLocalArchivesIterator() { - return _localrsrcArchives.values().iterator(); - } -} diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java index a287e959d3c..1f7ee006dd8 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java @@ -19,18 +19,24 @@ package org.apache.storm.localizer; import java.io.IOException; +import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.LinkOption; import java.nio.file.Path; +import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import org.apache.storm.blobstore.BlobStore; import org.apache.storm.blobstore.ClientBlobStore; +import org.apache.storm.blobstore.InputStreamWithMeta; +import org.apache.storm.daemon.supervisor.IAdvancedFSOps; import org.apache.storm.generated.AuthorizationException; import org.apache.storm.generated.KeyNotFoundException; +import org.apache.storm.utils.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,9 +47,10 @@ public abstract class LocallyCachedBlob { private static final Logger LOG = LoggerFactory.getLogger(LocallyCachedBlob.class); public static final long NOT_DOWNLOADED_VERSION = -1; // A callback that does nothing. - private static final BlobChangingCallback NOOP_CB = (assignment, port, resource, go) -> {}; + private static final BlobChangingCallback NOOP_CB = (assignment, port, resource, go) -> { + }; - private long lastUsed = System.currentTimeMillis(); + private long lastUsed = Time.currentTimeMillis(); private final Map references = new HashMap<>(); private final String blobDescription; private final String blobKey; @@ -51,8 +58,9 @@ public abstract class LocallyCachedBlob { /** * Create a new LocallyCachedBlob. + * * @param blobDescription a description of the blob this represents. Typically it should at least be the blob key, but ideally also - * include if it is an archive or not, what user or topology it is for, or if it is a storm.jar etc. + * include if it is an archive or not, what user or topology it is for, or if it is a storm.jar etc. */ protected LocallyCachedBlob(String blobDescription, String blobKey) { this.blobDescription = blobDescription; @@ -60,26 +68,54 @@ protected LocallyCachedBlob(String blobDescription, String blobKey) { } /** - * Get the version of the blob cached locally. If the version is unknown or it has not been downloaded NOT_DOWNLOADED_VERSION - * should be returned. - * PRECONDITION: this can only be called with a lock on this instance held. + * Get the version of the blob cached locally. If the version is unknown or it has not been downloaded NOT_DOWNLOADED_VERSION should be + * returned. PRECONDITION: this can only be called with a lock on this instance held. */ public abstract long getLocalVersion(); /** - * Get the version of the blob in the blob store. - * PRECONDITION: this can only be called with a lock on this instance held. + * Get the version of the blob in the blob store. PRECONDITION: this can only be called with a lock on this instance held. */ public abstract long getRemoteVersion(ClientBlobStore store) throws KeyNotFoundException, AuthorizationException; /** * Download the latest version to a temp location. This may also include unzipping some or all of the data to a temp location. * PRECONDITION: this can only be called with a lock on this instance held. + * * @param store the store to us to download the data. * @return the version that was downloaded. */ public abstract long downloadToTempLocation(ClientBlobStore store) throws IOException, KeyNotFoundException, AuthorizationException; + protected static long downloadToTempLocation(ClientBlobStore store, String key, long currentVersion, IAdvancedFSOps fsOps, + Function getTempPath) + throws KeyNotFoundException, AuthorizationException, IOException { + try (InputStreamWithMeta in = store.getBlob(key)) { + long newVersion = in.getVersion(); + if (newVersion == currentVersion) { + LOG.warn("The version did not change, but going to download again {} {}", currentVersion, key); + } + Path tmpLocation = getTempPath.apply(newVersion); + long totalRead = 0; + //Make sure the parent directory is there and ready to go + fsOps.forceMkdir(tmpLocation.getParent()); + try (OutputStream outStream = fsOps.getOutputStream(tmpLocation.toFile())) { + byte[] buffer = new byte[4096]; + int read = 0; + while ((read = in.read(buffer)) > 0) { + outStream.write(buffer, 0, read); + totalRead += read; + } + } + long expectedSize = in.getFileLength(); + if (totalRead != expectedSize) { + throw new IOException("We expected to download " + expectedSize + " bytes but found we got " + totalRead); + } + + return newVersion; + } + } + /** * Commit the new version and make it available for the end user. * PRECONDITION: uncompressToTempLocationIfNeeded will have been called. @@ -109,33 +145,12 @@ protected LocallyCachedBlob(String blobDescription, String blobKey) { */ public abstract long getSizeOnDisk(); - /** - * Updates the last updated time. This should be called when references are added or removed. - */ - private synchronized void touch() { - lastUsed = System.currentTimeMillis(); - } - - /** - * Get the last time that this used for LRU calculations. - */ - public synchronized long getLastUsed() { - return lastUsed; - } - - /** - * Return true if this blob is actively being used, else false (meaning it can be deleted, but might not be). - */ - public synchronized boolean isUsed() { - return !references.isEmpty(); - } - /** * Get the size of p in bytes. * @param p the path to read. * @return the size of p in bytes. */ - protected long getSizeOnDisk(Path p) throws IOException { + protected static long getSizeOnDisk(Path p) throws IOException { if (!Files.exists(p)) { return 0; } else if (Files.isRegularFile(p)) { @@ -155,6 +170,28 @@ protected long getSizeOnDisk(Path p) throws IOException { } } + /** + * Updates the last updated time. This should be called when references are added or removed. + */ + protected synchronized void touch() { + lastUsed = Time.currentTimeMillis(); + LOG.debug("Setting {} ts to {}", blobKey, lastUsed); + } + + /** + * Get the last time that this used for LRU calculations. + */ + public synchronized long getLastUsed() { + return lastUsed; + } + + /** + * Return true if this blob is actively being used, else false (meaning it can be deleted, but might not be). + */ + public synchronized boolean isUsed() { + return !references.isEmpty(); + } + /** * Mark that a given port and assignment are using this. * @param pna the slot and assignment that are using this blob. @@ -177,6 +214,7 @@ public void removeReference(final PortAndAssignment pna) { if (references.remove(pna) == null) { LOG.warn("{} had no reservation for {}", pna, blobDescription); } + touch(); } /** @@ -217,4 +255,11 @@ public synchronized void informAllChangeComplete() { public String getKey() { return blobKey; } + + + public Collection getDependencies() { + return references.keySet(); + } + + public abstract boolean isFullyDownloaded(); } diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java index 35371b53797..68415e1caec 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java @@ -115,14 +115,13 @@ public String getExtractionDir() { public String getTempExtractionDir(long version) { return extractionDir + "." + version; } - }; + } private final TopologyBlobType type; private final String topologyId; private final boolean isLocalMode; private final Path topologyBasicBlobsRootDir; private final AdvancedFSOps fsOps; - private final Map conf; private volatile long version = NOT_DOWNLOADED_VERSION; private volatile long size = 0; @@ -139,7 +138,6 @@ protected LocallyCachedTopologyBlob(final String topologyId, final boolean isLoc this.type = type; this.isLocalMode = isLocalMode; this.fsOps = fsOps; - this.conf = conf; topologyBasicBlobsRootDir = Paths.get(ConfigUtils.supervisorStormDistRoot(conf, topologyId)); readVersion(); updateSizeOnDisk(); @@ -203,31 +201,11 @@ public long downloadToTempLocation(ClientBlobStore store) return LOCAL_MODE_JAR_VERSION; } - long newVersion; - Path tmpLocation; - String key = type.getKey(topologyId); - try (InputStreamWithMeta in = store.getBlob(key)) { - newVersion = in.getVersion(); - long expectedSize = in.getFileLength(); - if (newVersion == version) { - throw new RuntimeException("The version did not change, but we tried to download it. " + version + " " + key); - } - tmpLocation = topologyBasicBlobsRootDir.resolve(type.getTempFileName(newVersion)); - long totalRead = 0; - //Make sure the parent directory is there and ready to go - fsOps.forceMkdir(tmpLocation.getParent()); - try (OutputStream outStream = fsOps.getOutputStream(tmpLocation.toFile())) { - byte [] buffer = new byte[4096]; - int read = 0; - while ((read = in.read(buffer)) > 0) { - outStream.write(buffer, 0, read); - totalRead += read; - } - } - if (totalRead != expectedSize) { - throw new IOException("We expected to download " + expectedSize + " bytes but found we got " + totalRead); - } - } + + long newVersion = downloadToTempLocation(store, type.getKey(topologyId), version, fsOps, + (version) -> topologyBasicBlobsRootDir.resolve(type.getTempFileName(version))); + + Path tmpLocation = topologyBasicBlobsRootDir.resolve(type.getTempFileName(newVersion)); if (type.needsExtraction()) { Path extractionDest = topologyBasicBlobsRootDir.resolve(type.getTempExtractionDir(newVersion)); @@ -247,10 +225,10 @@ protected void extractDirFromJar(String jarpath, String dir, Path dest) throws I String name = entry.getName(); if (!entry.isDirectory() && name.startsWith(toRemove)) { String shortenedName = name.replace(toRemove, ""); - Path aFile = dest.resolve(shortenedName); - LOG.debug("EXTRACTING {} SHORTENED to {} into {}", name, shortenedName, aFile); - fsOps.forceMkdir(aFile.getParent()); - try (FileOutputStream out = new FileOutputStream(aFile.toFile()); + Path targetFile = dest.resolve(shortenedName); + LOG.debug("EXTRACTING {} SHORTENED to {} into {}", name, shortenedName, targetFile); + fsOps.forceMkdir(targetFile.getParent()); + try (FileOutputStream out = new FileOutputStream(targetFile.toFile()); InputStream in = jarFile.getInputStream(entry)) { IOUtils.copy(in, out); } @@ -259,6 +237,21 @@ protected void extractDirFromJar(String jarpath, String dir, Path dest) throws I } } + @Override + public boolean isFullyDownloaded() { + Path versionFile = topologyBasicBlobsRootDir.resolve(type.getVersionFileName()); + boolean ret = Files.exists(versionFile); + Path dest = topologyBasicBlobsRootDir.resolve(type.getFileName()); + if (!(isLocalMode && type == TopologyBlobType.TOPO_JAR)) { + ret = ret && Files.exists(dest); + } + if (type.needsExtraction()) { + Path extractionDest = topologyBasicBlobsRootDir.resolve(type.getExtractionDir()); + ret = ret && Files.exists(extractionDest); + } + return ret; + } + @Override public void commitNewVersion(long newVersion) throws IOException { //This is not atomic (so if something bad happens in the middle we need to be able to recover @@ -325,6 +318,7 @@ public void completelyRemove() throws IOException { if (type.needsExtraction()) { removeAll(type.getExtractionDir()); } + touch(); } private void removeAll(String baseName) throws IOException { diff --git a/storm-server/src/main/java/org/apache/storm/localizer/PortAndAssignment.java b/storm-server/src/main/java/org/apache/storm/localizer/PortAndAssignment.java index 081c8117493..bd92173e85a 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/PortAndAssignment.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/PortAndAssignment.java @@ -21,7 +21,7 @@ import org.apache.storm.generated.LocalAssignment; /** - * A Port and a LocalAssignment used to reference count Resources + * A Port and a LocalAssignment used to reference count resources. */ class PortAndAssignment { private final int port; @@ -45,6 +45,10 @@ public String getToplogyId() { return assignment.get_topology_id(); } + public String getOwner() { + return assignment.get_owner(); + } + @Override public int hashCode() { return (17 * port) + assignment.hashCode(); @@ -52,7 +56,7 @@ public int hashCode() { @Override public String toString() { - return "{" + port + " " + assignment + "}"; + return "{" + assignment.get_topology_id() + " on " + port + "}"; } /** diff --git a/storm-server/src/main/java/org/apache/storm/utils/ServerUtils.java b/storm-server/src/main/java/org/apache/storm/utils/ServerUtils.java index 6a4454a0a85..340db1cba7e 100644 --- a/storm-server/src/main/java/org/apache/storm/utils/ServerUtils.java +++ b/storm-server/src/main/java/org/apache/storm/utils/ServerUtils.java @@ -82,10 +82,7 @@ public class ServerUtils { public static final Logger LOG = LoggerFactory.getLogger(ServerUtils.class); public static final boolean IS_ON_WINDOWS = "Windows_NT".equals(System.getenv("OS")); - public static final String CURRENT_BLOB_SUFFIX_ID = "current"; - public static final String DEFAULT_CURRENT_BLOB_SUFFIX = "." + CURRENT_BLOB_SUFFIX_ID; - public static final String DEFAULT_BLOB_VERSION_SUFFIX = ".version"; public static final int SIGKILL = 9; public static final int SIGTERM = 15; @@ -166,14 +163,6 @@ public static String shellCmd (List command) { return StringUtils.join(changedCommands, " "); } - public static String constructVersionFileName(String fileName) { - return fileName + DEFAULT_BLOB_VERSION_SUFFIX; - } - - public static String constructBlobCurrentSymlinkName(String fileName) { - return fileName + DEFAULT_CURRENT_BLOB_SUFFIX; - } - /** * Takes an input dir or file and returns the disk usage on that local directory. * Very basic implementation. @@ -206,14 +195,6 @@ public static long getDU(File dir) { } } - public static long localVersionOfBlob(String localFile) { - return Utils.getVersionFromBlobVersionFile(new File(localFile + DEFAULT_BLOB_VERSION_SUFFIX)); - } - - public static String constructBlobWithVersionFileName(String fileName, long version) { - return fileName + "." + version; - } - public static ClientBlobStore getClientBlobStoreForSupervisor(Map conf) { ClientBlobStore store; if (ConfigUtils.isLocalMode(conf)) { diff --git a/storm-server/src/test/java/org/apache/storm/TestingTest.java b/storm-server/src/test/java/org/apache/storm/TestingTest.java new file mode 100644 index 00000000000..201b4706c06 --- /dev/null +++ b/storm-server/src/test/java/org/apache/storm/TestingTest.java @@ -0,0 +1,139 @@ +/** + * 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.storm; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.storm.testing.CompleteTopologyParam; +import org.apache.storm.testing.FixedTuple; +import org.apache.storm.testing.IntegrationTest; +import org.apache.storm.testing.MkClusterParam; +import org.apache.storm.testing.MockedSources; +import org.apache.storm.testing.TestAggregatesCounter; +import org.apache.storm.testing.TestGlobalCount; +import org.apache.storm.testing.TestJob; +import org.apache.storm.testing.TestWordCounter; +import org.apache.storm.testing.TestWordSpout; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import static org.junit.Assert.*; + +/** + * Test that the testing class does what it should do. + */ +public class TestingTest { + + private static final TestJob COMPLETE_TOPOLOGY_TESTJOB = (cluster) -> { + TopologyBuilder tb = new TopologyBuilder(); + tb.setSpout("spout", new TestWordSpout(true), 3); + tb.setBolt("2", new TestWordCounter(), 4) + .fieldsGrouping("spout", new Fields("word")); + tb.setBolt("3", new TestGlobalCount()) + .globalGrouping("spout"); + tb.setBolt("4", new TestAggregatesCounter()) + .globalGrouping("2"); + + MockedSources mocked = new MockedSources(); + mocked.addMockData("spout", + new Values("nathan"), + new Values("bob"), + new Values("joey"), + new Values("nathan")); + + Config topoConf = new Config(); + topoConf.setNumWorkers(2); + + CompleteTopologyParam ctp = new CompleteTopologyParam(); + ctp.setMockedSources(mocked); + ctp.setStormConf(topoConf); + + Map> results = Testing.completeTopology(cluster, tb.createTopology(), ctp); + List> spoutTuples = Testing.readTuples(results, "spout"); + List> expectedSpoutTuples = Arrays.asList(Arrays.asList("nathan"), Arrays.asList("bob"), Arrays.asList("joey"), + Arrays.asList("nathan")); + assertTrue(expectedSpoutTuples + " expected, but found " + spoutTuples, + Testing.multiseteq(expectedSpoutTuples, spoutTuples)); + + List> twoTuples = Testing.readTuples(results, "2"); + List> expectedTwoTuples = Arrays.asList(Arrays.asList("nathan", 1), Arrays.asList("nathan", 2), + Arrays.asList("bob", 1), Arrays.asList("joey", 1)); + assertTrue(expectedTwoTuples + " expected, but found " + twoTuples, + Testing.multiseteq(expectedTwoTuples, twoTuples)); + + List> threeTuples = Testing.readTuples(results, "3"); + List> expectedThreeTuples = Arrays.asList(Arrays.asList(1), Arrays.asList(2), + Arrays.asList(3), Arrays.asList(4)); + assertTrue(expectedThreeTuples + " expected, but found " + threeTuples, + Testing.multiseteq(expectedThreeTuples, threeTuples)); + + List> fourTuples = Testing.readTuples(results, "4"); + List> expectedFourTuples = Arrays.asList(Arrays.asList(1), Arrays.asList(2), + Arrays.asList(3), Arrays.asList(4)); + assertTrue(expectedFourTuples + " expected, but found " + fourTuples, + Testing.multiseteq(expectedFourTuples, fourTuples)); + }; + + @Test + @Category(IntegrationTest.class) + public void testCompleteTopologyNettySimulated() throws Exception { + Config daemonConf = new Config(); + daemonConf.put(Config.STORM_LOCAL_MODE_ZMQ, true); + MkClusterParam param = new MkClusterParam(); + param.setSupervisors(4); + param.setDaemonConf(daemonConf); + + Testing.withSimulatedTimeLocalCluster(param, COMPLETE_TOPOLOGY_TESTJOB); + } + + @Test + @Category(IntegrationTest.class) + public void testCompleteTopologyNetty() throws Exception { + Config daemonConf = new Config(); + daemonConf.put(Config.STORM_LOCAL_MODE_ZMQ, true); + MkClusterParam param = new MkClusterParam(); + param.setSupervisors(4); + param.setDaemonConf(daemonConf); + + Testing.withLocalCluster(param, COMPLETE_TOPOLOGY_TESTJOB); + } + + @Test + @Category(IntegrationTest.class) + public void testCompleteTopologyLocalSimulated() throws Exception { + MkClusterParam param = new MkClusterParam(); + param.setSupervisors(4); + + Testing.withSimulatedTimeLocalCluster(param, COMPLETE_TOPOLOGY_TESTJOB); + } + + @Test + @Category(IntegrationTest.class) + public void testCompleteTopologyLocal() throws Exception { + MkClusterParam param = new MkClusterParam(); + param.setSupervisors(4); + + Testing.withLocalCluster(param, COMPLETE_TOPOLOGY_TESTJOB); + } + +} \ No newline at end of file diff --git a/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java b/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java index 65f22def116..251f029f80e 100644 --- a/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java +++ b/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java @@ -19,7 +19,7 @@ package org.apache.storm.localizer; import static org.apache.storm.blobstore.BlobStoreAclHandler.WORLD_EVERYTHING; -import static org.apache.storm.localizer.AsyncLocalizer.USERCACHE; +import static org.apache.storm.localizer.LocalizedResource.USERCACHE; import static org.junit.Assert.*; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -41,17 +41,20 @@ import java.io.IOException; import java.io.InputStream; import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.storm.DaemonConfig; @@ -68,6 +71,7 @@ import org.apache.storm.utils.ConfigUtils; import org.apache.storm.utils.ServerUtils; import org.apache.storm.utils.ReflectionUtils; +import org.apache.storm.utils.Time; import org.apache.storm.utils.Utils; import org.junit.After; import org.junit.Before; @@ -80,8 +84,11 @@ import org.apache.storm.generated.StormTopology; import org.apache.storm.security.auth.DefaultPrincipalToLocal; import org.mockito.Mockito; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class AsyncLocalizerTest { + private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizerTest.class); private static String getTestLocalizerRoot() { File f = new File("./target/" + Thread.currentThread().getStackTrace()[2].getMethodName() + "/localizer/"); @@ -142,7 +149,7 @@ public void testRequestDownloadBaseTopologyBlobs() throws Exception { ReflectionUtils mockedRU = mock(ReflectionUtils.class); ServerUtils mockedU = mock(ServerUtils.class); - AsyncLocalizer bl = spy(new AsyncLocalizer(conf, ops, getTestLocalizerRoot(), new AtomicReference<>(new HashMap<>()), null)); + AsyncLocalizer bl = spy(new AsyncLocalizer(conf, ops, getTestLocalizerRoot())); LocallyCachedTopologyBlob jarBlob = mock(LocallyCachedTopologyBlob.class); doReturn(jarBlob).when(bl).getTopoJar(topoId); when(jarBlob.getLocalVersion()).thenReturn(-1L); @@ -166,7 +173,8 @@ public void testRequestDownloadBaseTopologyBlobs() throws Exception { try { when(mockedRU.newInstanceImpl(ClientBlobStore.class)).thenReturn(blobStore); - Future f = bl.requestDownloadBaseTopologyBlobs(la, port, null); + PortAndAssignment pna = new PortAndAssignment(port, la); + Future f = bl.requestDownloadBaseTopologyBlobs(pna, null); f.get(20, TimeUnit.SECONDS); verify(jarBlob).downloadToTempLocation(any()); @@ -214,8 +222,7 @@ public void testRequestDownloadTopologyBlobs() throws Exception { final String stormRoot = stormLocal+topoId+"/"; final String localizerRoot = getTestLocalizerRoot(); - final String simpleLocalFile = localizerRoot + user + "/simple"; - final String simpleCurrentLocalFile = localizerRoot + user + "/simple.current"; + final String simpleCurrentLocalFile = localizerRoot + "/usercache/" + user + "/filecache/files/simple.current"; final StormTopology st = new StormTopology(); st.set_spouts(new HashMap<>()); @@ -238,10 +245,10 @@ public void testRequestDownloadTopologyBlobs() throws Exception { topoConf.put(Config.TOPOLOGY_NAME, topoName); List localizedList = new ArrayList<>(); - LocalizedResource simpleLocal = new LocalizedResource(simpleKey, simpleLocalFile, false); + LocalizedResource simpleLocal = new LocalizedResource(simpleKey, Paths.get(localizerRoot), false, ops, conf, user); localizedList.add(simpleLocal); - AsyncLocalizer bl = spy(new AsyncLocalizer(conf, ops, localizerRoot, new AtomicReference<>(new HashMap<>()), null)); + AsyncLocalizer bl = spy(new AsyncLocalizer(conf, ops, localizerRoot)); ConfigUtils orig = ConfigUtils.setInstance(mockedCU); try { when(mockedCU.supervisorStormDistRootImpl(conf, topoId)).thenReturn(stormRoot); @@ -250,9 +257,9 @@ public void testRequestDownloadTopologyBlobs() throws Exception { //Write the mocking backwards so the actual method is not called on the spy object doReturn(CompletableFuture.supplyAsync(() -> null)).when(bl) - .requestDownloadBaseTopologyBlobs(la, port, null); + .requestDownloadBaseTopologyBlobs(any(), eq(null)); doReturn(userDir).when(bl).getLocalUserFileCacheDir(user); - doReturn(localizedList).when(bl).getBlobs(any(List.class), eq(user), eq(topoName), eq(userDir)); + doReturn(localizedList).when(bl).getBlobs(any(List.class), any(), any()); Future f = bl.requestDownloadTopologyBlobs(la, port, null); f.get(20, TimeUnit.SECONDS); @@ -263,12 +270,16 @@ public void testRequestDownloadTopologyBlobs() throws Exception { verify(ops).fileExists(userDir); verify(ops).forceMkdir(userDir); - verify(bl).getBlobs(any(List.class), eq(user), eq(topoName), eq(userDir)); + verify(bl).getBlobs(any(List.class), any(), any()); verify(ops).createSymlink(new File(stormRoot, simpleLocalName), new File(simpleCurrentLocalFile)); } finally { - bl.close(); - ConfigUtils.setInstance(orig); + try { + ConfigUtils.setInstance(orig); + bl.close(); + } catch (Throwable e) { + LOG.error("ERROR trying to close an object", e); + } } } @@ -286,29 +297,80 @@ public void testRequestDownloadTopologyBlobs() throws Exception { class TestLocalizer extends AsyncLocalizer { TestLocalizer(Map conf, String baseDir) throws IOException { - super(conf, AdvancedFSOps.make(conf), baseDir, new AtomicReference<>(new HashMap<>()), null); + super(conf, AdvancedFSOps.make(conf), baseDir); } @Override protected ClientBlobStore getClientBlobStore() { return mockblobstore; } + + synchronized void addReferences(List localresource, PortAndAssignment pna, BlobChangingCallback cb) { + String user = pna.getOwner(); + for (LocalResource blob : localresource) { + ConcurrentMap lrsrcSet = blob.shouldUncompress() ? userArchives.get(user) : userFiles.get(user); + if (lrsrcSet != null) { + LocalizedResource lrsrc = lrsrcSet.get(blob.getBlobName()); + if (lrsrc != null) { + lrsrc.addReference(pna, blob.needsCallback() ? cb : null); + lrsrc.addReference(pna, blob.needsCallback() ? cb : null); + LOG.debug("added reference for topo: {} key: {}", pna, blob); + } else { + LOG.warn("trying to add reference to non-existent blob, key: {} topo: {}", blob, pna); + } + } else { + LOG.warn("trying to add reference to non-existent local resource set, user: {} topo: {}", user, pna); + } + } + } + + void setTargetCacheSize(long size) { + cacheTargetSize = size; + } + + // For testing, be careful as it doesn't clone + ConcurrentMap> getUserFiles() { + return userFiles; + } + + ConcurrentMap> getUserArchives() { + return userArchives; + } + + /** + * This function either returns the blob in the existing cache or if it doesn't exist in the + * cache, it will download the blob and will block until the download is complete. + */ + LocalizedResource getBlob(LocalResource localResource, PortAndAssignment pna, BlobChangingCallback cb) + throws AuthorizationException, KeyNotFoundException, IOException { + ArrayList arr = new ArrayList<>(); + arr.add(localResource); + List results = getBlobs(arr, pna, cb); + if (results.isEmpty() || results.size() != 1) { + throw new IOException("Unknown error getting blob: " + localResource + ", for user: " + pna.getOwner() + + ", topo: " + pna); + } + return results.get(0); + } } class TestInputStreamWithMeta extends InputStreamWithMeta { + private final long version; private InputStream iostream; - public TestInputStreamWithMeta() { + public TestInputStreamWithMeta(long version) { iostream = IOUtils.toInputStream("some test data for my input stream"); + this.version = version; } - public TestInputStreamWithMeta(InputStream istream) { + public TestInputStreamWithMeta(InputStream istream, long version) { iostream = istream; + this.version = version; } @Override public long getVersion() throws IOException { - return 1; + return version; } @Override @@ -356,11 +418,11 @@ public String constructUserCacheDir(String base, String user) { } public String constructExpectedFilesDir(String base, String user) { - return joinPath(constructUserCacheDir(base, user), AsyncLocalizer.FILECACHE, AsyncLocalizer.FILESDIR); + return joinPath(constructUserCacheDir(base, user), LocalizedResource.FILECACHE, LocalizedResource.FILESDIR); } public String constructExpectedArchivesDir(String base, String user) { - return joinPath(constructUserCacheDir(base, user), AsyncLocalizer.FILECACHE, AsyncLocalizer.ARCHIVESDIR); + return joinPath(constructUserCacheDir(base, user), LocalizedResource.FILECACHE, LocalizedResource.ARCHIVESDIR); } @Test @@ -372,14 +434,14 @@ public void testDirPaths() throws Exception { assertEquals("get local user dir doesn't return right value", expectedDir, localizer.getLocalUserDir(user1).toString()); - String expectedFileDir = joinPath(expectedDir, AsyncLocalizer.FILECACHE); + String expectedFileDir = joinPath(expectedDir, LocalizedResource.FILECACHE); assertEquals("get local user file dir doesn't return right value", expectedFileDir, localizer.getLocalUserFileCacheDir(user1).toString()); } @Test public void testReconstruct() throws Exception { - Map conf = new HashMap(); + Map conf = new HashMap<>(); String expectedFileDir1 = constructExpectedFilesDir(baseDir.toString(), user1); String expectedArchiveDir1 = constructExpectedArchivesDir(baseDir.toString(), user1); @@ -394,13 +456,13 @@ public void testReconstruct() throws Exception { String archive1 = "archive1"; String archive2 = "archive2"; - File user1file1 = new File(expectedFileDir1, key1 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); - File user1file2 = new File(expectedFileDir1, key2 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); - File user2file3 = new File(expectedFileDir2, key3 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); - File user2file4 = new File(expectedFileDir2, key4 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); + File user1file1 = new File(expectedFileDir1, key1 + LocalizedResource.CURRENT_BLOB_SUFFIX); + File user1file2 = new File(expectedFileDir1, key2 + LocalizedResource.CURRENT_BLOB_SUFFIX); + File user2file3 = new File(expectedFileDir2, key3 + LocalizedResource.CURRENT_BLOB_SUFFIX); + File user2file4 = new File(expectedFileDir2, key4 + LocalizedResource.CURRENT_BLOB_SUFFIX); - File user1archive1 = new File(expectedArchiveDir1, archive1 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); - File user2archive2 = new File(expectedArchiveDir2, archive2 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); + File user1archive1 = new File(expectedArchiveDir1, archive1 + LocalizedResource.CURRENT_BLOB_SUFFIX); + File user2archive2 = new File(expectedArchiveDir2, archive2 + LocalizedResource.CURRENT_BLOB_SUFFIX); File user1archive1file = new File(user1archive1, "file1"); File user2archive2file = new File(user2archive2, "file2"); @@ -416,44 +478,46 @@ public void testReconstruct() throws Exception { assertTrue("Failed setup file in archivedir1", user1archive1file.createNewFile()); assertTrue("Failed setup file in archivedir2", user2archive2file.createNewFile()); - AsyncLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); + TestLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); - ArrayList arrUser1Keys = new ArrayList(); - arrUser1Keys.add(new LocalResource(key1, false)); - arrUser1Keys.add(new LocalResource(archive1, true)); - localizer.addReferences(arrUser1Keys, user1, "topo1"); + ArrayList arrUser1Keys = new ArrayList<>(); + arrUser1Keys.add(new LocalResource(key1, false, false)); + arrUser1Keys.add(new LocalResource(archive1, true, false)); + LocalAssignment topo1 = new LocalAssignment("topo1", Collections.emptyList()); + topo1.set_owner(user1); + localizer.addReferences(arrUser1Keys, new PortAndAssignment(1, topo1), null); - LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1); - assertEquals("local resource set size wrong", 3, lrsrcSet.getSize()); - assertEquals("user doesn't match", user1, lrsrcSet.getUser()); - LocalizedResource key1rsrc = lrsrcSet.get(key1, false); + ConcurrentMap lrsrcFiles = localizer.getUserFiles().get(user1); + ConcurrentMap lrsrcArchives = localizer.getUserArchives().get(user1); + assertEquals("local resource set size wrong", 3, lrsrcFiles.size() + lrsrcArchives.size()); + LocalizedResource key1rsrc = lrsrcFiles.get(key1); assertNotNull("Local resource doesn't exist but should", key1rsrc); assertEquals("key doesn't match", key1, key1rsrc.getKey()); - assertEquals("refcount doesn't match", 1, key1rsrc.getRefCount()); - LocalizedResource key2rsrc = lrsrcSet.get(key2, false); + assertEquals("references doesn't match " + key1rsrc.getDependencies(), true, key1rsrc.isUsed()); + LocalizedResource key2rsrc = lrsrcFiles.get(key2); assertNotNull("Local resource doesn't exist but should", key2rsrc); assertEquals("key doesn't match", key2, key2rsrc.getKey()); - assertEquals("refcount doesn't match", 0, key2rsrc.getRefCount()); - LocalizedResource archive1rsrc = lrsrcSet.get(archive1, true); + assertEquals("refcount doesn't match " + key2rsrc.getDependencies(), false, key2rsrc.isUsed()); + LocalizedResource archive1rsrc = lrsrcArchives.get(archive1); assertNotNull("Local resource doesn't exist but should", archive1rsrc); assertEquals("key doesn't match", archive1, archive1rsrc.getKey()); - assertEquals("refcount doesn't match", 1, archive1rsrc.getRefCount()); + assertEquals("refcount doesn't match " + archive1rsrc.getDependencies(), true, archive1rsrc.isUsed()); - LocalizedResourceSet lrsrcSet2 = localizer.getUserResources().get(user2); - assertEquals("local resource set size wrong", 3, lrsrcSet2.getSize()); - assertEquals("user doesn't match", user2, lrsrcSet2.getUser()); - LocalizedResource key3rsrc = lrsrcSet2.get(key3, false); + ConcurrentMap lrsrcFiles2 = localizer.getUserFiles().get(user2); + ConcurrentMap lrsrcArchives2 = localizer.getUserArchives().get(user2); + assertEquals("local resource set size wrong", 3, lrsrcFiles2.size() + lrsrcArchives2.size()); + LocalizedResource key3rsrc = lrsrcFiles2.get(key3); assertNotNull("Local resource doesn't exist but should", key3rsrc); assertEquals("key doesn't match", key3, key3rsrc.getKey()); - assertEquals("refcount doesn't match", 0, key3rsrc.getRefCount()); - LocalizedResource key4rsrc = lrsrcSet2.get(key4, false); + assertEquals("refcount doesn't match " + key3rsrc.getDependencies(), false, key3rsrc.isUsed()); + LocalizedResource key4rsrc = lrsrcFiles2.get(key4); assertNotNull("Local resource doesn't exist but should", key4rsrc); assertEquals("key doesn't match", key4, key4rsrc.getKey()); - assertEquals("refcount doesn't match", 0, key4rsrc.getRefCount()); - LocalizedResource archive2rsrc = lrsrcSet2.get(archive2, true); + assertEquals("refcount doesn't match " + key4rsrc.getDependencies(), false, key4rsrc.isUsed()); + LocalizedResource archive2rsrc = lrsrcArchives2.get(archive2); assertNotNull("Local resource doesn't exist but should", archive2rsrc); assertEquals("key doesn't match", archive2, archive2rsrc.getKey()); - assertEquals("refcount doesn't match", 0, archive2rsrc.getRefCount()); + assertEquals("refcount doesn't match " + archive2rsrc.getDependencies(), false, archive2rsrc.isUsed()); } @Test @@ -492,236 +556,269 @@ public void testArchives(File archiveFile, boolean supportSymlinks, int size) th // Windows should set this to false cause symlink in compressed file doesn't work properly. supportSymlinks = false; } + try (Time.SimulatedTime st = new Time.SimulatedTime()) { + + Map conf = new HashMap<>(); + // set clean time really high so doesn't kick in + conf.put(DaemonConfig.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60 * 60 * 1000); + + String key1 = archiveFile.getName(); + String topo1 = "topo1"; + LOG.info("About to create new AsyncLocalizer..."); + TestLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); + // set really small so will do cleanup + localizer.setTargetCacheSize(1); + LOG.info("created AsyncLocalizer..."); + + ReadableBlobMeta rbm = new ReadableBlobMeta(); + rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); + when(mockblobstore.getBlobMeta(key1)).thenReturn(rbm); + + when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta(new + FileInputStream(archiveFile.getAbsolutePath()), 0)); + + long timeBefore = Time.currentTimeMillis(); + Time.advanceTime(10); + File user1Dir = localizer.getLocalUserFileCacheDir(user1); + assertTrue("failed to create user dir", user1Dir.mkdirs()); + LocalAssignment topo1Assignment = new LocalAssignment(topo1, Collections.emptyList()); + topo1Assignment.set_owner(user1); + PortAndAssignment topo1Pna = new PortAndAssignment(1, topo1Assignment); + LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, true, false), topo1Pna, null); + Time.advanceTime(10); + long timeAfter = Time.currentTimeMillis(); + Time.advanceTime(10); + + String expectedUserDir = joinPath(baseDir.toString(), USERCACHE, user1); + String expectedFileDir = joinPath(expectedUserDir, LocalizedResource.FILECACHE, LocalizedResource.ARCHIVESDIR); + assertTrue("user filecache dir not created", new File(expectedFileDir).exists()); + File keyFile = new File(expectedFileDir, key1 + ".0"); + assertTrue("blob not created " + keyFile, keyFile.exists()); + assertTrue("blob is not uncompressed", keyFile.isDirectory()); + File symlinkFile = new File(keyFile, "tmptestsymlink"); + + if (supportSymlinks) { + assertTrue("blob uncompressed doesn't contain symlink", Files.isSymbolicLink( + symlinkFile.toPath())); + } else { + assertTrue("blob symlink file doesn't exist", symlinkFile.exists()); + } - Map conf = new HashMap(); - // set clean time really high so doesn't kick in - conf.put(DaemonConfig.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60*60*1000); - - String key1 = archiveFile.getName(); - String topo1 = "topo1"; - AsyncLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); - // set really small so will do cleanup - localizer.setTargetCacheSize(1); - - ReadableBlobMeta rbm = new ReadableBlobMeta(); - rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); - when(mockblobstore.getBlobMeta(key1)).thenReturn(rbm); - - when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta(new - FileInputStream(archiveFile.getAbsolutePath()))); - - long timeBefore = System.nanoTime(); - File user1Dir = localizer.getLocalUserFileCacheDir(user1); - assertTrue("failed to create user dir", user1Dir.mkdirs()); - LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, true), user1, topo1, - user1Dir); - long timeAfter = System.nanoTime(); - - String expectedUserDir = joinPath(baseDir.toString(), USERCACHE, user1); - String expectedFileDir = joinPath(expectedUserDir, AsyncLocalizer.FILECACHE, AsyncLocalizer.ARCHIVESDIR); - assertTrue("user filecache dir not created", new File(expectedFileDir).exists()); - File keyFile = new File(expectedFileDir, key1 + ".0"); - assertTrue("blob not created", keyFile.exists()); - assertTrue("blob is not uncompressed", keyFile.isDirectory()); - File symlinkFile = new File(keyFile, "tmptestsymlink"); - - if (supportSymlinks) { - assertTrue("blob uncompressed doesn't contain symlink", Files.isSymbolicLink( - symlinkFile.toPath())); - } else { - assertTrue("blob symlink file doesn't exist", symlinkFile.exists()); + ConcurrentMap lrsrcSet = localizer.getUserArchives().get(user1); + assertEquals("local resource set size wrong", 1, lrsrcSet.size()); + LocalizedResource key1rsrc = lrsrcSet.get(key1); + assertNotNull("Local resource doesn't exist but should", key1rsrc); + assertEquals("key doesn't match", key1, key1rsrc.getKey()); + assertEquals("refcount doesn't match " + key1rsrc.getDependencies(), true, key1rsrc.isUsed()); + assertEquals("file path doesn't match", keyFile.toPath(), key1rsrc.getFilePathWithVersion()); + assertEquals("size doesn't match", size, key1rsrc.getSizeOnDisk()); + assertTrue("timestamp not within range", (key1rsrc.getLastUsed() >= timeBefore && key1rsrc + .getLastUsed() <= timeAfter)); + + timeBefore = Time.currentTimeMillis(); + Time.advanceTime(10); + localizer.removeBlobReference(lrsrc.getKey(), topo1Pna, true); + Time.advanceTime(10); + timeAfter = Time.currentTimeMillis(); + Time.advanceTime(10); + + lrsrcSet = localizer.getUserArchives().get(user1); + assertEquals("local resource set size wrong", 1, lrsrcSet.size()); + key1rsrc = lrsrcSet.get(key1); + assertNotNull("Local resource doesn't exist but should", key1rsrc); + assertEquals("refcount doesn't match " + key1rsrc.getDependencies(), false, key1rsrc.isUsed()); + assertTrue("timestamp not within range", (key1rsrc.getLastUsed() >= timeBefore && key1rsrc + .getLastUsed() <= timeAfter)); + + // should remove the blob since cache size set really small + localizer.cleanup(); + + lrsrcSet = localizer.getUserArchives().get(user1); + assertFalse("blob contents not deleted", symlinkFile.exists()); + assertFalse("blob not deleted", keyFile.exists()); + assertFalse("blob file dir not deleted", new File(expectedFileDir).exists()); + assertFalse("blob dir not deleted", new File(expectedUserDir).exists()); + assertNull("user set should be null", lrsrcSet); } - - LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1); - assertEquals("local resource set size wrong", 1, lrsrcSet.getSize()); - assertEquals("user doesn't match", user1, lrsrcSet.getUser()); - LocalizedResource key1rsrc = lrsrcSet.get(key1, true); - assertNotNull("Local resource doesn't exist but should", key1rsrc); - assertEquals("key doesn't match", key1, key1rsrc.getKey()); - assertEquals("refcount doesn't match", 1, key1rsrc.getRefCount()); - assertEquals("file path doesn't match", keyFile.toString(), key1rsrc.getFilePathWithVersion()); - assertEquals("size doesn't match", size, key1rsrc.getSize()); - assertTrue("timestamp not within range", (key1rsrc.getLastAccessTime() >= timeBefore && key1rsrc - .getLastAccessTime() <= timeAfter)); - - timeBefore = System.nanoTime(); - localizer.removeBlobReference(lrsrc.getKey(), user1, topo1, true); - timeAfter = System.nanoTime(); - - lrsrcSet = localizer.getUserResources().get(user1); - assertEquals("local resource set size wrong", 1, lrsrcSet.getSize()); - key1rsrc = lrsrcSet.get(key1, true); - assertNotNull("Local resource doesn't exist but should", key1rsrc); - assertEquals("refcount doesn't match", 0, key1rsrc.getRefCount()); - assertTrue("timestamp not within range", (key1rsrc.getLastAccessTime() >= timeBefore && key1rsrc - .getLastAccessTime() <= timeAfter)); - - // should remove the blob since cache size set really small - localizer.cleanup(); - - lrsrcSet = localizer.getUserResources().get(user1); - assertFalse("blob contents not deleted", symlinkFile.exists()); - assertFalse("blob not deleted", keyFile.exists()); - assertFalse("blob file dir not deleted", new File(expectedFileDir).exists()); - assertFalse("blob dir not deleted", new File(expectedUserDir).exists()); - assertNull("user set should be null", lrsrcSet); - } @Test public void testBasic() throws Exception { - Map conf = new HashMap(); - // set clean time really high so doesn't kick in - conf.put(DaemonConfig.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60*60*1000); - - String key1 = "key1"; - String topo1 = "topo1"; - AsyncLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); - // set really small so will do cleanup - localizer.setTargetCacheSize(1); - - ReadableBlobMeta rbm = new ReadableBlobMeta(); - rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); - when(mockblobstore.getBlobMeta(key1)).thenReturn(rbm); - - when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta()); - - long timeBefore = System.nanoTime(); - File user1Dir = localizer.getLocalUserFileCacheDir(user1); - assertTrue("failed to create user dir", user1Dir.mkdirs()); - LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, false), user1, topo1, - user1Dir); - long timeAfter = System.nanoTime(); - - String expectedUserDir = joinPath(baseDir.toString(), USERCACHE, user1); - String expectedFileDir = joinPath(expectedUserDir, AsyncLocalizer.FILECACHE, AsyncLocalizer.FILESDIR); - assertTrue("user filecache dir not created", new File(expectedFileDir).exists()); - File keyFile = new File(expectedFileDir, key1); - File keyFileCurrentSymlink = new File(expectedFileDir, key1 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); - - assertTrue("blob not created", keyFileCurrentSymlink.exists()); - - LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1); - assertEquals("local resource set size wrong", 1, lrsrcSet.getSize()); - assertEquals("user doesn't match", user1, lrsrcSet.getUser()); - LocalizedResource key1rsrc = lrsrcSet.get(key1, false); - assertNotNull("Local resource doesn't exist but should", key1rsrc); - assertEquals("key doesn't match", key1, key1rsrc.getKey()); - assertEquals("refcount doesn't match", 1, key1rsrc.getRefCount()); - assertEquals("file path doesn't match", keyFile.toString(), key1rsrc.getFilePath()); - assertEquals("size doesn't match", 34, key1rsrc.getSize()); - assertTrue("timestamp not within range", (key1rsrc.getLastAccessTime() >= timeBefore && key1rsrc - .getLastAccessTime() <= timeAfter)); - - timeBefore = System.nanoTime(); - localizer.removeBlobReference(lrsrc.getKey(), user1, topo1, false); - timeAfter = System.nanoTime(); - - lrsrcSet = localizer.getUserResources().get(user1); - assertEquals("local resource set size wrong", 1, lrsrcSet.getSize()); - key1rsrc = lrsrcSet.get(key1, false); - assertNotNull("Local resource doesn't exist but should", key1rsrc); - assertEquals("refcount doesn't match", 0, key1rsrc.getRefCount()); - assertTrue("timestamp not within range", (key1rsrc.getLastAccessTime() >= timeBefore && key1rsrc - .getLastAccessTime() <= timeAfter)); - - // should remove the blob since cache size set really small - localizer.cleanup(); - - lrsrcSet = localizer.getUserResources().get(user1); - assertNull("user set should be null", lrsrcSet); - assertFalse("blob not deleted", keyFile.exists()); - assertFalse("blob dir not deleted", new File(expectedFileDir).exists()); - assertFalse("blob dir not deleted", new File(expectedUserDir).exists()); + try (Time.SimulatedTime st = new Time.SimulatedTime()) { + Map conf = new HashMap(); + // set clean time really high so doesn't kick in + conf.put(DaemonConfig.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60 * 60 * 1000); + + String key1 = "key1"; + String topo1 = "topo1"; + TestLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); + // set really small so will do cleanup + localizer.setTargetCacheSize(1); + + ReadableBlobMeta rbm = new ReadableBlobMeta(); + rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); + when(mockblobstore.getBlobMeta(key1)).thenReturn(rbm); + + when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta(1)); + + long timeBefore = Time.currentTimeMillis(); + Time.advanceTime(10); + File user1Dir = localizer.getLocalUserFileCacheDir(user1); + assertTrue("failed to create user dir", user1Dir.mkdirs()); + Time.advanceTime(10); + LocalAssignment topo1Assignment = new LocalAssignment(topo1, Collections.emptyList()); + topo1Assignment.set_owner(user1); + PortAndAssignment topo1Pna = new PortAndAssignment(1, topo1Assignment); + LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, false, false), topo1Pna, null); + long timeAfter = Time.currentTimeMillis(); + Time.advanceTime(10); + + String expectedUserDir = joinPath(baseDir.toString(), USERCACHE, user1); + String expectedFileDir = joinPath(expectedUserDir, LocalizedResource.FILECACHE, LocalizedResource.FILESDIR); + assertTrue("user filecache dir not created", new File(expectedFileDir).exists()); + File keyFile = new File(expectedFileDir, key1 + ".current"); + File keyFileCurrentSymlink = new File(expectedFileDir, key1 + LocalizedResource.CURRENT_BLOB_SUFFIX); + + assertTrue("blob not created", keyFileCurrentSymlink.exists()); + + ConcurrentMap lrsrcSet = localizer.getUserFiles().get(user1); + assertEquals("local resource set size wrong", 1, lrsrcSet.size()); + LocalizedResource key1rsrc = lrsrcSet.get(key1); + assertNotNull("Local resource doesn't exist but should", key1rsrc); + assertEquals("key doesn't match", key1, key1rsrc.getKey()); + assertEquals("refcount doesn't match " + key1rsrc.getDependencies(), true, key1rsrc.isUsed()); + assertEquals("file path doesn't match", keyFile.toPath(), key1rsrc.getCurrentSymlinkPath()); + assertEquals("size doesn't match", 34, key1rsrc.getSizeOnDisk()); + assertTrue("timestamp not within range", (key1rsrc.getLastUsed() >= timeBefore && key1rsrc + .getLastUsed() <= timeAfter)); + + timeBefore = Time.currentTimeMillis(); + Time.advanceTime(10); + localizer.removeBlobReference(lrsrc.getKey(), topo1Pna, false); + Time.advanceTime(10); + timeAfter = Time.currentTimeMillis(); + Time.advanceTime(10); + + lrsrcSet = localizer.getUserFiles().get(user1); + assertEquals("local resource set size wrong", 1, lrsrcSet.size()); + key1rsrc = lrsrcSet.get(key1); + assertNotNull("Local resource doesn't exist but should", key1rsrc); + assertEquals("refcount doesn't match " + key1rsrc.getDependencies(), false, key1rsrc.isUsed()); + assertTrue("timestamp not within range " + timeBefore + " " + key1rsrc.getLastUsed() + " " + timeAfter, + (key1rsrc.getLastUsed() >= timeBefore && key1rsrc.getLastUsed() <= timeAfter)); + + // should remove the blob since cache size set really small + localizer.cleanup(); + + lrsrcSet = localizer.getUserFiles().get(user1); + assertNull("user set should be null", lrsrcSet); + assertFalse("blob not deleted", keyFile.exists()); + assertFalse("blob dir not deleted", new File(expectedFileDir).exists()); + assertFalse("blob dir not deleted", new File(expectedUserDir).exists()); + } } @Test public void testMultipleKeysOneUser() throws Exception { - Map conf = new HashMap(); - // set clean time really high so doesn't kick in - conf.put(DaemonConfig.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60*60*1000); - - String key1 = "key1"; - String topo1 = "topo1"; - String key2 = "key2"; - String key3 = "key3"; - AsyncLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); - // set to keep 2 blobs (each of size 34) - localizer.setTargetCacheSize(68); - - ReadableBlobMeta rbm = new ReadableBlobMeta(); - rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); - when(mockblobstore.getBlobMeta(anyString())).thenReturn(rbm); - when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta()); - when(mockblobstore.getBlob(key2)).thenReturn(new TestInputStreamWithMeta()); - when(mockblobstore.getBlob(key3)).thenReturn(new TestInputStreamWithMeta()); - - List keys = Arrays.asList(new LocalResource[]{new LocalResource(key1, false), - new LocalResource(key2, false), new LocalResource(key3, false)}); - File user1Dir = localizer.getLocalUserFileCacheDir(user1); - assertTrue("failed to create user dir", user1Dir.mkdirs()); - - List lrsrcs = localizer.getBlobs(keys, user1, topo1, user1Dir); - LocalizedResource lrsrc = lrsrcs.get(0); - LocalizedResource lrsrc2 = lrsrcs.get(1); - LocalizedResource lrsrc3 = lrsrcs.get(2); + try (Time.SimulatedTime st = new Time.SimulatedTime()) { + Map conf = new HashMap<>(); + // set clean time really high so doesn't kick in + conf.put(DaemonConfig.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60 * 60 * 1_000); + + String key1 = "key1"; + String topo1 = "topo1"; + String key2 = "key2"; + String key3 = "key3"; + TestLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); + // set to keep 2 blobs (each of size 34) + localizer.setTargetCacheSize(68); + + ReadableBlobMeta rbm = new ReadableBlobMeta(); + rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); + when(mockblobstore.getBlobMeta(anyString())).thenReturn(rbm); + when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta(0)); + when(mockblobstore.getBlob(key2)).thenReturn(new TestInputStreamWithMeta(0)); + when(mockblobstore.getBlob(key3)).thenReturn(new TestInputStreamWithMeta(0)); + + List keys = Arrays.asList(new LocalResource[]{new LocalResource(key1, false, false), + new LocalResource(key2, false, false), new LocalResource(key3, false, false)}); + File user1Dir = localizer.getLocalUserFileCacheDir(user1); + assertTrue("failed to create user dir", user1Dir.mkdirs()); + + LocalAssignment topo1Assignment = new LocalAssignment(topo1, Collections.emptyList()); + topo1Assignment.set_owner(user1); + PortAndAssignment topo1Pna = new PortAndAssignment(1, topo1Assignment); + List lrsrcs = localizer.getBlobs(keys, topo1Pna, null); + LocalizedResource lrsrc = lrsrcs.get(0); + LocalizedResource lrsrc2 = lrsrcs.get(1); + LocalizedResource lrsrc3 = lrsrcs.get(2); + + String expectedFileDir = joinPath(baseDir.toString(), USERCACHE, user1, + LocalizedResource.FILECACHE, LocalizedResource.FILESDIR); + assertTrue("user filecache dir not created", new File(expectedFileDir).exists()); + File keyFile = new File(expectedFileDir, key1 + LocalizedResource.CURRENT_BLOB_SUFFIX); + File keyFile2 = new File(expectedFileDir, key2 + LocalizedResource.CURRENT_BLOB_SUFFIX); + File keyFile3 = new File(expectedFileDir, key3 + LocalizedResource.CURRENT_BLOB_SUFFIX); + + assertTrue("blob not created", keyFile.exists()); + assertTrue("blob not created", keyFile2.exists()); + assertTrue("blob not created", keyFile3.exists()); + assertEquals("size doesn't match", 34, keyFile.length()); + assertEquals("size doesn't match", 34, keyFile2.length()); + assertEquals("size doesn't match", 34, keyFile3.length()); + assertEquals("size doesn't match", 34, lrsrc.getSizeOnDisk()); + assertEquals("size doesn't match", 34, lrsrc3.getSizeOnDisk()); + assertEquals("size doesn't match", 34, lrsrc2.getSizeOnDisk()); + + ConcurrentMap lrsrcSet = localizer.getUserFiles().get(user1); + assertEquals("local resource set size wrong", 3, lrsrcSet.size()); + + LOG.info("Removing blob references..."); + long timeBefore = Time.nanoTime(); + Time.advanceTime(10); + localizer.removeBlobReference(lrsrc.getKey(), topo1Pna, false); + Time.advanceTime(10); + localizer.removeBlobReference(lrsrc2.getKey(), topo1Pna, false); + Time.advanceTime(10); + localizer.removeBlobReference(lrsrc3.getKey(), topo1Pna, false); + Time.advanceTime(10); + long timeAfter = Time.nanoTime(); + LOG.info("Done removing blob references..."); + + // add reference to one and then remove reference again so it has newer timestamp + LOG.info("Get Blob..."); + lrsrc = localizer.getBlob(new LocalResource(key1, false, false), topo1Pna, null); + LOG.info("Got Blob..."); + assertTrue("timestamp not within range " + timeBefore + " <= " + lrsrc.getLastUsed() + " <= " + timeAfter, + (lrsrc.getLastUsed() >= timeBefore && lrsrc.getLastUsed() <= timeAfter)); + //Resets the last access time for key1 + localizer.removeBlobReference(lrsrc.getKey(), topo1Pna, false); + + // should remove the second blob first + localizer.cleanup(); + + lrsrcSet = localizer.getUserFiles().get(user1); + assertEquals("local resource set size wrong", 2, lrsrcSet.size()); + long end = System.currentTimeMillis() + 100; + while ((end - System.currentTimeMillis()) >= 0 && keyFile2.exists()) { + Thread.sleep(1); + } + assertTrue("blob deleted", keyFile.exists()); + assertFalse("blob not deleted", keyFile2.exists()); + assertTrue("blob deleted", keyFile3.exists()); - String expectedFileDir = joinPath(baseDir.toString(), USERCACHE, user1, - AsyncLocalizer.FILECACHE, AsyncLocalizer.FILESDIR); - assertTrue("user filecache dir not created", new File(expectedFileDir).exists()); - File keyFile = new File(expectedFileDir, key1 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); - File keyFile2 = new File(expectedFileDir, key2 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); - File keyFile3 = new File(expectedFileDir, key3 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); + // set size to cleanup another one + localizer.setTargetCacheSize(34); - assertTrue("blob not created", keyFile.exists()); - assertTrue("blob not created", keyFile2.exists()); - assertTrue("blob not created", keyFile3.exists()); - assertEquals("size doesn't match", 34, keyFile.length()); - assertEquals("size doesn't match", 34, keyFile2.length()); - assertEquals("size doesn't match", 34, keyFile3.length()); - assertEquals("size doesn't match", 34, lrsrc.getSize()); - assertEquals("size doesn't match", 34, lrsrc3.getSize()); - assertEquals("size doesn't match", 34, lrsrc2.getSize()); - - LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1); - assertEquals("local resource set size wrong", 3, lrsrcSet.getSize()); - assertEquals("user doesn't match", user1, lrsrcSet.getUser()); - - long timeBefore = System.nanoTime(); - localizer.removeBlobReference(lrsrc.getKey(), user1, topo1, false); - localizer.removeBlobReference(lrsrc2.getKey(), user1, topo1, false); - localizer.removeBlobReference(lrsrc3.getKey(), user1, topo1, false); - long timeAfter = System.nanoTime(); - - // add reference to one and then remove reference again so it has newer timestamp - lrsrc = localizer.getBlob(new LocalResource(key1, false), user1, topo1, user1Dir); - assertTrue("timestamp not within range", (lrsrc.getLastAccessTime() >= timeBefore && lrsrc - .getLastAccessTime() <= timeAfter)); - localizer.removeBlobReference(lrsrc.getKey(), user1, topo1, false); - - // should remove the second blob first - localizer.cleanup(); + // should remove the third blob, because the first has the reset timestamp + localizer.cleanup(); - lrsrcSet = localizer.getUserResources().get(user1); - assertEquals("local resource set size wrong", 2, lrsrcSet.getSize()); - long end = System.currentTimeMillis() + 100; - while ((end - System.currentTimeMillis()) >= 0 && keyFile2.exists()) { - Thread.sleep(1); + lrsrcSet = localizer.getUserFiles().get(user1); + assertEquals("local resource set size wrong", 1, lrsrcSet.size()); + assertTrue("blob deleted", keyFile.exists()); + assertFalse("blob not deleted", keyFile2.exists()); + assertFalse("blob not deleted", keyFile3.exists()); } - assertFalse("blob not deleted", keyFile2.exists()); - assertTrue("blob deleted", keyFile.exists()); - assertTrue("blob deleted", keyFile3.exists()); - - // set size to cleanup another one - localizer.setTargetCacheSize(34); - - // should remove the third blob - localizer.cleanup(); - - lrsrcSet = localizer.getUserResources().get(user1); - assertEquals("local resource set size wrong", 1, lrsrcSet.getSize()); - assertTrue("blob deleted", keyFile.exists()); - assertFalse("blob not deleted", keyFile3.exists()); } @Test(expected = AuthorizationException.class) @@ -734,7 +831,7 @@ public void testFailAcls() throws Exception { String topo1 = "topo1"; String key1 = "key1"; - AsyncLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); + TestLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); ReadableBlobMeta rbm = new ReadableBlobMeta(); // set acl so user doesn't have read access @@ -742,12 +839,15 @@ public void testFailAcls() throws Exception { acl.set_name(user1); rbm.set_settable(new SettableBlobMeta(Arrays.asList(acl))); when(mockblobstore.getBlobMeta(anyString())).thenReturn(rbm); - when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta()); + when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta(1)); File user1Dir = localizer.getLocalUserFileCacheDir(user1); assertTrue("failed to create user dir", user1Dir.mkdirs()); + LocalAssignment topo1Assignment = new LocalAssignment(topo1, Collections.emptyList()); + topo1Assignment.set_owner(user1); + PortAndAssignment topo1Pna = new PortAndAssignment(1, topo1Assignment); // This should throw AuthorizationException because auth failed - localizer.getBlob(new LocalResource(key1, false), user1, topo1, user1Dir); + localizer.getBlob(new LocalResource(key1, false, false), topo1Pna, null); } @Test(expected = KeyNotFoundException.class) @@ -765,7 +865,7 @@ public void testKeyNotFoundException() throws Exception { @Test public void testMultipleUsers() throws Exception { - Map conf = new HashMap(); + Map conf = new HashMap<>(); // set clean time really high so doesn't kick in conf.put(DaemonConfig.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60*60*1000); @@ -775,16 +875,16 @@ public void testMultipleUsers() throws Exception { String key1 = "key1"; String key2 = "key2"; String key3 = "key3"; - AsyncLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); + TestLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); // set to keep 2 blobs (each of size 34) localizer.setTargetCacheSize(68); ReadableBlobMeta rbm = new ReadableBlobMeta(); rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); when(mockblobstore.getBlobMeta(anyString())).thenReturn(rbm); - when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta()); - when(mockblobstore.getBlob(key2)).thenReturn(new TestInputStreamWithMeta()); - when(mockblobstore.getBlob(key3)).thenReturn(new TestInputStreamWithMeta()); + when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta(1)); + when(mockblobstore.getBlob(key2)).thenReturn(new TestInputStreamWithMeta(1)); + when(mockblobstore.getBlob(key3)).thenReturn(new TestInputStreamWithMeta(1)); File user1Dir = localizer.getLocalUserFileCacheDir(user1); assertTrue("failed to create user dir", user1Dir.mkdirs()); @@ -793,53 +893,61 @@ public void testMultipleUsers() throws Exception { File user3Dir = localizer.getLocalUserFileCacheDir(user3); assertTrue("failed to create user dir", user3Dir.mkdirs()); - LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, false), user1, topo1, - user1Dir); - LocalizedResource lrsrc2 = localizer.getBlob(new LocalResource(key2, false), user2, topo2, - user2Dir); - LocalizedResource lrsrc3 = localizer.getBlob(new LocalResource(key3, false), user3, topo3, - user3Dir); + LocalAssignment topo1Assignment = new LocalAssignment(topo1, Collections.emptyList()); + topo1Assignment.set_owner(user1); + PortAndAssignment topo1Pna = new PortAndAssignment(1, topo1Assignment); + LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, false, false), topo1Pna, null); + + LocalAssignment topo2Assignment = new LocalAssignment(topo2, Collections.emptyList()); + topo2Assignment.set_owner(user2); + PortAndAssignment topo2Pna = new PortAndAssignment(2, topo2Assignment); + LocalizedResource lrsrc2 = localizer.getBlob(new LocalResource(key2, false, false), topo2Pna, null); + + LocalAssignment topo3Assignment = new LocalAssignment(topo3, Collections.emptyList()); + topo3Assignment.set_owner(user3); + PortAndAssignment topo3Pna = new PortAndAssignment(3, topo3Assignment); + LocalizedResource lrsrc3 = localizer.getBlob(new LocalResource(key3, false, false), topo3Pna, null); + // make sure we support different user reading same blob - LocalizedResource lrsrc1_user3 = localizer.getBlob(new LocalResource(key1, false), user3, - topo3, user3Dir); + LocalizedResource lrsrc1_user3 = localizer.getBlob(new LocalResource(key1, false, false), topo3Pna, null); String expectedUserDir1 = joinPath(baseDir.toString(), USERCACHE, user1); - String expectedFileDirUser1 = joinPath(expectedUserDir1, AsyncLocalizer.FILECACHE, AsyncLocalizer.FILESDIR); + String expectedFileDirUser1 = joinPath(expectedUserDir1, LocalizedResource.FILECACHE, LocalizedResource.FILESDIR); String expectedFileDirUser2 = joinPath(baseDir.toString(), USERCACHE, user2, - AsyncLocalizer.FILECACHE, AsyncLocalizer.FILESDIR); + LocalizedResource.FILECACHE, LocalizedResource.FILESDIR); String expectedFileDirUser3 = joinPath(baseDir.toString(), USERCACHE, user3, - AsyncLocalizer.FILECACHE, AsyncLocalizer.FILESDIR); + LocalizedResource.FILECACHE, LocalizedResource.FILESDIR); assertTrue("user filecache dir user1 not created", new File(expectedFileDirUser1).exists()); assertTrue("user filecache dir user2 not created", new File(expectedFileDirUser2).exists()); assertTrue("user filecache dir user3 not created", new File(expectedFileDirUser3).exists()); - File keyFile = new File(expectedFileDirUser1, key1 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); - File keyFile2 = new File(expectedFileDirUser2, key2 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); - File keyFile3 = new File(expectedFileDirUser3, key3 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); - File keyFile1user3 = new File(expectedFileDirUser3, key1 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); + File keyFile = new File(expectedFileDirUser1, key1 + LocalizedResource.CURRENT_BLOB_SUFFIX); + File keyFile2 = new File(expectedFileDirUser2, key2 + LocalizedResource.CURRENT_BLOB_SUFFIX); + File keyFile3 = new File(expectedFileDirUser3, key3 + LocalizedResource.CURRENT_BLOB_SUFFIX); + File keyFile1user3 = new File(expectedFileDirUser3, key1 + LocalizedResource.CURRENT_BLOB_SUFFIX); assertTrue("blob not created", keyFile.exists()); assertTrue("blob not created", keyFile2.exists()); assertTrue("blob not created", keyFile3.exists()); assertTrue("blob not created", keyFile1user3.exists()); - LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1); - assertEquals("local resource set size wrong", 1, lrsrcSet.getSize()); - LocalizedResourceSet lrsrcSet2 = localizer.getUserResources().get(user2); - assertEquals("local resource set size wrong", 1, lrsrcSet2.getSize()); - LocalizedResourceSet lrsrcSet3 = localizer.getUserResources().get(user3); - assertEquals("local resource set size wrong", 2, lrsrcSet3.getSize()); + ConcurrentMap lrsrcSet = localizer.getUserFiles().get(user1); + assertEquals("local resource set size wrong", 1, lrsrcSet.size()); + ConcurrentMap lrsrcSet2 = localizer.getUserFiles().get(user2); + assertEquals("local resource set size wrong", 1, lrsrcSet2.size()); + ConcurrentMap lrsrcSet3 = localizer.getUserFiles().get(user3); + assertEquals("local resource set size wrong", 2, lrsrcSet3.size()); - localizer.removeBlobReference(lrsrc.getKey(), user1, topo1, false); + localizer.removeBlobReference(lrsrc.getKey(), topo1Pna, false); // should remove key1 localizer.cleanup(); - lrsrcSet = localizer.getUserResources().get(user1); - lrsrcSet3 = localizer.getUserResources().get(user3); + lrsrcSet = localizer.getUserFiles().get(user1); + lrsrcSet3 = localizer.getUserFiles().get(user3); assertNull("user set should be null", lrsrcSet); assertFalse("blob dir not deleted", new File(expectedFileDirUser1).exists()); assertFalse("blob dir not deleted", new File(expectedUserDir1).exists()); - assertEquals("local resource set size wrong", 2, lrsrcSet3.getSize()); + assertEquals("local resource set size wrong", 2, lrsrcSet3.size()); assertTrue("blob deleted", keyFile2.exists()); assertFalse("blob not deleted", keyFile.exists()); @@ -849,55 +957,62 @@ public void testMultipleUsers() throws Exception { @Test public void testUpdate() throws Exception { - Map conf = new HashMap(); + Map conf = new HashMap<>(); // set clean time really high so doesn't kick in conf.put(DaemonConfig.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60*60*1000); String key1 = "key1"; String topo1 = "topo1"; String topo2 = "topo2"; - AsyncLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); + TestLocalizer localizer = new TestLocalizer(conf, baseDir.toString()); ReadableBlobMeta rbm = new ReadableBlobMeta(); rbm.set_version(1); rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); when(mockblobstore.getBlobMeta(key1)).thenReturn(rbm); - when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta()); + when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta(1)); File user1Dir = localizer.getLocalUserFileCacheDir(user1); assertTrue("failed to create user dir", user1Dir.mkdirs()); - LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, false), user1, topo1, - user1Dir); + LocalAssignment topo1Assignment = new LocalAssignment(topo1, Collections.emptyList()); + topo1Assignment.set_owner(user1); + PortAndAssignment topo1Pna = new PortAndAssignment(1, topo1Assignment); + LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, false, false), topo1Pna, null); String expectedUserDir = joinPath(baseDir.toString(), USERCACHE, user1); - String expectedFileDir = joinPath(expectedUserDir, AsyncLocalizer.FILECACHE, AsyncLocalizer.FILESDIR); + String expectedFileDir = joinPath(expectedUserDir, LocalizedResource.FILECACHE, LocalizedResource.FILESDIR); assertTrue("user filecache dir not created", new File(expectedFileDir).exists()); - File keyFile = new File(expectedFileDir, key1); - File keyFileCurrentSymlink = new File(expectedFileDir, key1 + ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); + Path keyVersionFile = Paths.get(expectedFileDir, key1 + ".version"); + File keyFileCurrentSymlink = new File(expectedFileDir, key1 + LocalizedResource.CURRENT_BLOB_SUFFIX); assertTrue("blob not created", keyFileCurrentSymlink.exists()); - File versionFile = new File(expectedFileDir, key1 + ServerUtils.DEFAULT_BLOB_VERSION_SUFFIX); + File versionFile = new File(expectedFileDir, key1 + LocalizedResource.BLOB_VERSION_SUFFIX); assertTrue("blob version file not created", versionFile.exists()); - assertEquals("blob version not correct", 1, ServerUtils.localVersionOfBlob(keyFile.toString())); + assertEquals("blob version not correct", 1, LocalizedResource.localVersionOfBlob(keyVersionFile)); - LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1); - assertEquals("local resource set size wrong", 1, lrsrcSet.getSize()); + ConcurrentMap lrsrcSet = localizer.getUserFiles().get(user1); + assertEquals("local resource set size wrong", 1, lrsrcSet.size()); // test another topology getting blob with updated version - it should update version now rbm.set_version(2); + when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta(2)); - localizer.getBlob(new LocalResource(key1, false), user1, topo2, user1Dir); + LocalAssignment topo2Assignment = new LocalAssignment(topo2, Collections.emptyList()); + topo2Assignment.set_owner(user1); + PortAndAssignment topo2Pna = new PortAndAssignment(1, topo2Assignment); + localizer.getBlob(new LocalResource(key1, false, false), topo2Pna, null); assertTrue("blob version file not created", versionFile.exists()); - assertEquals("blob version not correct", 2, ServerUtils.localVersionOfBlob(keyFile.toString())); - assertTrue("blob file with version 2 not created", new File(keyFile + ".2").exists()); + assertEquals("blob version not correct", 2, LocalizedResource.localVersionOfBlob(keyVersionFile)); + assertTrue("blob file with version 2 not created", new File(expectedFileDir, key1 + ".2").exists()); // now test regular updateBlob rbm.set_version(3); + when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta(3)); - ArrayList arr = new ArrayList(); - arr.add(new LocalResource(key1, false)); - localizer.updateBlobs(arr, user1); + ArrayList arr = new ArrayList<>(); + arr.add(new LocalResource(key1, false, false)); + localizer.updateBlobs(); assertTrue("blob version file not created", versionFile.exists()); - assertEquals("blob version not correct", 3, ServerUtils.localVersionOfBlob(keyFile.toString())); - assertTrue("blob file with version 3 not created", new File(keyFile + ".3").exists()); + assertEquals("blob version not correct", 3, LocalizedResource.localVersionOfBlob(keyVersionFile)); + assertTrue("blob file with version 3 not created", new File(expectedFileDir, key1 + ".3").exists()); } } diff --git a/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java b/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java index 150bea579ec..4ebf5cfd4c3 100644 --- a/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java +++ b/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java @@ -18,71 +18,96 @@ package org.apache.storm.localizer; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.storm.blobstore.ClientBlobStore; +import org.apache.storm.daemon.supervisor.IAdvancedFSOps; +import org.apache.storm.generated.LocalAssignment; +import org.apache.storm.generated.ReadableBlobMeta; +import org.apache.storm.generated.SettableBlobMeta; import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.mockito.Matchers.any; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; public class LocalizedResourceRetentionSetTest { - @Test - public void testAddResources() throws Exception { - LocalizedResourceRetentionSet lrretset = new LocalizedResourceRetentionSet(10); - LocalizedResourceSet lrset = new LocalizedResourceSet("user1"); - LocalizedResource localresource1 = new LocalizedResource("key1", "testfile1", false, "topo1"); - LocalizedResource localresource2 = new LocalizedResource("key2", "testfile2", false, "topo1"); - // check adding reference to local resource with topology of same name - localresource2.addReference(("topo2")); + @Test + public void testAddResources() throws Exception { + PortAndAssignment pna1 = new PortAndAssignment(1, new LocalAssignment("topo1", Collections.emptyList())); + PortAndAssignment pna2 = new PortAndAssignment(1, new LocalAssignment("topo2", Collections.emptyList())); + String user = "user"; + Map conf = new HashMap<>(); + IAdvancedFSOps ops = mock(IAdvancedFSOps.class); + LocalizedResourceRetentionSet lrretset = new LocalizedResourceRetentionSet(10); + ConcurrentMap lrset = new ConcurrentHashMap<>(); + LocalizedResource localresource1 = new LocalizedResource("key1", Paths.get("testfile1"), false, ops, conf, user); + localresource1.addReference(pna1, null); + LocalizedResource localresource2 = new LocalizedResource("key2", Paths.get("testfile2"), false, ops, conf, user); + localresource2.addReference(pna1, null); + // check adding reference to local resource with topology of same name + localresource2.addReference(pna2, null); - lrset.add("key1", localresource1, false); - lrset.add("key2", localresource2, false); - lrretset.addResources(lrset); - assertEquals("number to clean is not 0 " + lrretset.noReferences, 0, lrretset.getSizeWithNoReferences()); - localresource1.removeReference(("topo1")); - lrretset = new LocalizedResourceRetentionSet(10); - lrretset.addResources(lrset); - assertEquals("number to clean is not 1 " + lrretset.noReferences, 1, lrretset.getSizeWithNoReferences()); + lrset.put("key1", localresource1); + lrset.put("key2", localresource2); + lrretset.addResources(lrset); + assertEquals("number to clean is not 0 " + lrretset.noReferences, 0, lrretset.getSizeWithNoReferences()); + localresource1.removeReference(pna1); + lrretset = new LocalizedResourceRetentionSet(10); + lrretset.addResources(lrset); + assertEquals("number to clean is not 1 " + lrretset.noReferences, 1, lrretset.getSizeWithNoReferences()); - localresource2.removeReference(("topo1")); - lrretset = new LocalizedResourceRetentionSet(10); - lrretset.addResources(lrset); - assertEquals("number to clean is not 1 " + lrretset.noReferences, 1, lrretset.getSizeWithNoReferences()); + localresource2.removeReference(pna1); + lrretset = new LocalizedResourceRetentionSet(10); + lrretset.addResources(lrset); + assertEquals("number to clean is not 1 " + lrretset.noReferences, 1, lrretset.getSizeWithNoReferences()); - localresource2.removeReference(("topo2")); - lrretset = new LocalizedResourceRetentionSet(10); - lrretset.addResources(lrset); - assertEquals("number to clean is not 2 " + lrretset.noReferences, 2, lrretset.getSizeWithNoReferences()); - } + localresource2.removeReference(pna2); + lrretset = new LocalizedResourceRetentionSet(10); + lrretset.addResources(lrset); + assertEquals("number to clean is not 2 " + lrretset.noReferences, 2, lrretset.getSizeWithNoReferences()); + } - @Test - public void testCleanup() throws Exception { - LocalizedResourceRetentionSet lrretset = spy(new LocalizedResourceRetentionSet(10)); - LocalizedResourceSet lrset = new LocalizedResourceSet("user1"); - // no reference to key1 - LocalizedResource localresource1 = new LocalizedResource("key1", "./target/TESTING/testfile1", false); - localresource1.setSize(10); - // no reference to archive1 - LocalizedResource archiveresource1 = new LocalizedResource("archive1", "./target/TESTING/testarchive1", true); - archiveresource1.setSize(20); - // reference to key2 - LocalizedResource localresource2 = new LocalizedResource("key2", "./target/TESTING/testfile2", false, "topo1"); - // check adding reference to local resource with topology of same name - localresource2.addReference(("topo1")); - localresource2.setSize(10); - lrset.add("key1", localresource1, false); - lrset.add("key2", localresource2, false); - lrset.add("archive1", archiveresource1, true); + @Test + public void testCleanup() throws Exception { + ClientBlobStore mockBlobstore = mock(ClientBlobStore.class); + when (mockBlobstore.getBlobMeta(any())).thenReturn(new ReadableBlobMeta(new SettableBlobMeta(), 1)); + PortAndAssignment pna1 = new PortAndAssignment(1, new LocalAssignment("topo1", Collections.emptyList())); + String user = "user"; + Map conf = new HashMap<>(); + IAdvancedFSOps ops = mock(IAdvancedFSOps.class); + LocalizedResourceRetentionSet lrretset = spy(new LocalizedResourceRetentionSet(10)); + ConcurrentMap lrFiles = new ConcurrentHashMap<>(); + ConcurrentMap lrArchives = new ConcurrentHashMap<>(); + // no reference to key1 + LocalizedResource localresource1 = new LocalizedResource("key1", Paths.get("./target/TESTING/testfile1"), false, ops, conf, + user); + localresource1.setSize(10); + // no reference to archive1 + LocalizedResource archiveresource1 = new LocalizedResource("archive1", Paths.get("./target/TESTING/testarchive1"), true, ops, + conf, user); + archiveresource1.setSize(20); + // reference to key2 + LocalizedResource localresource2 = new LocalizedResource("key2", Paths.get("./target/TESTING/testfile2"), false, ops, conf, + user); + localresource2.addReference(pna1, null); + // check adding reference to local resource with topology of same name + localresource2.addReference(pna1, null); + localresource2.setSize(10); + lrFiles.put("key1", localresource1); + lrFiles.put("key2", localresource2); + lrArchives.put("archive1", archiveresource1); - lrretset.addResources(lrset); - assertEquals("number to clean is not 2", 2, lrretset.getSizeWithNoReferences()); + lrretset.addResources(lrFiles); + lrretset.addResources(lrArchives); + assertEquals("number to clean is not 2", 2, lrretset.getSizeWithNoReferences()); - // make deleteUnderlyingResource return true even though file doesn't exist - doReturn(true).when(lrretset).deleteResource(any(), any()); - - lrretset.cleanup(); - assertEquals("resource not cleaned up", 0, lrretset.getSizeWithNoReferences()); - } + lrretset.cleanup(mockBlobstore); + assertEquals("resource not cleaned up", 0, lrretset.getSizeWithNoReferences()); + } } diff --git a/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceSetTest.java b/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceSetTest.java deleted file mode 100644 index 550d69506a0..00000000000 --- a/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceSetTest.java +++ /dev/null @@ -1,74 +0,0 @@ -/** - * 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.storm.localizer; - -import org.junit.Test; - -import static org.junit.Assert.*; - -public class LocalizedResourceSetTest { - - private final String user1 = "user1"; - - @Test - public void testGetUser() throws Exception { - LocalizedResourceSet lrset = new LocalizedResourceSet(user1); - assertEquals("user is wrong", user1, lrset.getUser()); - } - - @Test - public void testGetSize() throws Exception { - LocalizedResourceSet lrset = new LocalizedResourceSet(user1); - LocalizedResource localresource1 = new LocalizedResource("key1", "testfile1", false, "topo1"); - LocalizedResource localresource2 = new LocalizedResource("key2", "testfile2", true, "topo1"); - assertEquals("size is wrong", 0, lrset.getSize()); - lrset.add("key1", localresource1, false); - assertEquals("size is wrong", 1, lrset.getSize()); - lrset.add("key2", localresource2, true); - assertEquals("size is wrong", 2, lrset.getSize()); - } - - @Test - public void testGet() throws Exception { - LocalizedResourceSet lrset = new LocalizedResourceSet(user1); - LocalizedResource localresource1 = new LocalizedResource("key1", "testfile1", false, "topo1"); - LocalizedResource localresource2 = new LocalizedResource("key2", "testfile2", true, "topo1"); - lrset.add("key1", localresource1, false); - lrset.add("key2", localresource2, true); - assertEquals("get doesn't return same object", localresource1, lrset.get("key1", false)); - assertEquals("get doesn't return same object", localresource2, lrset.get("key2", true)); - - } - - @Test - public void testExists() throws Exception { - LocalizedResourceSet lrset = new LocalizedResourceSet(user1); - LocalizedResource localresource1 = new LocalizedResource("key1", "testfile1", false, "topo1"); - LocalizedResource localresource2 = new LocalizedResource("key2", "testfile2", true, "topo1"); - lrset.add("key1", localresource1, false); - lrset.add("key2", localresource2, true); - assertEquals("doesn't exist", true, lrset.exists("key1", false)); - assertEquals("doesn't exist", true, lrset.exists("key2", true)); - boolean val = lrset.remove(localresource1); - assertTrue("remove failed", val); - assertEquals("does exist", false, lrset.exists("key1", false)); - assertEquals("doesn't exist", true, lrset.exists("key2", true)); - val = lrset.remove(localresource1); - assertFalse("remove success when shouldn't have been", val); - } -} From 20a1477d5d3927db88ee5f7d95f1578aaf2abccc Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 11 Oct 2017 10:09:31 -0500 Subject: [PATCH 2/4] STORM-2759: Addressed review comments --- .../storm/localizer/AsyncLocalizerTest.java | 31 ------------------- 1 file changed, 31 deletions(-) diff --git a/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java b/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java index 251f029f80e..63f4e9ce1ab 100644 --- a/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java +++ b/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java @@ -96,36 +96,6 @@ private static String getTestLocalizerRoot() { return f.getPath(); } - private class MockInputStreamWithMeta extends InputStreamWithMeta { - private int at = 0; - private final int len; - private final int version; - - public MockInputStreamWithMeta(int len, int version) { - this.len = len; - this.version = version; - } - - @Override - public long getVersion() throws IOException { - return version; - } - - @Override - public long getFileLength() throws IOException { - return len; - } - - @Override - public int read() throws IOException { - at++; - if (at > len) { - return -1; - } - return 0; - } - } - @Test public void testRequestDownloadBaseTopologyBlobs() throws Exception { final String topoId = "TOPO"; @@ -312,7 +282,6 @@ synchronized void addReferences(List localresource, PortAndAssign if (lrsrcSet != null) { LocalizedResource lrsrc = lrsrcSet.get(blob.getBlobName()); if (lrsrc != null) { - lrsrc.addReference(pna, blob.needsCallback() ? cb : null); lrsrc.addReference(pna, blob.needsCallback() ? cb : null); LOG.debug("added reference for topo: {} key: {}", pna, blob); } else { From be8d6eec2d46955a37ef147caeca2039f8d6b368 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 19 Oct 2017 10:11:11 -0500 Subject: [PATCH 3/4] Addressed review comment --- .../src/jvm/org/apache/storm/utils/NimbusClient.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java b/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java index 8589d65f9de..89e7087c757 100644 --- a/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java +++ b/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java @@ -90,6 +90,10 @@ public static NimbusClient getConfiguredClient(Map conf, Integer private static String oldLeader = ""; private static synchronized boolean shouldLogLeader(String leader) { + if (LOG.isDebugEnabled()) { + //If debug logging is turned on we shoudl just log the leader all the time.... + return true; + } //Only log if the leader has changed. It is not interesting otherwise. if (oldLeader.equals(leader)) { return false; @@ -137,7 +141,7 @@ public static NimbusClient getConfiguredClientAs(Map conf, Strin nimbusSummary = client.getClient().getLeader(); if (nimbusSummary != null) { String leaderNimbus = nimbusSummary.get_host() + ":" + nimbusSummary.get_port(); - if (LOG.isDebugEnabled() || shouldLogLeader(leaderNimbus)) { + if (shouldLogLeader(leaderNimbus)) { LOG.info("Found leader nimbus : {}", leaderNimbus); } if (nimbusSummary.get_host().equals(host) && nimbusSummary.get_port() == port) { From 7b9f122f206d5b8dab11dadf17b6abdea03fdde8 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 26 Oct 2017 14:45:50 -0500 Subject: [PATCH 4/4] Addressed more review comments --- .../src/jvm/org/apache/storm/utils/NimbusClient.java | 9 ++++++++- .../apache/storm/daemon/supervisor/SupervisorUtils.java | 8 ++++---- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java b/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java index 89e7087c757..2f4f6dd7933 100644 --- a/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java +++ b/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java @@ -89,9 +89,16 @@ public static NimbusClient getConfiguredClient(Map conf, Integer private static String oldLeader = ""; + /** + * Check to see if we should log the leader we are connecting to or not. This typically happens when the leader changes or if debug + * logging is enabled. The code remembers the last leader it was called with, but it should be transparent to the caller. + * @param leader the leader we are trying to connect to. + * @return true if it should be logged else false. + */ private static synchronized boolean shouldLogLeader(String leader) { + assert leader != null; if (LOG.isDebugEnabled()) { - //If debug logging is turned on we shoudl just log the leader all the time.... + //If debug logging is turned on we should just log the leader all the time.... return true; } //Only log if the leader has changed. It is not interesting otherwise. diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java index 30031f65326..81421b24cd9 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java @@ -77,10 +77,10 @@ public static boolean shouldUncompressBlob(Map blobInfo) { * Given the blob information returns the value of the workerRestart field, handling it either being a string or a boolean value, or * if it's not specified then returns false * - * @param blobInfo - * @return + * @param blobInfo the info for the blob. + * @return true if the blob needs a worker restart by way of the callback else false. */ - public static boolean needsCallback(Map blobInfo) { + public static boolean blobNeedsWorkerRestart(Map blobInfo) { return ObjectReader.getBoolean(blobInfo.get("workerRestart"), false); } @@ -95,7 +95,7 @@ public static List blobstoreMapToLocalresources(Map> map : blobstoreMap.entrySet()) { Map blobConf = map.getValue(); - LocalResource localResource = new LocalResource(map.getKey(), shouldUncompressBlob(blobConf), needsCallback(blobConf)); + LocalResource localResource = new LocalResource(map.getKey(), shouldUncompressBlob(blobConf), blobNeedsWorkerRestart(blobConf)); localResourceList.add(localResource); } }