diff --git a/common/src/main/java/io/druid/concurrent/Execs.java b/common/src/main/java/io/druid/concurrent/Execs.java index 3122eaa62224..04b84f135bbc 100644 --- a/common/src/main/java/io/druid/concurrent/Execs.java +++ b/common/src/main/java/io/druid/concurrent/Execs.java @@ -19,6 +19,8 @@ package io.druid.concurrent; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; import com.google.common.util.concurrent.ThreadFactoryBuilder; import javax.annotation.Nullable; @@ -83,9 +85,21 @@ public static ThreadFactory makeThreadFactory(@NotNull String nameFormat, @Nulla if (priority != null) { builder.setPriority(priority); } + return builder.build(); } + public static Thread makeThread(String name, Runnable runnable, boolean isDaemon) + { + Preconditions.checkArgument(!Strings.isNullOrEmpty(name), "name null/empty"); + Preconditions.checkNotNull(runnable, "null runnable"); + + Thread t = new Thread(runnable); + t.setName(name); + t.setDaemon(isDaemon); + return t; + } + /** * @param nameFormat nameformat for threadFactory * @param capacity maximum capacity after which the executorService will block on accepting new tasks diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index cd57d92e5021..080d71f19c7d 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -64,86 +64,123 @@ These endpoints will return one of the following results: ## Configuration propagation behavior The configuration is propagated to the query serving nodes (broker / router / peon / historical) by the coordinator. -The query serving nodes have an internal API for managing `POST`/`GET`/`DELETE` of lookups. -The coordinator periodically checks the dynamic configuration for changes and, when it detects a change it does the following: +The query serving nodes have an internal API for managing lookups on the node and those are used by the coordinator. +The coordinator periodically checks if any of the nodes need to load/drop lookups and updates them appropriately. -1. Post all lookups for a tier to all Druid nodes within that tier. -2. Delete lookups from a tier which were dropped between the prior configuration values and this one. - -If there is no configuration change, the coordinator checks for any nodes which might be new since the last time it propagated lookups and adds all lookups for that node (assuming that node's tier has lookups). -If there are errors while trying to add or update configuration on a node, that node is temporarily skipped until the next management period. The next management period the update will attempt to be propagated again. -If there is an error while trying to delete a lookup from a node (or if a node is down when the coordinator is propagating the config), the delete is not attempted again. In such a case it is possible that a node has lookups that are no longer managed by the coordinator. ## Bulk update Lookups can be updated in bulk by posting a JSON object to `/druid/coordinator/v1/lookups`. The format of the json object is as follows: ```json { - "tierName": { - "lookupExtractorFactoryName": { - "type": "someExtractorFactoryType", - "someExtractorField": "someExtractorValue" + "": { + "": { + "version": "", + "lookupExtractorFactory": { + "type": "", + "": "" + } } } } ``` -So a config might look something like: +Note that "version" is an arbitrary string assigned by the user, when making updates to existing lookup then user would need to specify a lexicographically higher version. + +For example, a config might look something like: ```json { - "__default": { - "country_code": { - "type": "map", - "map": {"77483": "United States"} - }, - "site_id": { - "type": "cachedNamespace", - "extractionNamespace": { - "type": "jdbc", - "connectorConfig": { - "createTables": true, - "connectURI": "jdbc:mysql:\/\/localhost:3306\/druid", - "user": "druid", - "password": "diurd" - }, - "table": "lookupTable", - "keyColumn": "country_id", - "valueColumn": "country_name", - "tsColumn": "timeColumn" + "__default": { + "country_code": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "77483": "United States" + } + } + }, + "site_id": { + "version": "v0", + "lookupExtractorFactory": { + "type": "cachedNamespace", + "extractionNamespace": { + "type": "jdbc", + "connectorConfig": { + "createTables": true, + "connectURI": "jdbc:mysql:\/\/localhost:3306\/druid", + "user": "druid", + "password": "diurd" }, - "firstCacheTimeout": 120000, - "injective":true - }, - "site_id_customer1": { - "type": "map", - "map": {"847632": "Internal Use Only"} + "table": "lookupTable", + "keyColumn": "country_id", + "valueColumn": "country_name", + "tsColumn": "timeColumn" }, - "site_id_customer2": { - "type": "map", - "map": {"AHF77": "Home"} + "firstCacheTimeout": 120000, + "injective": true + } + }, + "site_id_customer1": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "847632": "Internal Use Only" } + } }, - "realtime_customer1": { - "country_code": { - "type": "map", - "map": {"77483": "United States"} - }, - "site_id_customer1": { - "type": "map", - "map": {"847632": "Internal Use Only"} + "site_id_customer2": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "AHF77": "Home" } + } + } + }, + "realtime_customer1": { + "country_code": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "77483": "United States" + } + } }, - "realtime_customer2": { - "country_code": { - "type": "map", - "map": {"77483": "United States"} - }, - "site_id_customer2": { - "type": "map", - "map": {"AHF77": "Home"} + "site_id_customer1": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "847632": "Internal Use Only" + } + } + } + }, + "realtime_customer2": { + "country_code": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "77483": "United States" + } + } + }, + "site_id_customer2": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "AHF77": "Home" } + } } + } } ``` @@ -156,8 +193,13 @@ For example, a post to `/druid/coordinator/v1/lookups/realtime_customer1/site_id ```json { - "type": "map", - "map": {"847632": "Internal Use Only"} + "version": "v1", + "lookupExtractorFactory": { + "type": "map", + "map": { + "847632": "Internal Use Only" + } + } } ``` @@ -170,8 +212,13 @@ Using the prior example, a `GET` to `/druid/coordinator/v1/lookups/realtime_cust ```json { - "type": "map", - "map": {"AHF77": "Home"} + "version": "v1", + "lookupExtractorFactory": { + "type": "map", + "map": { + "AHF77": "Home" + } + } } ``` @@ -189,9 +236,8 @@ A `GET` to `/druid/coordinator/v1/lookups/{tier}` will return a list of known lo The Peon, Router, Broker, and Historical nodes all have the ability to consume lookup configuration. There is an internal API these nodes use to list/load/drop their lookups starting at `/druid/listen/v1/lookups`. -These follow the same convention for return values as the cluster wide dynamic configuration. -Usage of these endpoints is quite advanced and not recommended for most users. -The endpoints are as follows: +These follow the same convention for return values as the cluster wide dynamic configuration. Following endpoints +can be used for debugging purposes but not otherwise. ## Get Lookups @@ -199,14 +245,17 @@ A `GET` to the node at `/druid/listen/v1/lookups` will return a json map of all The return value will be a json map of the lookups to their extractor factories. ```json - { - "some_lookup_name": { - "type": "map", - "map": {"77483": "United States"} + "site_id_customer2": { + "version": "v1", + "lookupExtractorFactory": { + "type": "map", + "map": { + "AHF77": "Home" + } + } } } - ``` ## Get Lookup @@ -216,70 +265,16 @@ The return value will be the json representation of the factory. ```json { - "type": "map", - "map": {"77483", "United States"} -} -``` - -## Bulk Add or Update Lookups - -A `POST` to the node at `/druid/listen/v1/lookups` of a JSON map of lookup names to LookupExtractorFactory will cause the service to add or update its lookups. -The return value will be a JSON map in the following format: - -```json -{ - "status": "accepted", - "failedUpdates": {} -} - -``` - -If a lookup cannot be started, or is left in an undefined state, the lookup in error will be returned in the `failedUpdates` field as per: - -```json -{ - "status": "accepted", - "failedUpdates": { - "country_code": { - "type": "map", - "map": {"77483": "United States"} - } - } -} - -``` - -The `failedUpdates` field of the return value should be checked if a user is wanting to assure that every update succeeded. - -## Add or Update Lookup - -A `POST` to the node at `/druid/listen/v1/lookups/some_lookup_name` will behave very similarly to a bulk update. - -If `some_lookup_name` is desired to have the LookupExtractorFactory definition of - -```json -{ - "type": "map", - "map": {"77483": "United States"} -} -``` - -Then a post to `/druid/listen/v1/lookups/some_lookup_name` will behave the same as a `POST` to `/druid/listen/v1/lookups` of - -```json - -{ - "some_lookup_name": { + "version": "v1", + "lookupExtractorFactory": { "type": "map", - "map": {"77483": "United States"} + "map": { + "AHF77": "Home" + } } } - ``` -## Remove a Lookup -A `DELETE` to `/druid/listen/v1/lookups/some_lookup_name` will remove that lookup from the node. Success will reflect the ID. - # Configuration See the [coordinator configuration guilde](../configuration/coordinator.html) for coordinator configuration @@ -294,10 +289,9 @@ To configure the behavior of the dynamic configuration manager, use the followin |Property|Description|Default| |--------|-----------|-------| -|`druid.manager.lookups.hostDeleteTimeout`|Timeout (in ms) PER HOST for processing DELETE requests for dropping lookups|`1000`(1 second)| -|`druid.manager.lookups.hostUpdateTimeout`|Timeout (in ms) PER HOST for processing an update/add (POST) for new or updated lookups|`10000`(10 seconds)| -|`druid.manager.lookups.updateAllTimeout`|Timeout (in ms) TOTAL for processing update/adds on ALL hosts. Safety valve in case too many hosts timeout on their update|`60000`(1 minute)| -|`druid.manager.lookups.period`|How long to pause between management cycles|`30000`(30 seconds)| +|`druid.manager.lookups.hostTimeout`|Timeout (in ms) PER HOST for processing request|`2000`(2 seconds)| +|`druid.manager.lookups.allHostTimeout`|Timeout (in ms) to finish lookup management on all the nodes.|`900000`(15 mins)| +|`druid.manager.lookups.period`|How long to pause between management cycles|`120000`(2 mins)| |`druid.manager.lookups.threadPoolSize`|Number of service nodes that can be managed concurrently|`10`| ## Saving configuration across restarts @@ -312,4 +306,11 @@ It is possible to save the configuration across restarts such that a node will n Lookup implementations can provide some introspection capabilities by implementing `LookupIntrospectHandler`. User will send request to `/druid/lookups/v1/introspect/{lookupId}` to enable introspection on a given lookup. -For instance you can list all the keys/values of a map based lookup by issuing a `GET` request to `/druid/lookups/v1/introspect/{lookupId}/keys"` or `/druid/lookups/v1/introspect/{lookupId}/values"` +For instance you can list all the keys/values of a map based lookup by issuing a `GET` request to `/druid/lookups/v1/introspect/{lookupId}/keys"` or `/druid/lookups/v1/introspect/{lookupId}/values"` + +## Druid version 0.10.0 to 0.10.1 upgrade/downgrade +Overall druid cluster lookups configuration is persisted in metadata store and also individual lookup nodes optionally persist a snapshot of loaded lookups on disk. +If upgrading from druid version 0.10.0 to 0.10.1, then migration for all persisted metadata is handled automatically. +If downgrading from 0.10.1 to 0.9.0 then lookups updates done via coordinator while 0.10.1 was running, would be lost. + + diff --git a/java-util/src/main/java/io/druid/java/util/common/IOE.java b/java-util/src/main/java/io/druid/java/util/common/IOE.java new file mode 100644 index 000000000000..c92d37ad1b43 --- /dev/null +++ b/java-util/src/main/java/io/druid/java/util/common/IOE.java @@ -0,0 +1,37 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.java.util.common; + +import java.io.IOException; + +/** + */ +public class IOE extends IOException +{ + public IOE(String formatText, Object... arguments) + { + super(StringUtils.safeFormat(formatText, arguments)); + } + + public IOE(Throwable cause, String formatText, Object... arguments) + { + super(StringUtils.safeFormat(formatText, arguments), cause); + } +} diff --git a/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java index 498fb47a8b96..97448a55cb10 100644 --- a/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java @@ -140,7 +140,7 @@ public ExtractionFn getExtractionFn() lookupReferencesManager.get(name), "Lookup [%s] not found", name - ).get(); + ).getLookupExtractorFactory().get(); return new LookupExtractionFn( lookupExtractor, diff --git a/processing/src/main/java/io/druid/query/lookup/LookupBean.java b/processing/src/main/java/io/druid/query/lookup/LookupBean.java index 4c1acd4b2c5d..6c7ded2405a4 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupBean.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupBean.java @@ -19,15 +19,65 @@ package io.druid.query.lookup; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import java.util.Objects; -//TODO merge this code to the same definition when pr/1576 is merged class LookupBean { + private final LookupExtractorFactoryContainer container; + private final String name; + + //to support rollback from 0.10.1 to 0.9.0 if necessary + @Deprecated + private final LookupExtractorFactory factory; + + + @JsonCreator + public LookupBean( + @JsonProperty("name") String name, + //kept for backward compatibility with druid ver <= 0.10.0 persisted snapshots + @Deprecated @JsonProperty("factory") LookupExtractorFactory factory, + @JsonProperty("container") LookupExtractorFactoryContainer container + ) + { + Preconditions.checkArgument(factory != null || container != null, "either one of factory or container must exist"); + + this.name = name; + this.container = container != null ? container : new LookupExtractorFactoryContainer(null, factory); + this.factory = factory != null ? factory : container.getLookupExtractorFactory(); + } + @JsonProperty - LookupExtractorFactory factory; + public String getName() + { + return name; + } + + @JsonProperty + public LookupExtractorFactoryContainer getContainer() + { + return container; + } + + @Deprecated @JsonProperty - String name; + public LookupExtractorFactory getFactory() + { + return factory; + } + + @Override + public String toString() + { + return "LookupBean{" + + "container=" + container + + ", name='" + name + '\'' + + ", factory=" + factory + + '}'; + } @Override public boolean equals(Object o) @@ -35,16 +85,18 @@ public boolean equals(Object o) if (this == o) { return true; } - if (!(o instanceof LookupBean)) { + if (o == null || getClass() != o.getClass()) { return false; } - LookupBean that = (LookupBean) o; + return Objects.equals(container, that.container) && + Objects.equals(name, that.name) && + Objects.equals(factory, that.factory); + } - if (!factory.equals(that.factory)) { - return false; - } - return name.equals(that.name); - + @Override + public int hashCode() + { + return Objects.hash(container, name, factory); } } diff --git a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java index bec1e07feb07..cb881296c834 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactory.java @@ -50,12 +50,12 @@ public interface LookupExtractorFactory extends Supplier * @return Returns false if not successfully closed the {@link LookupExtractor} otherwise returns true */ public boolean close(); + /** - * Determine if this LookupExtractorFactory should replace some other LookupExtractorFactory. - * This is used to implement no-down-time - * @param other Some other LookupExtractorFactory which might need replaced - * @return `true` if the other should be replaced by this one. `false` if this one should not replace the other factory + * This method is deprecated and is not removed only to allow 0.10.0 to 0.10.1 transition. It is not used + * on a cluster that is running 0.10.1. It will be removed in a later release. */ + @Deprecated boolean replaces(@Nullable LookupExtractorFactory other); /** diff --git a/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java new file mode 100644 index 000000000000..6d9f73d92113 --- /dev/null +++ b/processing/src/main/java/io/druid/query/lookup/LookupExtractorFactoryContainer.java @@ -0,0 +1,96 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.lookup; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.Ordering; + +import java.util.Objects; + +/** + */ +public class LookupExtractorFactoryContainer +{ + private final static Ordering VERSION_COMPARATOR = Ordering.natural().nullsFirst(); + + private final String version; + private final LookupExtractorFactory lookupExtractorFactory; + + @JsonCreator + public LookupExtractorFactoryContainer( + @JsonProperty("version") String version, + @JsonProperty("lookupExtractorFactory") LookupExtractorFactory lookupExtractorFactory + ) + { + this.version = version; + this.lookupExtractorFactory = Preconditions.checkNotNull(lookupExtractorFactory, "null factory"); + } + + @JsonProperty + public String getVersion() + { + return version; + } + + @JsonProperty + public LookupExtractorFactory getLookupExtractorFactory() + { + return lookupExtractorFactory; + } + + public boolean replaces(LookupExtractorFactoryContainer other) { + if (version == null && other.getVersion() == null) { + return this.lookupExtractorFactory.replaces(other.getLookupExtractorFactory()); + } + + return VERSION_COMPARATOR.compare(version, other.getVersion()) > 0; + } + + @Override + public String toString() + { + return "LookupExtractorFactoryContainer{" + + "version='" + version + '\'' + + ", lookupExtractorFactory=" + lookupExtractorFactory + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LookupExtractorFactoryContainer that = (LookupExtractorFactoryContainer) o; + return Objects.equals(version, that.version) && + Objects.equals(lookupExtractorFactory, that.lookupExtractorFactory); + } + + @Override + public int hashCode() + { + return Objects.hash(version, lookupExtractorFactory); + } +} diff --git a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java index c40b972c3327..46158ab177e4 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -21,29 +21,32 @@ import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; -import com.google.common.base.Throwables; -import com.google.common.collect.Collections2; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; +import com.metamx.emitter.EmittingLogger; +import io.druid.concurrent.Execs; +import io.druid.concurrent.LifecycleLock; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Json; import io.druid.java.util.common.ISE; import io.druid.java.util.common.lifecycle.LifecycleStart; import io.druid.java.util.common.lifecycle.LifecycleStop; -import io.druid.java.util.common.logger.Logger; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.LockSupport; +import java.util.function.Function; /** * This class provide a basic {@link LookupExtractorFactory} references manager. @@ -52,333 +55,400 @@ * It is used by Lookup configuration manager to add/remove or list lookups configuration via HTTP or other protocols. * It does periodic snap shot of the list of lookup in order to bootstrap nodes after restart. */ - @ManageLifecycle public class LookupReferencesManager { - private static final Logger LOGGER = new Logger(LookupReferencesManager.class); - private final ConcurrentMap lookupMap = new ConcurrentHashMap<>(); - // This is a lock against the state of the REFERENCE MANAGER (aka start/stop state), NOT of the lookup itself. - private final ReadWriteLock startStopLock = new ReentrantReadWriteLock(true); - private final AtomicBoolean started = new AtomicBoolean(false); + private static final EmittingLogger LOG = new EmittingLogger(LookupReferencesManager.class); + + // Lookups state (loaded/to-be-loaded/to-be-dropped etc) is managed by immutable LookupUpdateState instance. + // Any update to state is done by creating updated LookupUpdateState instance and atomically setting that + // into the ref here. + // this allows getAllLookupsState() to provide a consistent view without using locks. + @VisibleForTesting + final AtomicReference stateRef = new AtomicReference<>(); + + @VisibleForTesting + final LookupSnapshotTaker lookupSnapshotTaker; + + @VisibleForTesting + final LifecycleLock lifecycleLock = new LifecycleLock(); + + @VisibleForTesting + Thread mainThread; - private final LookupSnapshotTaker lookupSnapshotTaker; + //for unit testing only + private final boolean testMode; @Inject - public LookupReferencesManager(LookupConfig lookupConfig, final @Json ObjectMapper objectMapper) + public LookupReferencesManager(LookupConfig lookupConfig, @Json ObjectMapper objectMapper) + { + this(lookupConfig, objectMapper, false); + } + + @VisibleForTesting + LookupReferencesManager(LookupConfig lookupConfig, ObjectMapper objectMapper, boolean testMode) { if (Strings.isNullOrEmpty(lookupConfig.getSnapshotWorkingDir())) { this.lookupSnapshotTaker = null; } else { this.lookupSnapshotTaker = new LookupSnapshotTaker(objectMapper, lookupConfig.getSnapshotWorkingDir()); } + + this.testMode = testMode; } @LifecycleStart public void start() { - startStopLock.writeLock().lock(); + if (!lifecycleLock.canStart()) { + throw new ISE("can't start."); + } + try { - if (!started.getAndSet(true)) { - if (lookupSnapshotTaker != null) { - final List lookupBeanList = lookupSnapshotTaker.pullExistingSnapshot(); - for (LookupBean lookupBean : lookupBeanList) { - this.put(lookupBean.name, lookupBean.factory); - } - } - LOGGER.info("Started lookup factory references manager"); + LOG.info("LookupReferencesManager is starting."); + + loadSnapshotAndInitStateRef(); + + if (!testMode) { + mainThread = Execs.makeThread( + "LookupReferencesManager-MainThread", + new Runnable() + { + @Override + public void run() + { + try { + + if (!lifecycleLock.awaitStarted()) { + LOG.error("WTF! lifecycle not started, lookup update notices will not be handled."); + return; + } + + while (!Thread.interrupted() && lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) { + try { + handlePendingNotices(); + LockSupport.parkNanos(LookupReferencesManager.this, TimeUnit.MINUTES.toNanos(1)); + } + catch (Throwable t) { + LOG.makeAlert(t, "Error occured while lookup notice handling.").emit(); + } + } + } + catch (Throwable t) { + LOG.error(t, "Error while waiting for lifecycle start. lookup updates notices will not be handled"); + } + finally { + LOG.info("Lookup Management loop exited, Lookup notices are not handled anymore."); + } + } + }, + true + ); + + mainThread.start(); } + + LOG.info("LookupReferencesManager is started."); + lifecycleLock.started(); } finally { - startStopLock.writeLock().unlock(); + lifecycleLock.exitStart(); } } - @LifecycleStop - public void stop() + @VisibleForTesting + void handlePendingNotices() { - startStopLock.writeLock().lock(); - try { - if (started.getAndSet(false)) { - if (lookupSnapshotTaker != null) { - lookupSnapshotTaker.takeSnapshot(getAllAsList()); - } - LOGGER.info("Stopping lookup factory references manager"); - for (String lookupName : lookupMap.keySet()) { - lookupMap.remove(lookupName).close(); - } - } + if (stateRef.get().pendingNotices.isEmpty()) { + return; } - finally { - startStopLock.writeLock().unlock(); + + LookupUpdateState swappedState = atomicallyUpdateStateRef( + oldState -> new LookupUpdateState(oldState.lookupMap, ImmutableList.of(), oldState.pendingNotices) + ); + + Map lookupMap = new HashMap<>(swappedState.lookupMap); + for (Notice notice : swappedState.noticesBeingHandled) { + try { + notice.handle(lookupMap); + } catch (Exception ex) { + LOG.error(ex, "Exception occured while handling lookup notice [%s].", notice); + LOG.makeAlert("Exception occured while handling lookup notice, with message [%s].", ex.getMessage()).emit(); + } } + + takeSnapshot(lookupMap); + + ImmutableMap immutableLookupMap = ImmutableMap.copyOf(lookupMap); + + atomicallyUpdateStateRef( + oldState -> new LookupUpdateState(immutableLookupMap, oldState.pendingNotices, ImmutableList.of()) + ); } - /** - * @param lookupName name of the lookupExtractorFactory object - * @param lookupExtractorFactory {@link LookupExtractorFactory} implementation reference. - * - * @return true if the lookup is added otherwise false. - * - * @throws IllegalStateException If the manager is closed or if start of lookup returns false. - */ - public boolean put(String lookupName, final LookupExtractorFactory lookupExtractorFactory) + @LifecycleStop + public void stop() { - try { - startStopLock.readLock().lockInterruptibly(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); + if (!lifecycleLock.canStop()) { + throw new ISE("can't stop."); } - try { - assertStarted(); - if (lookupMap.containsKey(lookupName)) { - LOGGER.warn("lookup [%s] is not add, another lookup with the same name already exist", lookupName); - return false; + + LOG.info("LookupReferencesManager is stopping."); + + if (!testMode) { + mainThread.interrupt(); + + try { + mainThread.join(); } - if (!lookupExtractorFactory.start()) { - throw new ISE("start method returned false for lookup [%s]", lookupName); + catch (InterruptedException ex) { + throw new ISE("failed to stop, mainThread couldn't finish."); } - final boolean noPrior = null == lookupMap.putIfAbsent(lookupName, lookupExtractorFactory); - if (noPrior) { - if (lookupSnapshotTaker != null) { - lookupSnapshotTaker.takeSnapshot(getAllAsList()); - } - } else { - if (!lookupExtractorFactory.close()) { - throw new ISE("Error closing [%s] on race condition", lookupName); + } + + for (Map.Entry e : stateRef.get().lookupMap.entrySet()) { + try { + LOG.info("Closing lookup [%s]", e.getKey()); + if (!e.getValue().getLookupExtractorFactory().close()) { + LOG.error("Failed to close lookup [%s].", e.getKey()); } } - return noPrior; - } - finally { - startStopLock.readLock().unlock(); + catch (Exception ex) { + LOG.error(ex, "Failed to close lookup [%s].", e.getKey()); + } } + + LOG.info("LookupReferencesManager is stopped."); } - /** - * @param lookups {@link Map} containing all the lookup as one batch. - * - * @throws IllegalStateException if the manager is closed or if {@link LookupExtractorFactory#start()} returns false - */ - public void put(Map lookups) + public void add(String lookupName, LookupExtractorFactoryContainer lookupExtractorFactoryContainer) { - Map failedExtractorFactoryMap = new HashMap<>(); - try { - startStopLock.readLock().lockInterruptibly(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - try { - assertStarted(); - for (Map.Entry entry : lookups.entrySet()) { - final String lookupName = entry.getKey(); - final LookupExtractorFactory lookupExtractorFactory = entry.getValue(); - if (lookupMap.containsKey(lookupName)) { - // Fail early without bothering to start - LOGGER.warn("lookup [%s] is not add, another lookup with the same name already exist", lookupName); - continue; - } - if (!lookupExtractorFactory.start()) { - failedExtractorFactoryMap.put(lookupName, lookupExtractorFactory); - continue; - } - if (null != lookupMap.putIfAbsent(lookupName, lookupExtractorFactory)) { - // handle race - LOGGER.warn("lookup [%s] is not add, another lookup with the same name already exist", lookupName); - if (!lookupExtractorFactory.close()) { - LOGGER.error("Failed to properly close stale lookup [%s]", lookupExtractorFactory); + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + addNotice(new LoadNotice(lookupName, lookupExtractorFactoryContainer)); + } + + public void remove(String lookupName) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + addNotice(new DropNotice(lookupName)); + } + + private void addNotice(Notice notice) + { + atomicallyUpdateStateRef( + oldState -> { + if (oldState.pendingNotices.size() > 10000) { //don't let pendingNotices grow indefinitely + throw new ISE("There are too many [%d] pendingNotices.", oldState.pendingNotices.size()); } - continue; - } - if (lookupSnapshotTaker != null) { - lookupSnapshotTaker.takeSnapshot(getAllAsList()); + + ImmutableList.Builder builder = ImmutableList.builder(); + builder.addAll(oldState.pendingNotices); + builder.add(notice); + + return new LookupUpdateState( + oldState.lookupMap, builder.build(), oldState.noticesBeingHandled + + ); } + ); + LockSupport.unpark(mainThread); + } + + @Nullable + public LookupExtractorFactoryContainer get(String lookupName) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + return stateRef.get().lookupMap.get(lookupName); + } + + // Note that this should ensure that "toLoad" and "toDrop" are disjoint. + public LookupsState getAllLookupsState() + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + + LookupUpdateState lookupUpdateState = stateRef.get(); + + Map lookupsToLoad = new HashMap<>(); + Set lookupsToDrop = new HashSet<>(); + + updateToLoadAndDrop(lookupUpdateState.noticesBeingHandled, lookupsToLoad, lookupsToDrop); + updateToLoadAndDrop(lookupUpdateState.pendingNotices, lookupsToLoad, lookupsToDrop); + + return new LookupsState<>(lookupUpdateState.lookupMap, lookupsToLoad, lookupsToDrop); + } + + private void updateToLoadAndDrop(List notices, Map lookupsToLoad, Set lookupsToDrop) + { + for (Notice notice : notices) { + if (notice instanceof LoadNotice) { + LoadNotice loadNotice = (LoadNotice) notice; + lookupsToLoad.put(loadNotice.lookupName, loadNotice.lookupExtractorFactoryContainer); + lookupsToDrop.remove(loadNotice.lookupName); + } else if (notice instanceof DropNotice) { + DropNotice dropNotice = (DropNotice) notice; + lookupsToDrop.add(dropNotice.lookupName); + lookupsToLoad.remove(dropNotice.lookupName); + } else { + throw new ISE("Unknown Notice type [%s].", notice.getClass().getName()); } - if (!failedExtractorFactoryMap.isEmpty()) { - throw new ISE( - "was not able to start the following lookup(s) [%s]", - failedExtractorFactoryMap.keySet().toString() - ); - } - } - finally { - startStopLock.readLock().unlock(); } } - /** - * Add or update a lookup factory - * - * @param lookupName The name of the lookup - * @param lookupExtractorFactory The factory of the lookup - * - * @return True if the lookup was updated, false otherwise - * - * @throws IllegalStateException if start of the factory fails - */ - public boolean updateIfNew(String lookupName, final LookupExtractorFactory lookupExtractorFactory) + private void takeSnapshot(Map lookupMap) { - boolean update = false; - try { - startStopLock.readLock().lockInterruptibly(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); + if (lookupSnapshotTaker != null) { + List lookups = new ArrayList<>(lookupMap.size()); + for (Map.Entry e : lookupMap.entrySet()) { + lookups.add(new LookupBean(e.getKey(), null, e.getValue())); + } + + lookupSnapshotTaker.takeSnapshot(lookups); } - try { - assertStarted(); - LookupExtractorFactory prior = lookupMap.get(lookupName); - update = lookupExtractorFactory.replaces(prior); - if (update) { - if (!lookupExtractorFactory.start()) { - throw new ISE("Could not start [%s]", lookupName); - } - boolean racy; - do { - if (prior == null) { - racy = null != lookupMap.putIfAbsent(lookupName, lookupExtractorFactory); - } else { - racy = !lookupMap.replace(lookupName, prior, lookupExtractorFactory); - } + } - if (racy) { - prior = lookupMap.get(lookupName); - update = lookupExtractorFactory.replaces(prior); - } - } while (racy && update); + private void loadSnapshotAndInitStateRef() + { + if (lookupSnapshotTaker != null) { + ImmutableMap.Builder builder = ImmutableMap.builder(); - if (prior != null && update) { - if (!prior.close()) { - LOGGER.error("Error closing [%s]:[%s]", lookupName, prior); - } - } + final List lookupBeanList = lookupSnapshotTaker.pullExistingSnapshot(); + for (LookupBean lookupBean : lookupBeanList) { + LookupExtractorFactoryContainer container = lookupBean.getContainer(); - if (!update) { - // We started the lookup, failed a race, and now need to cleanup - if (!lookupExtractorFactory.close()) { - LOGGER.error("Error closing [%s]:[%s]", lookupExtractorFactory); - } + if (container.getLookupExtractorFactory().start()) { + builder.put(lookupBean.getName(), container); + } else { + throw new ISE("Failed to start lookup [%s]:[%s]", lookupBean.getName(), container); } } + + stateRef.set(new LookupUpdateState(builder.build(), ImmutableList.of(), ImmutableList.of())); + } else { + stateRef.set(new LookupUpdateState(ImmutableMap.of(), ImmutableList.of(), ImmutableList.of())); } - finally { - startStopLock.readLock().unlock(); - } - return update; } - /** - * @param lookupName name of {@link LookupExtractorFactory} to delete from the reference registry. - * this function does call the cleaning method {@link LookupExtractorFactory#close()} - * - * @return true only if {@code lookupName} is removed and the lookup correctly stopped - */ - public boolean remove(String lookupName) + private LookupUpdateState atomicallyUpdateStateRef(Function fn) { - try { - startStopLock.readLock().lockInterruptibly(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - try { - final LookupExtractorFactory lookupExtractorFactory = lookupMap.remove(lookupName); - if (lookupExtractorFactory != null) { - LOGGER.debug("Removed lookup [%s]", lookupName); - if (lookupSnapshotTaker != null) { - lookupSnapshotTaker.takeSnapshot(getAllAsList()); - } - return lookupExtractorFactory.close(); + while(true) { + LookupUpdateState old = stateRef.get(); + LookupUpdateState newState = fn.apply(old); + if (stateRef.compareAndSet(old, newState)) { + return newState; } } - finally { - startStopLock.readLock().unlock(); - } - return false; } - /** - * @param lookupName key to fetch the reference of the object {@link LookupExtractor} - * - * @return reference of {@link LookupExtractorFactory} that correspond the {@code lookupName} or null if absent - * - * @throws IllegalStateException if the {@link LookupReferencesManager} is closed or did not start yet - */ - @Nullable - public LookupExtractorFactory get(String lookupName) + @VisibleForTesting + interface Notice { - try { - startStopLock.readLock().lockInterruptibly(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - try { - final LookupExtractorFactory lookupExtractorFactory = lookupMap.get(lookupName); - assertStarted(); - return lookupExtractorFactory; - } - finally { - startStopLock.readLock().unlock(); - } + void handle(Map lookupMap); } - /** - * @return Returns {@link Map} containing a copy of the current state. - * - * @throws ISE if the is is closed or did not start yet. - */ - public Map getAll() + private class LoadNotice implements Notice { - try { - startStopLock.readLock().lockInterruptibly(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); + private final String lookupName; + private final LookupExtractorFactoryContainer lookupExtractorFactoryContainer; + + public LoadNotice(String lookupName, LookupExtractorFactoryContainer lookupExtractorFactoryContainer) + { + this.lookupName = lookupName; + this.lookupExtractorFactoryContainer = lookupExtractorFactoryContainer; } - try { - assertStarted(); - return Maps.newHashMap(lookupMap); + + @Override + public void handle(Map lookupMap) + { + LookupExtractorFactoryContainer old = lookupMap.get(lookupName); + if (old != null && !lookupExtractorFactoryContainer.replaces(old)) { + LOG.warn( + "got notice to load lookup [%s] that can't replace existing [%s].", + lookupExtractorFactoryContainer, + old + ); + return; + } + + if (!lookupExtractorFactoryContainer.getLookupExtractorFactory().start()) { + throw new ISE( + "start method returned false for lookup [%s]:[%s]", + lookupName, + lookupExtractorFactoryContainer + ); + } + + old = lookupMap.put(lookupName, lookupExtractorFactoryContainer); + + LOG.debug("Loaded lookup [%s] with spec [%s].", lookupName, lookupExtractorFactoryContainer); + + if (old != null) { + if (!old.getLookupExtractorFactory().close()) { + throw new ISE("close method returned false for lookup [%s]:[%s]", lookupName, old); + } + } } - finally { - startStopLock.readLock().unlock(); + + @Override + public String toString() + { + return "LoadNotice{" + + "lookupName='" + lookupName + '\'' + + ", lookupExtractorFactoryContainer=" + lookupExtractorFactoryContainer + + '}'; } } - private void assertStarted() throws ISE + private class DropNotice implements Notice { - if (isClosed()) { - throw new ISE("lookup manager is closed"); + private final String lookupName; + + public DropNotice(String lookupName) + { + this.lookupName = lookupName; } - } - public boolean isClosed() - { - return !started.get(); + @Override + public void handle(Map lookupMap) + { + final LookupExtractorFactoryContainer lookupExtractorFactoryContainer = lookupMap.remove(lookupName); + + if (lookupExtractorFactoryContainer != null) { + LOG.debug("Removed lookup [%s] with spec [%s].", lookupName, lookupExtractorFactoryContainer); + + if (!lookupExtractorFactoryContainer.getLookupExtractorFactory().close()) { + throw new ISE( + "close method returned false for lookup [%s]:[%s]", + lookupName, + lookupExtractorFactoryContainer + ); + } + } + } + + @Override + public String toString() + { + return "DropNotice{" + + "lookupName='" + lookupName + '\'' + + '}'; + } } - private List getAllAsList() + private class LookupUpdateState { - return Lists.newArrayList( - Collections2.transform( - lookupMap.entrySet(), - new Function, LookupBean>() - { - @Nullable - @Override - public LookupBean apply( - @Nullable - Map.Entry input - ) - { - final LookupBean lookupBean = new LookupBean(); - lookupBean.factory = input.getValue(); - lookupBean.name = input.getKey(); - return lookupBean; - } - } - )); + private final ImmutableMap lookupMap; + private final ImmutableList pendingNotices; + private final ImmutableList noticesBeingHandled; + + LookupUpdateState( + ImmutableMap lookupMap, + ImmutableList pendingNotices, + ImmutableList noticesBeingHandled + ) + { + this.lookupMap = lookupMap; + this.pendingNotices = pendingNotices; + this.noticesBeingHandled = noticesBeingHandled; + } } } diff --git a/processing/src/main/java/io/druid/query/lookup/LookupsState.java b/processing/src/main/java/io/druid/query/lookup/LookupsState.java new file mode 100644 index 000000000000..1646cbb92274 --- /dev/null +++ b/processing/src/main/java/io/druid/query/lookup/LookupsState.java @@ -0,0 +1,98 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.lookup; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Collections; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + */ +public class LookupsState +{ + private final Map current; + private final Map toLoad; + private final Set toDrop; + + @JsonCreator + public LookupsState( + @JsonProperty("current") Map current, + @JsonProperty("toLoad") Map toLoad, + @JsonProperty("toDrop") Set toDrop + ) + { + this.current = current == null ? Collections.EMPTY_MAP : current; + this.toLoad = toLoad == null ? Collections.EMPTY_MAP : toLoad; + this.toDrop = toDrop == null ? Collections.EMPTY_SET : toDrop; + } + + @JsonProperty + public Map getCurrent() + { + return current; + } + + @JsonProperty + public Map getToLoad() + { + return toLoad; + } + + @JsonProperty + public Set getToDrop() + { + return toDrop; + } + + @Override + public String toString() + { + return "LookupsState{" + + "current=" + current + + ", toLoad=" + toLoad + + ", toDrop=" + toDrop + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LookupsState that = (LookupsState) o; + return Objects.equals(current, that.current) && + Objects.equals(toLoad, that.toLoad) && + Objects.equals(toDrop, that.toDrop); + } + + @Override + public int hashCode() + { + return Objects.hash(current, toLoad, toDrop); + } +} diff --git a/processing/src/main/java/io/druid/query/lookup/RegisteredLookupExtractionFn.java b/processing/src/main/java/io/druid/query/lookup/RegisteredLookupExtractionFn.java index 0cdd9d5d8df3..98de89cc8de0 100644 --- a/processing/src/main/java/io/druid/query/lookup/RegisteredLookupExtractionFn.java +++ b/processing/src/main/java/io/druid/query/lookup/RegisteredLookupExtractionFn.java @@ -141,7 +141,9 @@ private LookupExtractionFn ensureDelegate() synchronized (delegateLock) { if (null == delegate) { delegate = new LookupExtractionFn( - Preconditions.checkNotNull(manager.get(getLookup()), "Lookup [%s] not found", getLookup()).get(), + Preconditions.checkNotNull(manager.get(getLookup()), "Lookup [%s] not found", getLookup()) + .getLookupExtractorFactory() + .get(), isRetainMissingValue(), getReplaceMissingValueWith(), isInjective(), diff --git a/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java b/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java new file mode 100644 index 000000000000..58b939415a45 --- /dev/null +++ b/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java @@ -0,0 +1,118 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.lookup; + +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; + +/** + */ +public class LookupExtractorFactoryContainerTest +{ + @Test + public void testSerde() throws Exception + { + String jsonStr = "{\n" + + "\"version\": \"v1\",\n" + + "\"lookupExtractorFactory\": {\n" + + " \"type\": \"test\"\n" + + " }\n" + + "}\n"; + + final ObjectMapper mapper = new DefaultObjectMapper(); + mapper.registerSubtypes(TestLookupExtractorFactory.class); + + LookupExtractorFactoryContainer actual = mapper.readValue( + mapper.writeValueAsString( + mapper.readValue(jsonStr, LookupExtractorFactoryContainer.class) + ), + LookupExtractorFactoryContainer.class + ); + + Assert.assertEquals( + new LookupExtractorFactoryContainer( + "v1", + new TestLookupExtractorFactory() + ), + actual + ); + } + + @Test + public void testReplaces() throws Exception + { + LookupExtractorFactoryContainer l0 = new LookupExtractorFactoryContainer(null, new TestLookupExtractorFactory()); + LookupExtractorFactoryContainer l1 = new LookupExtractorFactoryContainer(null, new TestLookupExtractorFactory()); + LookupExtractorFactoryContainer l2 = new LookupExtractorFactoryContainer("v0", new TestLookupExtractorFactory()); + LookupExtractorFactoryContainer l3 = new LookupExtractorFactoryContainer("v1", new TestLookupExtractorFactory()); + + Assert.assertTrue(l0.replaces(l1)); + Assert.assertFalse(l1.replaces(l2)); + Assert.assertTrue(l2.replaces(l1)); + Assert.assertFalse(l2.replaces(l3)); + Assert.assertTrue(l3.replaces(l2)); + } + + @JsonTypeName("test") + static class TestLookupExtractorFactory implements LookupExtractorFactory + { + + @Override + public boolean start() + { + return false; + } + + @Override + public boolean close() + { + return false; + } + + @Override + public boolean replaces(@Nullable LookupExtractorFactory other) + { + return true; + } + + @Nullable + @Override + public LookupIntrospectHandler getIntrospectHandler() + { + return null; + } + + @Override + public LookupExtractor get() + { + return null; + } + + public boolean equals(Object other) + { + return other instanceof TestLookupExtractorFactory; + } + } +} diff --git a/processing/src/test/java/io/druid/query/lookup/LookupsStateTest.java b/processing/src/test/java/io/druid/query/lookup/LookupsStateTest.java new file mode 100644 index 000000000000..545f4b55df2d --- /dev/null +++ b/processing/src/test/java/io/druid/query/lookup/LookupsStateTest.java @@ -0,0 +1,93 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.lookup; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +/** + */ +public class LookupsStateTest +{ + @Test + public void testSerde() throws Exception + { + String jsonStr = "{\n" + + " \"current\": {\n" + + " \"l1\": {\n" + + " \"version\": \"v1\",\n" + + " \"lookupExtractorFactory\": {\n" + + " \"type\": \"test\"\n" + + " }\n" + + " }\n" + + " },\n" + + " \"toLoad\": {\n" + + " \"l2\": {\n" + + " \"version\": \"v1\",\n" + + " \"lookupExtractorFactory\": {\n" + + " \"type\": \"test\"\n" + + " }\n" + + " }\n" + + " },\n" + + " \"toDrop\": [\"l3\"]\n" + + "}"; + + TypeReference> typeRef = + new TypeReference>() + { + }; + + final ObjectMapper mapper = new DefaultObjectMapper(); + mapper.registerSubtypes(LookupExtractorFactoryContainerTest.TestLookupExtractorFactory.class); + + LookupsState actual = mapper.readValue( + mapper.writeValueAsString( + mapper.readValue(jsonStr, typeRef) + ), + typeRef + ); + + Assert.assertEquals( + new LookupsState<>( + ImmutableMap.of( + "l1", + new LookupExtractorFactoryContainer( + "v1", + new LookupExtractorFactoryContainerTest.TestLookupExtractorFactory() + ) + ), + ImmutableMap.of( + "l2", + new LookupExtractorFactoryContainer( + "v1", + new LookupExtractorFactoryContainerTest.TestLookupExtractorFactory() + ) + ), + ImmutableSet.of("l3") + ), + actual + ); + } +} diff --git a/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java b/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java index 1f0877150d71..f3d1abd2e529 100644 --- a/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java @@ -226,38 +226,42 @@ public void testEquals() private void managerReturnsMap(LookupReferencesManager manager) { - EasyMock.expect(manager.get(EasyMock.eq(LOOKUP_NAME))).andReturn(new LookupExtractorFactory() - { - @Override - public boolean start() - { - return false; - } + EasyMock.expect(manager.get(EasyMock.eq(LOOKUP_NAME))).andReturn( + new LookupExtractorFactoryContainer( + "v0", new LookupExtractorFactory() + { + @Override + public boolean start() + { + return false; + } - @Override - public boolean close() - { - return false; - } + @Override + public boolean replaces(@Nullable LookupExtractorFactory other) + { + return false; + } - @Override - public boolean replaces(@Nullable LookupExtractorFactory other) - { - return false; - } + @Override + public boolean close() + { + return false; + } - @Nullable - @Override - public LookupIntrospectHandler getIntrospectHandler() - { - return null; - } + @Nullable + @Override + public LookupIntrospectHandler getIntrospectHandler() + { + return null; + } - @Override - public LookupExtractor get() - { - return LOOKUP_EXTRACTOR; - } - }).anyTimes(); + @Override + public LookupExtractor get() + { + return LOOKUP_EXTRACTOR; + } + } + ) + ).anyTimes(); } } diff --git a/server/src/main/java/io/druid/query/lookup/LookupIntrospectionResource.java b/server/src/main/java/io/druid/query/lookup/LookupIntrospectionResource.java index 80bc763b8c8d..9591b7e0f4e4 100644 --- a/server/src/main/java/io/druid/query/lookup/LookupIntrospectionResource.java +++ b/server/src/main/java/io/druid/query/lookup/LookupIntrospectionResource.java @@ -20,7 +20,6 @@ package io.druid.query.lookup; import com.google.inject.Inject; - import io.druid.java.util.common.logger.Logger; import javax.ws.rs.Path; @@ -44,19 +43,20 @@ public LookupIntrospectionResource(@Context LookupReferencesManager lookupRefere @Path("/{lookupId}") public Object introspectLookup(@PathParam("lookupId") final String lookupId) { - final LookupExtractorFactory lookupExtractorFactory = lookupReferencesManager.get(lookupId); - if (lookupExtractorFactory == null) { + final LookupExtractorFactoryContainer container = lookupReferencesManager.get(lookupId); + + if (container == null) { LOGGER.error("trying to introspect non existing lookup [%s]", lookupId); return Response.status(Response.Status.NOT_FOUND).build(); } - LookupIntrospectHandler introspectHandler = lookupExtractorFactory.getIntrospectHandler(); + LookupIntrospectHandler introspectHandler = container.getLookupExtractorFactory().getIntrospectHandler(); if (introspectHandler != null) { return introspectHandler; } else { LOGGER.warn( "Trying to introspect lookup [%s] of type [%s] but implementation doesn't provide resource", lookupId, - lookupExtractorFactory.get().getClass() + container.getLookupExtractorFactory().get().getClass() ); return Response.status(Response.Status.NOT_FOUND).build(); } diff --git a/server/src/main/java/io/druid/query/lookup/LookupModule.java b/server/src/main/java/io/druid/query/lookup/LookupModule.java index 09d040efcc83..267b4bf13935 100644 --- a/server/src/main/java/io/druid/query/lookup/LookupModule.java +++ b/server/src/main/java/io/druid/query/lookup/LookupModule.java @@ -33,7 +33,7 @@ import com.google.common.net.HostAndPort; import com.google.inject.Binder; import com.google.inject.Inject; - +import io.druid.common.utils.ServletResourceUtils; import io.druid.curator.announcement.Announcer; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; @@ -43,8 +43,6 @@ import io.druid.guice.annotations.Self; import io.druid.guice.annotations.Smile; import io.druid.initialization.DruidModule; -import io.druid.java.util.common.ISE; -import io.druid.java.util.common.RE; import io.druid.java.util.common.logger.Logger; import io.druid.server.DruidNode; import io.druid.server.initialization.ZkPathsConfig; @@ -58,6 +56,9 @@ import org.apache.curator.utils.ZKPaths; import javax.ws.rs.Path; +import javax.ws.rs.core.Response; +import java.io.IOException; +import java.io.InputStream; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -104,6 +105,11 @@ class LookupListeningResource extends ListenerResource { private static final Logger LOG = new Logger(LookupListeningResource.class); + private static final TypeReference> LOOKUPS_STATE_TYPE_REFERENCE = + new TypeReference>() + { + }; + @Inject public LookupListeningResource( final @Json ObjectMapper jsonMapper, @@ -118,23 +124,45 @@ public LookupListeningResource( { }) { + @Override + public Response handleUpdates( + InputStream inputStream, ObjectMapper mapper + ) + { + final LookupsState state; + try { + state = mapper.readValue(inputStream, LOOKUPS_STATE_TYPE_REFERENCE); + } + catch (final IOException ex) { + LOG.debug(ex, "Bad request"); + return Response.status(Response.Status.BAD_REQUEST).entity(ServletResourceUtils.sanitizeException(ex)).build(); + } + + try { + state.getToLoad().forEach(manager::add); + state.getToDrop().forEach(manager::remove); + + return Response.status(Response.Status.ACCEPTED).entity(manager.getAllLookupsState()).build(); + } + catch (Exception e) { + LOG.error(e, "Error handling request"); + return Response.serverError().entity(ServletResourceUtils.sanitizeException(e)).build(); + } + } + @Override public Object post(final Map lookups) throws Exception { final Map failedUpdates = new HashMap<>(); for (final String name : lookups.keySet()) { - final LookupExtractorFactory factory = lookups.get(name); - try { - // Only fail if it should have updated but didn't. - if (!manager.updateIfNew(name, factory) && factory.replaces(manager.get(name))) { - failedUpdates.put(name, factory); - } - } - catch (ISE ise) { - LOG.error(ise, "Error starting [%s]: [%s]", name, factory); - failedUpdates.put(name, factory); - } + + final LookupExtractorFactoryContainer factoryContainer = new LookupExtractorFactoryContainer( + null, + lookups.get(name) + ); + + manager.add(name, factoryContainer); } return ImmutableMap.of("status", "accepted", LookupModule.FAILED_UPDATES_KEY, failedUpdates); } @@ -146,24 +174,15 @@ public Object get(String id) } @Override - public Map getAll() + public LookupsState getAll() { - return manager.getAll(); + return manager.getAllLookupsState(); } @Override public Object delete(String id) { - if (manager.get(id) == null) { - return null; - } - if (!manager.remove(id)) { - if (manager.get(id) == null) { - return null; - } - // We don't have more information at this point. - throw new RE("Could not remove lookup [%s]", id); - } + manager.remove(id); return id; } } diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java index 50c693ad0f14..b36f0061049c 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -68,6 +68,7 @@ import io.druid.server.coordinator.rules.LoadRule; import io.druid.server.coordinator.rules.Rule; import io.druid.server.initialization.ZkPathsConfig; +import io.druid.server.lookup.cache.LookupCoordinatorManager; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.leader.LeaderLatch; @@ -133,6 +134,7 @@ public Interval apply(DataSegment segment) private volatile boolean leader = false; private volatile SegmentReplicantLookup segmentReplicantLookup = null; private final BalancerStrategyFactory factory; + private final LookupCoordinatorManager lookupCoordinatorManager; @Inject public DruidCoordinator( @@ -150,7 +152,8 @@ public DruidCoordinator( ServiceAnnouncer serviceAnnouncer, @Self DruidNode self, @CoordinatorIndexingServiceHelper Set indexingServiceHelpers, - BalancerStrategyFactory factory + BalancerStrategyFactory factory, + LookupCoordinatorManager lookupCoordinatorManager ) { this( @@ -169,7 +172,8 @@ public DruidCoordinator( self, Maps.newConcurrentMap(), indexingServiceHelpers, - factory + factory, + lookupCoordinatorManager ); } @@ -189,7 +193,8 @@ public DruidCoordinator( DruidNode self, ConcurrentMap loadQueuePeonMap, Set indexingServiceHelpers, - BalancerStrategyFactory factory + BalancerStrategyFactory factory, + LookupCoordinatorManager lookupCoordinatorManager ) { this.config = config; @@ -212,6 +217,7 @@ public DruidCoordinator( this.leaderLatch = new AtomicReference<>(null); this.loadManagementPeons = loadQueuePeonMap; this.factory = factory; + this.lookupCoordinatorManager = lookupCoordinatorManager; } public boolean isLeader() @@ -599,6 +605,8 @@ public ScheduledExecutors.Signal call() } ); } + + lookupCoordinatorManager.start(); } catch (Exception e) { log.makeAlert(e, "Unable to become leader") @@ -636,6 +644,8 @@ private void stopBeingLeader() serviceAnnouncer.unannounce(self); metadataRuleManager.stop(); metadataSegmentManager.stop(); + lookupCoordinatorManager.stop(); + leader = false; } catch (Exception e) { diff --git a/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java b/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java index 417433e22106..38d1725d3d88 100644 --- a/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java +++ b/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.base.Strings; import com.google.inject.Inject; - import io.druid.audit.AuditInfo; import io.druid.audit.AuditManager; import io.druid.common.utils.ServletResourceUtils; @@ -34,6 +33,7 @@ import io.druid.java.util.common.RE; import io.druid.java.util.common.logger.Logger; import io.druid.server.lookup.cache.LookupCoordinatorManager; +import io.druid.server.lookup.cache.LookupExtractorFactoryMapContainer; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -86,7 +86,7 @@ public Response getTiers( if (discover) { return Response.ok().entity(lookupCoordinatorManager.discoverTiers()).build(); } - final Map>> knownLookups = lookupCoordinatorManager.getKnownLookups(); + final Map> knownLookups = lookupCoordinatorManager.getKnownLookups(); if (knownLookups == null) { return Response.status(Response.Status.NOT_FOUND).build(); } else { @@ -112,9 +112,9 @@ public Response updateAllLookups( try { final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(req.getContentType()); final ObjectMapper mapper = isSmile ? smileMapper : jsonMapper; - final Map>> map; + final Map> map; try { - map = mapper.readValue(in, new TypeReference>>>() + map = mapper.readValue(in, new TypeReference>>() { }); } @@ -195,11 +195,9 @@ public Response createOrUpdateLookup( } final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(req.getContentType()); final ObjectMapper mapper = isSmile ? smileMapper : jsonMapper; - final Map lookupSpec; + final LookupExtractorFactoryMapContainer lookupSpec; try { - lookupSpec = mapper.readValue(in, new TypeReference>() - { - }); + lookupSpec = mapper.readValue(in, LookupExtractorFactoryMapContainer.class); } catch (IOException e) { return Response.status(Response.Status.BAD_REQUEST).entity(ServletResourceUtils.sanitizeException(e)).build(); @@ -240,7 +238,7 @@ public Response getSpecificLookup( .entity(ServletResourceUtils.sanitizeException(new NullPointerException("`lookup` required"))) .build(); } - final Map map = lookupCoordinatorManager.getLookup(tier, lookup); + final LookupExtractorFactoryMapContainer map = lookupCoordinatorManager.getLookup(tier, lookup); if (map == null) { return Response.status(Response.Status.NOT_FOUND) .entity(ServletResourceUtils.sanitizeException(new RE("lookup [%s] not found", lookup))) @@ -267,13 +265,13 @@ public Response getSpecificTier( .entity(ServletResourceUtils.sanitizeException(new NullPointerException("`tier` required"))) .build(); } - final Map>> map = lookupCoordinatorManager.getKnownLookups(); + final Map> map = lookupCoordinatorManager.getKnownLookups(); if (map == null) { return Response.status(Response.Status.NOT_FOUND) .entity(ServletResourceUtils.sanitizeException(new RE("No lookups found"))) .build(); } - final Map> tierLookups = map.get(tier); + final Map tierLookups = map.get(tier); if (tierLookups == null) { return Response.status(Response.Status.NOT_FOUND) .entity(ServletResourceUtils.sanitizeException(new RE("Tier [%s] not found", tier))) diff --git a/server/src/main/java/io/druid/server/listener/resource/AbstractListenerHandler.java b/server/src/main/java/io/druid/server/listener/resource/AbstractListenerHandler.java index 688968292d5f..0d31a6b49748 100644 --- a/server/src/main/java/io/druid/server/listener/resource/AbstractListenerHandler.java +++ b/server/src/main/java/io/druid/server/listener/resource/AbstractListenerHandler.java @@ -26,7 +26,6 @@ import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; - import io.druid.common.utils.ServletResourceUtils; import io.druid.java.util.common.logger.Logger; @@ -118,6 +117,8 @@ public ObjType apply(Object input) } } + + @Override public final Response handleGET(String id) { @@ -138,7 +139,7 @@ public final Response handleGET(String id) @Override public final Response handleGETAll() { - final Map all; + final Object all; try { all = getAll(); if (all == null) { @@ -183,9 +184,8 @@ public final void use_AbstractListenerHandler_instead() * * @return The object to be returned in the entity. A NULL return will cause a 404 response. A non-null return will cause a 202 response. An Exception thrown will cause a 500 response. */ - protected abstract @Nullable - Object delete(String id); + protected abstract Object delete(String id); /** * Get the object for a particular id @@ -194,13 +194,11 @@ public final void use_AbstractListenerHandler_instead() * * @return The object to be returned in the entity. A NULL return will cause a 404 response. A non-null return will cause a 200 response. An Exception thrown will cause a 500 response. */ - protected abstract @Nullable - Object get(String id); + protected abstract Object get(String id); - protected abstract @Nullable - Map getAll(); + protected abstract Object getAll(); /** * Process a POST request of the input items @@ -211,7 +209,6 @@ public final void use_AbstractListenerHandler_instead() * * @throws Exception */ - public abstract @Nullable - Object post(Map inputObject) throws Exception; + public abstract Object post(Map inputObject) throws Exception; } diff --git a/server/src/main/java/io/druid/server/listener/resource/ListenerHandler.java b/server/src/main/java/io/druid/server/listener/resource/ListenerHandler.java index 7c7c46a7bf05..39ba93c9b278 100644 --- a/server/src/main/java/io/druid/server/listener/resource/ListenerHandler.java +++ b/server/src/main/java/io/druid/server/listener/resource/ListenerHandler.java @@ -35,5 +35,9 @@ public interface ListenerHandler Response handleGET(String id); Response handleGETAll(); Response handleDELETE(String id); + Response handleUpdates(InputStream inputStream, ObjectMapper mapper); + void use_AbstractListenerHandler_instead(); + + } diff --git a/server/src/main/java/io/druid/server/listener/resource/ListenerResource.java b/server/src/main/java/io/druid/server/listener/resource/ListenerResource.java index 1ca2cd31c43a..b544166a3190 100644 --- a/server/src/main/java/io/druid/server/listener/resource/ListenerResource.java +++ b/server/src/main/java/io/druid/server/listener/resource/ListenerResource.java @@ -98,6 +98,27 @@ public Response serviceAnnouncementPOSTAll( } } + @POST + @Path("/updates") + @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + public Response serviceAnnouncementHandleUpdates( + final InputStream inputStream, + final @Context HttpServletRequest req // used only to get request content-type + ) + { + final String reqContentType = req.getContentType(); + final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(reqContentType); + final ObjectMapper mapper = isSmile ? smileMapper : jsonMapper; + try { + return handler.handleUpdates(inputStream, mapper); + } + catch (Exception e) { + LOG.error(e, "Exception in handling updates request"); + return Response.serverError().entity(ServletResourceUtils.sanitizeException(e)).build(); + } + } + @GET @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) public Response getAll() diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java index 9dce1b85776b..7b9409d2449c 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java @@ -19,16 +19,12 @@ package io.druid.server.lookup.cache; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Predicates; +import com.google.common.base.Preconditions; import com.google.common.base.Throwables; -import com.google.common.collect.Collections2; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import com.google.common.net.HostAndPort; @@ -48,15 +44,16 @@ import io.druid.audit.AuditInfo; import io.druid.common.config.JacksonConfigManager; import io.druid.concurrent.Execs; +import io.druid.concurrent.LifecycleLock; import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.IOE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StreamUtils; import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.lifecycle.LifecycleStart; -import io.druid.java.util.common.lifecycle.LifecycleStop; import io.druid.query.lookup.LookupModule; +import io.druid.query.lookup.LookupsState; import io.druid.server.listener.announcer.ListenerDiscoverer; import io.druid.server.listener.resource.ListenerResource; import org.jboss.netty.handler.codec.http.HttpHeaders; @@ -67,18 +64,22 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; +import java.net.HttpURLConnection; import java.net.MalformedURLException; import java.net.URL; +import java.util.AbstractMap; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -87,47 +88,45 @@ */ public class LookupCoordinatorManager { - public static final String LOOKUP_CONFIG_KEY = "lookups"; - // Doesn't have to be the same, but it makes things easy to look at - public static final String LOOKUP_LISTEN_ANNOUNCE_KEY = LOOKUP_CONFIG_KEY; + //key used in druid-0.10.0 with config manager + public static final String OLD_LOOKUP_CONFIG_KEY = "lookups"; + + public static final String LOOKUP_CONFIG_KEY = "lookupsConfig"; + public static final String LOOKUP_LISTEN_ANNOUNCE_KEY = "lookups"; + + private static final String LOOKUP_BASE_REQUEST_PATH = ListenerResource.BASE_PATH + + "/" + + LOOKUP_LISTEN_ANNOUNCE_KEY; + private static final String LOOKUP_UPDATE_REQUEST_PATH = LOOKUP_BASE_REQUEST_PATH + "/" + "updates"; + + private static final TypeReference> LOOKUPS_STATE_TYPE_REFERENCE = + new TypeReference>() + { + }; + private static final EmittingLogger LOG = new EmittingLogger(LookupCoordinatorManager.class); - private static final TypeReference> MAP_STRING_OBJ_TYPE = new TypeReference>() - { - }; - private final static Function HOST_TO_URL = new Function() - { - @Nullable - @Override - public URL apply(HostAndPort input) - { - if (input == null) { - LOG.warn("null entry in lookups"); - return null; - } - try { - return getLookupsURL(input); - } - catch (MalformedURLException e) { - LOG.warn(e, "Skipping node. Malformed URL from `%s`", input); - return null; - } - } - }; - private final ListeningScheduledExecutorService executorService; private final ListenerDiscoverer listenerDiscoverer; - private final HttpClient httpClient; - private final ObjectMapper smileMapper; private final JacksonConfigManager configManager; private final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig; - private final Object startStopSync = new Object(); + private final LookupsCommunicator lookupsCommunicator; + + // Known lookup state across various cluster nodes is managed in the reference here. On each lookup management loop + // state is rediscovered and updated in the ref here. If some lookup nodes have disappeared since last lookup + // management loop, then they get discarded automatically. + @VisibleForTesting + final AtomicReference>> knownOldState = + new AtomicReference<>(); + // Updated by config watching service - private AtomicReference>>> lookupMapConfigRef; - private volatile Map>> prior_update = ImmutableMap.of(); - private volatile boolean started = false; - private volatile ListenableScheduledFuture backgroundManagerFuture = null; - private final CountDownLatch backgroundManagerExitedLatch = new CountDownLatch(1); + private AtomicReference>> lookupMapConfigRef; + + @VisibleForTesting + final LifecycleLock lifecycleLock = new LifecycleLock(); + private ListeningScheduledExecutorService executorService; + private ListenableScheduledFuture backgroundManagerFuture; + private CountDownLatch backgroundManagerExitedLatch; @Inject public LookupCoordinatorManager( @@ -138,296 +137,67 @@ public LookupCoordinatorManager( final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig ) { - this.listenerDiscoverer = listenerDiscoverer; - this.configManager = configManager; - this.httpClient = httpClient; - this.smileMapper = smileMapper; - this.lookupCoordinatorManagerConfig = lookupCoordinatorManagerConfig; - executorService = MoreExecutors.listeningDecorator( - Executors.newScheduledThreadPool( - lookupCoordinatorManagerConfig.getThreadPoolSize(), - Execs.makeThreadFactory("LookupCoordinatorManager--%s") - ) + this( + listenerDiscoverer, + configManager, + lookupCoordinatorManagerConfig, + new LookupsCommunicator(httpClient, lookupCoordinatorManagerConfig, smileMapper) ); } - void deleteOnHost(final URL url) - throws ExecutionException, InterruptedException, IOException - { - final AtomicInteger returnCode = new AtomicInteger(0); - final AtomicReference reasonString = new AtomicReference<>(null); - LOG.debug("Dropping %s", url); - - try (final InputStream result = httpClient.go( - new Request(HttpMethod.DELETE, url) - .addHeader(HttpHeaders.Names.ACCEPT, SmileMediaTypes.APPLICATION_JACKSON_SMILE), - makeResponseHandler(returnCode, reasonString), - lookupCoordinatorManagerConfig.getHostDeleteTimeout() - ).get()) { - // 404 is ok here, that means it was already deleted - if (!httpStatusIsSuccess(returnCode.get()) && !httpStatusIsNotFound(returnCode.get())) { - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - try { - StreamUtils.copyAndClose(result, baos); - } - catch (IOException e2) { - LOG.warn(e2, "Error reading response from [%s]", url); - } - - throw new IOException( - String.format( - "Bad lookup delete request to [%s] : [%d] : [%s] Response: [%s]", - url, - returnCode.get(), - reasonString.get(), - StringUtils.fromUtf8(baos.toByteArray()) - ) - ); - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Delete to [%s] : Status: %s reason: [%s]", url, returnCode.get(), reasonString.get()); - } - } - } - } - - void updateAllOnHost(final URL url, Map> knownLookups) - throws IOException, InterruptedException, ExecutionException - { - final AtomicInteger returnCode = new AtomicInteger(0); - final AtomicReference reasonString = new AtomicReference<>(null); - final byte[] bytes; - try { - if (LOG.isDebugEnabled()) { - LOG.debug("Loading up %d lookups to %s", knownLookups.size(), url); - } - bytes = smileMapper.writeValueAsBytes(knownLookups); - } - catch (JsonProcessingException e) { - throw Throwables.propagate(e); - } - - try (final InputStream result = httpClient.go( - new Request(HttpMethod.POST, url) - .addHeader(HttpHeaders.Names.ACCEPT, SmileMediaTypes.APPLICATION_JACKSON_SMILE) - .addHeader(HttpHeaders.Names.CONTENT_TYPE, SmileMediaTypes.APPLICATION_JACKSON_SMILE) - .setContent(bytes), - makeResponseHandler(returnCode, reasonString), - lookupCoordinatorManagerConfig.getHostUpdateTimeout() - ).get()) { - if (!httpStatusIsSuccess(returnCode.get())) { - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - try { - StreamUtils.copyAndClose(result, baos); - } - catch (IOException e2) { - LOG.warn(e2, "Error reading response"); - } - - throw new IOException( - String.format( - "Bad update request to [%s] : [%d] : [%s] Response: [%s]", - url, - returnCode.get(), - reasonString.get(), - StringUtils.fromUtf8(baos.toByteArray()) - ) - ); - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Update on [%s], Status: %s reason: [%s]", url, returnCode.get(), reasonString.get()); - } - final Map resultMap = smileMapper.readValue(result, MAP_STRING_OBJ_TYPE); - final Object missingValuesObject = resultMap.get(LookupModule.FAILED_UPDATES_KEY); - if (null == missingValuesObject) { - throw new IAE("Update result did not have field for [%s]", LookupModule.FAILED_UPDATES_KEY); - } - - final Map missingValues = smileMapper.convertValue(missingValuesObject, MAP_STRING_OBJ_TYPE); - if (!missingValues.isEmpty()) { - throw new IAE("Lookups failed to update: %s", smileMapper.writeValueAsString(missingValues.keySet())); - } else { - LOG.debug("Updated all lookups on [%s]", url); - } - } - } - } - - // Overridden in unit tests - HttpResponseHandler makeResponseHandler( - final AtomicInteger returnCode, - final AtomicReference reasonString + @VisibleForTesting + LookupCoordinatorManager( + final ListenerDiscoverer listenerDiscoverer, + final JacksonConfigManager configManager, + final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig, + final LookupsCommunicator lookupsCommunicator ) { - return new SequenceInputStreamResponseHandler() - { - @Override - public ClientResponse handleResponse(HttpResponse response) - { - returnCode.set(response.getStatus().getCode()); - reasonString.set(response.getStatus().getReasonPhrase()); - return super.handleResponse(response); - } - }; - } - - void deleteAllOnTier(final String tier, final Collection dropLookups) - throws ExecutionException, InterruptedException, IOException - { - if (dropLookups.isEmpty()) { - LOG.debug("Nothing to drop"); - return; - } - final Collection urls = getAllHostsAnnounceEndpoint(tier); - final List> futures = new ArrayList<>(urls.size()); - for (final URL url : urls) { - futures.add(executorService.submit(new Runnable() - { - @Override - public void run() - { - for (final String drop : dropLookups) { - final URL lookupURL; - try { - lookupURL = new URL( - url.getProtocol(), - url.getHost(), - url.getPort(), - String.format("%s/%s", url.getFile(), drop) - ); - } - catch (MalformedURLException e) { - throw new ISE(e, "Error creating url for [%s]/[%s]", url, drop); - } - try { - deleteOnHost(lookupURL); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOG.warn("Delete [%s] interrupted", lookupURL); - throw Throwables.propagate(e); - } - catch (IOException | ExecutionException e) { - // Don't raise as ExecutionException. Just log and continue - LOG.makeAlert(e, "Error deleting [%s]", lookupURL).emit(); - } - } - } - })); - } - final ListenableFuture allFuture = Futures.allAsList(futures); - try { - allFuture.get(lookupCoordinatorManagerConfig.getUpdateAllTimeout().getMillis(), TimeUnit.MILLISECONDS); - } - catch (TimeoutException e) { - // This should cause Interrupted exceptions on the offending ones - allFuture.cancel(true); - throw new ExecutionException("Timeout in updating hosts! Attempting to cancel", e); - } - } - - void updateAllNewOnTier(final String tier, final Map> knownLookups) - throws InterruptedException, ExecutionException, IOException - { - final Collection urls = Collections2.transform( - listenerDiscoverer.getNewNodes(LookupModule.getTierListenerPath(tier)), - HOST_TO_URL - ); - if (urls.isEmpty() || knownLookups.isEmpty()) { - LOG.debug("Nothing new to report"); - return; - } - updateNodes(urls, knownLookups); - } - - void updateAllOnTier(final String tier, final Map> knownLookups) - throws InterruptedException, ExecutionException, IOException - { - updateNodes(getAllHostsAnnounceEndpoint(tier), knownLookups); - } - - void updateNodes(Collection urls, final Map> knownLookups) - throws IOException, InterruptedException, ExecutionException - { - if (knownLookups == null) { - LOG.debug("No config for lookups found"); - return; - } - if (knownLookups.isEmpty()) { - LOG.debug("No known lookups. Skipping update"); - return; - } - if (LOG.isDebugEnabled()) { - LOG.debug("Updating %d lookups on %d nodes", knownLookups.size(), urls.size()); - } - final List> futures = new ArrayList<>(urls.size()); - for (final URL url : urls) { - futures.add(executorService.submit(new Runnable() - { - @Override - public void run() - { - try { - updateAllOnHost(url, knownLookups); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOG.warn("Update on [%s] interrupted", url); - throw Throwables.propagate(e); - } - catch (IOException | ExecutionException e) { - // Don't raise as ExecutionException. Just log and continue - LOG.makeAlert(e, "Error submitting to [%s]", url).emit(); - } - } - })); - } - final ListenableFuture allFuture = Futures.allAsList(futures); - try { - allFuture.get(lookupCoordinatorManagerConfig.getUpdateAllTimeout().getMillis(), TimeUnit.MILLISECONDS); - } - catch (TimeoutException e) { - LOG.warn("Timeout in updating hosts! Attempting to cancel"); - // This should cause Interrupted exceptions on the offending ones - allFuture.cancel(true); - } - } - - Collection getAllHostsAnnounceEndpoint(final String tier) throws IOException - { - return ImmutableList.copyOf( - Collections2.filter( - Collections2.transform( - listenerDiscoverer.getNodes(LookupModule.getTierListenerPath(tier)), - HOST_TO_URL - ), - Predicates.notNull() - ) - ); + this.listenerDiscoverer = listenerDiscoverer; + this.configManager = configManager; + this.lookupCoordinatorManagerConfig = lookupCoordinatorManagerConfig; + this.lookupsCommunicator = lookupsCommunicator; } public boolean updateLookup( final String tier, final String lookupName, - Map spec, + LookupExtractorFactoryMapContainer spec, final AuditInfo auditInfo ) { return updateLookups( - ImmutableMap.>>of(tier, ImmutableMap.of(lookupName, spec)), + ImmutableMap.>of(tier, ImmutableMap.of(lookupName, spec)), auditInfo ); } - public boolean updateLookups(final Map>> updateSpec, AuditInfo auditInfo) + public boolean updateLookups(final Map> updateSpec, AuditInfo auditInfo) { - synchronized (startStopSync) { - final Map>> priorSpec = getKnownLookups(); + Preconditions.checkState(lifecycleLock.awaitStarted(5, TimeUnit.SECONDS), "not started"); + + if (updateSpec.isEmpty() && lookupMapConfigRef.get() != null) { + return true; + } + + //ensure all the lookups specs have version specified. ideally this should be done in the LookupExtractorFactoryMapContainer + //constructor but that allows null to enable backward compatibility with 0.10.0 lookup specs + for (final Map.Entry> tierEntry : updateSpec.entrySet()) { + for (Map.Entry e : tierEntry.getValue().entrySet()) { + Preconditions.checkNotNull( + e.getValue().getVersion(), + "lookup [%s]:[%s] does not have version.", tierEntry.getKey(), e.getKey() + ); + } + } + + synchronized(this) { + final Map> priorSpec = getKnownLookups(); if (priorSpec == null && !updateSpec.isEmpty()) { // To prevent accidentally erasing configs if we haven't updated our cache of the values throw new ISE("Not initialized. If this is the first lookup, post an empty map to initialize"); } - final Map>> updatedSpec; + final Map> updatedSpec; // Only add or update here, don't delete. if (priorSpec == null) { @@ -436,15 +206,28 @@ public boolean updateLookups(final Map>> } else { // Needs update updatedSpec = new HashMap<>(priorSpec); - for (final String tier : updateSpec.keySet()) { - final Map> priorTierSpec = priorSpec.get(tier); - final Map> updateTierSpec = updateSpec.get(tier); + for (final Map.Entry> tierEntry : updateSpec.entrySet()) { + final String tier = tierEntry.getKey(); + final Map updateTierSpec = tierEntry.getValue(); + final Map priorTierSpec = priorSpec.get(tier); + if (priorTierSpec == null) { // New tier updatedSpec.put(tier, updateTierSpec); } else { // Update existing tier - final Map> updatedTierSpec = new HashMap<>(priorTierSpec); + final Map updatedTierSpec = new HashMap<>(priorTierSpec); + + for (Map.Entry e : updateTierSpec.entrySet()) { + if (updatedTierSpec.containsKey(e.getKey()) && !e.getValue().replaces(updatedTierSpec.get(e.getKey()))) { + throw new IAE( + "given update for lookup [%s]:[%s] can't replace existing spec [%s].", + tier, + e.getKey(), + updatedTierSpec.get(e.getKey()) + ); + } + } updatedTierSpec.putAll(updateTierSpec); updatedSpec.put(tier, updatedTierSpec); } @@ -454,24 +237,24 @@ public boolean updateLookups(final Map>> } } - public Map>> getKnownLookups() + public Map> getKnownLookups() { - if (!started) { - throw new ISE("Not started"); - } + Preconditions.checkState(lifecycleLock.awaitStarted(5, TimeUnit.SECONDS), "not started"); return lookupMapConfigRef.get(); } public boolean deleteLookup(final String tier, final String lookup, AuditInfo auditInfo) { - synchronized (startStopSync) { - final Map>> priorSpec = getKnownLookups(); + Preconditions.checkState(lifecycleLock.awaitStarted(5, TimeUnit.SECONDS), "not started"); + + synchronized(this) { + final Map> priorSpec = getKnownLookups(); if (priorSpec == null) { LOG.warn("Requested delete lookup [%s]/[%s]. But no lookups exist!", tier, lookup); return false; } - final Map>> updateSpec = new HashMap<>(priorSpec); - final Map> priorTierSpec = updateSpec.get(tier); + final Map> updateSpec = new HashMap<>(priorSpec); + final Map priorTierSpec = updateSpec.get(tier); if (priorTierSpec == null) { LOG.warn("Requested delete of lookup [%s]/[%s] but tier does not exist!", tier, lookup); return false; @@ -482,7 +265,7 @@ public boolean deleteLookup(final String tier, final String lookup, AuditInfo au return false; } - final Map> updateTierSpec = new HashMap<>(priorTierSpec); + final Map updateTierSpec = new HashMap<>(priorTierSpec); updateTierSpec.remove(lookup); updateSpec.put(tier, updateTierSpec); return configManager.set(LOOKUP_CONFIG_KEY, updateSpec, auditInfo); @@ -506,16 +289,15 @@ public Collection discoverTiers() * * @return The lookupName spec if found or null if not found or if no lookups at all are found */ - public @Nullable - Map getLookup(final String tier, final String lookupName) + public LookupExtractorFactoryMapContainer getLookup(final String tier, final String lookupName) { - final Map>> prior = getKnownLookups(); + final Map> prior = getKnownLookups(); if (prior == null) { LOG.warn("Requested tier [%s] lookupName [%s]. But no lookups exist!", tier, lookupName); return null; } - final Map> tierLookups = prior.get(tier); + final Map tierLookups = prior.get(tier); if (tierLookups == null) { LOG.warn("Tier [%s] does not exist", tier); return null; @@ -523,121 +305,339 @@ Map getLookup(final String tier, final String lookupName) return tierLookups.get(lookupName); } - - @LifecycleStart + // start() and stop() are synchronized so that they never run in parallel in case of ZK acting funny or druid bug and + // coordinator becomes leader and drops leadership in quick succession. public void start() { - synchronized (startStopSync) { - if (started) { - return; + synchronized(lifecycleLock) { + if (!lifecycleLock.canStart()) { + throw new ISE("LookupCoordinatorManager can't start."); + } + + try { + LOG.debug("Starting."); + + //first ensure that previous executorService from last cycle of start/stop has finished completely. + //so that we don't have multiple live executorService instances lying around doing lookup management. + if (executorService != null && + !executorService.awaitTermination( + lookupCoordinatorManagerConfig.getHostTimeout().getMillis() * 10, + TimeUnit.MILLISECONDS + )) { + throw new ISE("WTF! LookupCoordinatorManager executor from last start() hasn't finished. Failed to Start."); + } + + executorService = MoreExecutors.listeningDecorator( + Executors.newScheduledThreadPool( + lookupCoordinatorManagerConfig.getThreadPoolSize(), + Execs.makeThreadFactory("LookupCoordinatorManager--%s") + ) + ); + + initializeLookupsConfigWatcher(); + + this.backgroundManagerExitedLatch = new CountDownLatch(1); + this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( + this::lookupManagementLoop, + lookupCoordinatorManagerConfig.getInitialDelay(), + lookupCoordinatorManagerConfig.getPeriod(), + TimeUnit.MILLISECONDS + ); + Futures.addCallback( + backgroundManagerFuture, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Object result) + { + backgroundManagerExitedLatch.countDown(); + LOG.debug("Exited background lookup manager"); + } + + @Override + public void onFailure(Throwable t) + { + backgroundManagerExitedLatch.countDown(); + if (backgroundManagerFuture.isCancelled()) { + LOG.debug("Exited background lookup manager due to cancellation."); + } else { + LOG.makeAlert(t, "Background lookup manager exited with error!").emit(); + } + } + } + ); + + LOG.debug("Started"); + } + catch (Exception ex) { + LOG.makeAlert(ex, "Got Exception while start()").emit(); + } + finally { + //so that subsequent stop() would happen, even if start() failed with exception + lifecycleLock.started(); + lifecycleLock.exitStart(); + } + } + } + + public void stop() + { + synchronized (lifecycleLock) { + if (!lifecycleLock.canStop()) { + throw new ISE("LookupCoordinatorManager can't stop."); + } + + try { + LOG.debug("Stopping"); + + if (backgroundManagerFuture != null && !backgroundManagerFuture.cancel(true)) { + LOG.warn("Background lookup manager thread could not be cancelled"); + } + + // signal the executorService to shut down ASAP, if this coordinator becomes leader again + // then start() would ensure that this executorService is finished before starting a + // new one. + if (executorService != null) { + executorService.shutdownNow(); + } + + LOG.debug("Stopped"); } - if (executorService.isShutdown()) { - throw new ISE("Cannot restart after stop!"); + catch (Exception ex) { + LOG.makeAlert(ex, "Got Exception while stop()").emit(); } - lookupMapConfigRef = configManager.watch( - LOOKUP_CONFIG_KEY, + finally { + //so that subsequent start() would happen, even if stop() failed with exception + lifecycleLock.exitStop(); + lifecycleLock.reset(); + } + } + } + + private void initializeLookupsConfigWatcher() + { + //Note: this call is idempotent, so multiple start() would not cause any problems. + lookupMapConfigRef = configManager.watch( + LOOKUP_CONFIG_KEY, + new TypeReference>>() + { + }, + null + ); + + // backward compatibility with 0.10.0 + if (lookupMapConfigRef.get() == null) { + Map>> oldLookups = configManager.watch( + OLD_LOOKUP_CONFIG_KEY, new TypeReference>>>() { }, null - ); - final ListenableScheduledFuture backgroundManagerFuture = this.backgroundManagerFuture = executorService.scheduleWithFixedDelay( - new Runnable() - { - @Override - public void run() - { - final Map>> allLookupTiers = lookupMapConfigRef.get(); - // Sanity check for if we are shutting down - if (Thread.currentThread().isInterrupted()) { - LOG.info("Not updating lookups because process was interrupted"); - return; - } - if (!started) { - LOG.info("Not started. Returning"); - return; + ).get(); + + if (oldLookups != null) { + Map> converted = new HashMap<>(); + oldLookups.forEach( + (tier, oldTierLookups) -> { + if (oldTierLookups != null && !oldTierLookups.isEmpty()) { + converted.put(tier, convertTierLookups(oldTierLookups)); } - if (allLookupTiers == null) { - LOG.info("Not updating lookups because no data exists"); - return; - } - for (final String tier : allLookupTiers.keySet()) { - try { - final Map> allLookups = allLookupTiers.get(tier); - final Map> oldLookups = prior_update.get(tier); - final Collection drops; - if (oldLookups == null) { - drops = ImmutableList.of(); - } else { - drops = Sets.difference(oldLookups.keySet(), allLookups.keySet()); - } - if (allLookupTiers == prior_update) { - LOG.debug("No updates"); - updateAllNewOnTier(tier, allLookups); - } else { - updateAllOnTier(tier, allLookups); - deleteAllOnTier(tier, drops); - } - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw Throwables.propagate(e); - } - catch (Exception e) { - LOG.error(e, "Error updating lookups for tier [%s]. Will try again soon", tier); - } - } - prior_update = allLookupTiers; } - }, - 0, - lookupCoordinatorManagerConfig.getPeriod(), - TimeUnit.MILLISECONDS - ); - Futures.addCallback(backgroundManagerFuture, new FutureCallback() - { - @Override - public void onSuccess(@Nullable Object result) - { - backgroundManagerExitedLatch.countDown(); - LOG.debug("Exited background lookup manager"); + ); + + configManager.set( + LOOKUP_CONFIG_KEY, + converted, + new AuditInfo("autoConversion", "autoConversion", "127.0.0.1") + ); + } + } + } + + private Map convertTierLookups( + Map> oldTierLookups + ) + { + Map convertedTierLookups = new HashMap<>(); + oldTierLookups.forEach( + (lookup, lookupExtractorFactory) -> { + convertedTierLookups.put(lookup, new LookupExtractorFactoryMapContainer(null, lookupExtractorFactory)); } + ); + return convertedTierLookups; + } - @Override - public void onFailure(Throwable t) - { - backgroundManagerExitedLatch.countDown(); - if (backgroundManagerFuture.isCancelled()) { - LOG.info("Background lookup manager exited"); - LOG.trace(t, "Background lookup manager exited with throwable"); - } else { - LOG.makeAlert(t, "Background lookup manager exited with error!").emit(); - } + @VisibleForTesting + void lookupManagementLoop() + { + // Sanity check for if we are shutting down + if (Thread.currentThread().isInterrupted() || !lifecycleLock.awaitStarted(15, TimeUnit.SECONDS)) { + LOG.info("Not updating lookups because process was interrupted or not finished starting yet."); + return; + } + + final Map> allLookupTiers = lookupMapConfigRef.get(); + + if (allLookupTiers == null) { + LOG.info("Not updating lookups because no data exists"); + return; + } + + LOG.debug("Starting lookup sync for on all nodes."); + + try { + List> futures = new ArrayList<>(); + for (Map.Entry> tierEntry : allLookupTiers.entrySet()) { + + LOG.debug("Starting lookup mgmt for tier [%s].", tierEntry.getKey()); + + final Map tierLookups = tierEntry.getValue(); + for (final HostAndPort node : listenerDiscoverer.getNodes(LookupModule.getTierListenerPath(tierEntry.getKey()))) { + + LOG.debug( + "Starting lookup mgmt for tier [%s] and host [%s:%s].", + tierEntry.getKey(), + node.getHostText(), + node.getPort() + ); + + futures.add( + executorService.submit( + () -> { + try { + return new AbstractMap.SimpleImmutableEntry<>(node, doLookupManagementOnNode(node, tierLookups)); + } + catch (InterruptedException ex) { + LOG.warn(ex, "lookup management on node [%s:%s] interrupted.", node.getHostText(), node.getPort()); + return null; + } + catch (Exception ex) { + LOG.makeAlert( + ex, + "Failed to finish lookup management on node [%s:%s]", + node.getHostText(), + node.getPort() + ).emit(); + return null; + } + } + ) + ); } - }); - started = true; - LOG.debug("Started"); + } + + final ListenableFuture> allFuture = Futures.allAsList(futures); + try { + ImmutableMap.Builder> stateBuilder = ImmutableMap.builder(); + allFuture.get(lookupCoordinatorManagerConfig.getAllHostTimeout().getMillis(), TimeUnit.MILLISECONDS) + .stream() + .filter(Objects::nonNull) + .forEach(stateBuilder::put) + ; + knownOldState.set(stateBuilder.build()); + } + catch (InterruptedException ex) { + allFuture.cancel(true); + Thread.currentThread().interrupt(); + throw ex; + } catch (Exception ex) { + allFuture.cancel(true); + throw ex; + } + + } catch (Exception ex) { + LOG.makeAlert(ex, "Failed to finish lookup management loop.").emit(); } + + LOG.debug("Finished lookup sync for on all nodes."); } - @LifecycleStop - public void stop() + private LookupsState doLookupManagementOnNode( + HostAndPort node, + Map nodeTierLookupsToBe + ) throws IOException, InterruptedException, ExecutionException { - synchronized (startStopSync) { - if (!started) { - LOG.warn("Not started, ignoring stop request"); - return; + LOG.debug("Starting lookup sync for node [%s].", node); + + LookupsState currLookupsStateOnNode = lookupsCommunicator.getLookupStateForNode( + node + ); + LOG.debug("Received lookups state from node [%s].", node); + + + // Compare currLookupsStateOnNode with nodeTierLookupsToBe to find what are the lookups + // we need to further ask node to load/drop + Map toLoad = getToBeLoadedOnNode( + currLookupsStateOnNode, + nodeTierLookupsToBe + ); + Set toDrop = getToBeDroppedFromNode(currLookupsStateOnNode, nodeTierLookupsToBe); + + if (!toLoad.isEmpty() || !toDrop.isEmpty()) { + // Send POST request to the node asking to load and drop the lookups necessary. + // no need to send "current" in the LookupsStateWithMap , that is not required + currLookupsStateOnNode = lookupsCommunicator.updateNode(node, new LookupsState<>(null, toLoad, toDrop)); + + LOG.debug( + "Sent lookup toAdd[%s] and toDrop[%s] updates to node [%s].", + toLoad.keySet(), + toDrop, + node + ); + } + + LOG.debug("Finished lookup sync for node [%s].", node); + return currLookupsStateOnNode; + } + + // Returns the Map that needs to be loaded by the node and it does not know about + // those already. + // It is assumed that currLookupsStateOnNode "toLoad" and "toDrop" are disjoint. + @VisibleForTesting + Map getToBeLoadedOnNode( + LookupsState currLookupsStateOnNode, + Map nodeTierLookupsToBe + ) + { + Map toLoad = new HashMap<>(); + for (Map.Entry e : nodeTierLookupsToBe.entrySet()) { + String name = e.getKey(); + LookupExtractorFactoryMapContainer lookupToBe = e.getValue(); + + // get it from the current pending notices list on the node + LookupExtractorFactoryMapContainer current = currLookupsStateOnNode.getToLoad().get(name); + + if (current == null) { + //ok, not on pending list, get from currently loaded lookups on node + current = currLookupsStateOnNode.getCurrent().get(name); } - started = false; - executorService.shutdownNow(); - final ListenableScheduledFuture backgroundManagerFuture = this.backgroundManagerFuture; - this.backgroundManagerFuture = null; - if (backgroundManagerFuture != null && !backgroundManagerFuture.cancel(true)) { - LOG.warn("Background lookup manager thread could not be cancelled"); + + if (current == null || //lookup is neither pending nor already loaded on the node OR + currLookupsStateOnNode.getToDrop().contains(name) || //it is being dropped on the node OR + lookupToBe.replaces(current) //lookup is already know to node, but lookupToBe overrides that + ) { + toLoad.put(name, lookupToBe); } - // NOTE: we can't un-watch the configuration key - LOG.debug("Stopped"); } + return toLoad; + } + + // Returns Set that should be dropped from the node which has them already either in pending to load + // state or loaded + // It is assumed that currLookupsStateOnNode "toLoad" and "toDrop" are disjoint. + @VisibleForTesting + Set getToBeDroppedFromNode( + LookupsState currLookupsStateOnNode, + Map nodeTierLookupsToBe + ) + { + Set toDrop = new HashSet<>(); + + // {currently loading/loaded on the node} - {currently pending deletion on node} - {lookups node should actually have} + toDrop.addAll(currLookupsStateOnNode.getCurrent().keySet()); + toDrop.addAll(currLookupsStateOnNode.getToLoad().keySet()); + toDrop = Sets.difference(toDrop, currLookupsStateOnNode.getToDrop()); + toDrop = Sets.difference(toDrop, nodeTierLookupsToBe.keySet()); + return toDrop; } static URL getLookupsURL(HostAndPort druidNode) throws MalformedURLException @@ -646,18 +646,23 @@ static URL getLookupsURL(HostAndPort druidNode) throws MalformedURLException "http", druidNode.getHostText(), druidNode.getPortOrDefault(-1), - ListenerResource.BASE_PATH + "/" + LOOKUP_LISTEN_ANNOUNCE_KEY + LOOKUP_BASE_REQUEST_PATH ); } - private static boolean httpStatusIsSuccess(int statusCode) + static URL getLookupsUpdateURL(HostAndPort druidNode) throws MalformedURLException { - return statusCode >= 200 && statusCode < 300; + return new URL( + "http", + druidNode.getHostText(), + druidNode.getPortOrDefault(-1), + LOOKUP_UPDATE_REQUEST_PATH + ); } - private static boolean httpStatusIsNotFound(int statusCode) + private static boolean httpStatusIsSuccess(int statusCode) { - return statusCode == 404; + return statusCode >= 200 && statusCode < 300; } @VisibleForTesting @@ -672,4 +677,153 @@ boolean waitForBackgroundTermination(long timeout) throws InterruptedException { return backgroundManagerExitedLatch.await(timeout, TimeUnit.MILLISECONDS); } + + @VisibleForTesting + public static class LookupsCommunicator + { + private final HttpClient httpClient; + private final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig; + private final ObjectMapper smileMapper; + + public LookupsCommunicator( + HttpClient httpClient, + LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig, + ObjectMapper smileMapper + ) + { + this.httpClient = httpClient; + this.lookupCoordinatorManagerConfig = lookupCoordinatorManagerConfig; + this.smileMapper = smileMapper; + } + + public LookupsState updateNode( + HostAndPort node, + LookupsState lookupsUpdate + ) + throws IOException, InterruptedException, ExecutionException + { + final AtomicInteger returnCode = new AtomicInteger(0); + final AtomicReference reasonString = new AtomicReference<>(null); + + final URL url = getLookupsUpdateURL(node); + + LOG.debug("Sending lookups load/drop request to [%s]. Request [%s]", url, lookupsUpdate); + + try (final InputStream result = httpClient.go( + new Request(HttpMethod.POST, url) + .addHeader(HttpHeaders.Names.ACCEPT, SmileMediaTypes.APPLICATION_JACKSON_SMILE) + .addHeader(HttpHeaders.Names.CONTENT_TYPE, SmileMediaTypes.APPLICATION_JACKSON_SMILE) + .setContent(smileMapper.writeValueAsBytes(lookupsUpdate)), + makeResponseHandler(returnCode, reasonString), + lookupCoordinatorManagerConfig.getHostTimeout() + ).get()) { + if (httpStatusIsSuccess(returnCode.get())) { + try { + final LookupsState response = smileMapper.readValue( + result, + LOOKUPS_STATE_TYPE_REFERENCE + ); + LOG.debug( + "Update on [%s], Status: %s reason: [%s], Response [%s].", url, returnCode.get(), reasonString.get(), + response + ); + return response; + } catch (IOException ex) { + throw new IOE( + ex, "Failed to parse update response from [%s]. response [%s]", url, result + ); + } + } else { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + try { + StreamUtils.copyAndClose(result, baos); + } + catch (IOException e2) { + LOG.warn(e2, "Error reading response"); + } + + throw new IOE( + "Bad update request to [%s] : [%d] : [%s] Response: [%s]", + url, + returnCode.get(), + reasonString.get(), + StringUtils.fromUtf8(baos.toByteArray()) + ); + } + } + } + + public LookupsState getLookupStateForNode( + HostAndPort node + ) throws IOException, InterruptedException, ExecutionException + { + final URL url = getLookupsURL(node); + final AtomicInteger returnCode = new AtomicInteger(0); + final AtomicReference reasonString = new AtomicReference<>(null); + + LOG.debug("Getting lookups from [%s]", url); + + try (final InputStream result = httpClient.go( + new Request(HttpMethod.GET, url) + .addHeader(HttpHeaders.Names.ACCEPT, SmileMediaTypes.APPLICATION_JACKSON_SMILE), + makeResponseHandler(returnCode, reasonString), + lookupCoordinatorManagerConfig.getHostTimeout() + ).get()) { + if (returnCode.get() == HttpURLConnection.HTTP_OK) { + try { + final LookupsState response = smileMapper.readValue( + result, + LOOKUPS_STATE_TYPE_REFERENCE + ); + LOG.debug( + "Get on [%s], Status: %s reason: [%s], Response [%s].", url, returnCode.get(), reasonString.get(), + response + ); + return response; + } catch(IOException ex) { + throw new IOE( + ex, + "Failed to parser GET lookups response from [%s]. response [%s].", + url, + result + ); + } + } else { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + try { + StreamUtils.copyAndClose(result, baos); + } + catch (IOException ex) { + LOG.warn(ex, "Error reading response from GET on url [%s]", url); + } + + throw new IOE( + "GET request failed to [%s] : [%d] : [%s] Response: [%s]", + url, + returnCode.get(), + reasonString.get(), + StringUtils.fromUtf8(baos.toByteArray()) + ); + } + } + } + + @VisibleForTesting + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) + { + return new SequenceInputStreamResponseHandler() + { + @Override + public ClientResponse handleResponse(HttpResponse response) + { + returnCode.set(response.getStatus().getCode()); + reasonString.set(response.getStatus().getReasonPhrase()); + return super.handleResponse(response); + } + }; + } + } } diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManagerConfig.java b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManagerConfig.java index 34c705b6b96d..37ef88e556a3 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManagerConfig.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManagerConfig.java @@ -26,63 +26,45 @@ public class LookupCoordinatorManagerConfig { - public static final Duration DEFAULT_HOST_DELETE_TIMEOUT = Duration.millis(1_000L); - public static final Duration DEFAULT_HOST_UPDATE_TIMEOUT = Duration.millis(10_000L); - public static final Duration DEFAULT_DELETE_ALL_TIMEOUT = Duration.millis(10_000L); - public static final Duration DEFAULT_UPDATE_ALL_TIMEOUT = Duration.millis(60_000L); - @JsonProperty - private Duration hostDeleteTimeout = null; - @JsonProperty - private Duration hostUpdateTimeout = null; + public static final Duration DEFAULT_HOST_TIMEOUT = Duration.millis(2_000L); + public static final Duration DEFAULT_ALL_HOST_TIMEOUT = Duration.millis(900_000L); + @JsonProperty - private Duration deleteAllTimeout = null; + private Duration hostTimeout = null; + @JsonProperty - private Duration updateAllTimeout = null; + private Duration allHostTimeout = null; + @JsonProperty - @Min(1) + @Min(2) //minimum 2 threads, one for lookupManagementLoop and one for talking to the lookup nodes private int threadPoolSize = 10; + @JsonProperty @Min(1) - private long period = 30_000L; - - public Duration getHostDeleteTimeout() - { - return hostDeleteTimeout == null ? DEFAULT_HOST_DELETE_TIMEOUT : hostDeleteTimeout; - } - - public void setHostDeleteTimeout(Duration hostDeleteTimeout) - { - this.hostDeleteTimeout = hostDeleteTimeout; - } + private long period = 120_000L; - public Duration getHostUpdateTimeout() - { - return hostUpdateTimeout == null ? DEFAULT_HOST_UPDATE_TIMEOUT : hostUpdateTimeout; - } - - public void setHostUpdateTimeout(Duration hostUpdateTimeout) - { - this.hostUpdateTimeout = hostUpdateTimeout; - } + @JsonProperty + @Min(1) + private long initialDelay = 2_000L; - public Duration getDeleteAllTimeout() + public Duration getHostTimeout() { - return deleteAllTimeout == null ? DEFAULT_DELETE_ALL_TIMEOUT : deleteAllTimeout; + return hostTimeout == null ? DEFAULT_HOST_TIMEOUT : hostTimeout; } - public void setDeleteAllTimeout(Duration deleteAllTimeout) + public void setHostTimeout(Duration hostTimeout) { - this.deleteAllTimeout = deleteAllTimeout; + this.hostTimeout = hostTimeout; } - public Duration getUpdateAllTimeout() + public Duration getAllHostTimeout() { - return updateAllTimeout == null ? DEFAULT_UPDATE_ALL_TIMEOUT : updateAllTimeout; + return allHostTimeout == null ? DEFAULT_ALL_HOST_TIMEOUT : allHostTimeout; } - public void setUpdateAllTimeout(Duration updateAllTimeout) + public void setAllHostTimeout(Duration allHostTimeout) { - this.updateAllTimeout = updateAllTimeout; + this.allHostTimeout = allHostTimeout; } public int getThreadPoolSize() @@ -104,4 +86,14 @@ public void setPeriod(long period) { this.period = period; } + + public long getInitialDelay() + { + return initialDelay; + } + + public void getInitialDelay(long initialDelay) + { + this.initialDelay = initialDelay; + } } diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainer.java b/server/src/main/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainer.java new file mode 100644 index 000000000000..d89d9c6315e3 --- /dev/null +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainer.java @@ -0,0 +1,100 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.lookup.cache; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.Ordering; + +import java.util.Map; +import java.util.Objects; + +/** + * This is same as LookupExtractorFactoryContainer except it uses Map instead of + * LookupExtractorFactory for referencing lookup spec so that lookup extensions are not required to + * be loaded at the Coordinator. + */ +public class LookupExtractorFactoryMapContainer +{ + private final static Ordering VERSION_COMPARATOR = Ordering.natural().nullsFirst(); + + private final String version; + private final Map lookupExtractorFactory; + + @JsonCreator + public LookupExtractorFactoryMapContainer( + @JsonProperty("version") String version, + @JsonProperty("lookupExtractorFactory") Map lookupExtractorFactory + ) + { + this.version = version; + this.lookupExtractorFactory = Preconditions.checkNotNull(lookupExtractorFactory, "null factory"); + } + + @JsonProperty + public String getVersion() + { + return version; + } + + @JsonProperty + public Map getLookupExtractorFactory() + { + return lookupExtractorFactory; + } + + public boolean replaces(LookupExtractorFactoryMapContainer other) { + if (version == null && other.getVersion() == null) { + return false; + } + + return VERSION_COMPARATOR.compare(version, other.getVersion()) > 0; + } + + @Override + public String toString() + { + return "LookupExtractorFactoryContainer{" + + "version='" + version + '\'' + + ", lookupExtractorFactory=" + lookupExtractorFactory + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LookupExtractorFactoryMapContainer that = (LookupExtractorFactoryMapContainer) o; + return Objects.equals(version, that.version) && + Objects.equals(lookupExtractorFactory, that.lookupExtractorFactory); + } + + @Override + public int hashCode() + { + return Objects.hash(version, lookupExtractorFactory); + } +} diff --git a/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java b/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java index 061bbefbd2e3..6c94da7f8967 100644 --- a/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java +++ b/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java @@ -27,6 +27,7 @@ import io.druid.query.extraction.ExtractionFn; import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.lookup.LookupExtractor; +import io.druid.query.lookup.LookupExtractorFactoryContainer; import io.druid.query.lookup.LookupReferencesManager; import io.druid.query.lookup.MapLookupExtractorFactory; import junitparams.JUnitParamsRunner; @@ -50,7 +51,11 @@ public class LookupDimensionSpecTest private static final LookupReferencesManager LOOKUP_REF_MANAGER = EasyMock.createMock(LookupReferencesManager.class); static { - EasyMock.expect(LOOKUP_REF_MANAGER.get(EasyMock.eq("lookupName"))).andReturn(new MapLookupExtractorFactory(STRING_MAP, false) + EasyMock.expect(LOOKUP_REF_MANAGER.get(EasyMock.eq("lookupName"))).andReturn( + new LookupExtractorFactoryContainer( + "v0", + new MapLookupExtractorFactory(STRING_MAP, false) + ) ).anyTimes(); EasyMock.replay(LOOKUP_REF_MANAGER); } diff --git a/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java b/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java index 2aaafd4ac6d7..5e012a7e82f6 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java @@ -53,7 +53,12 @@ public void setUp() throws Exception "key2", "value2" ), false); - EasyMock.expect(lookupReferencesManager.get("lookupId1")).andReturn(lookupExtractorFactory1).anyTimes(); + EasyMock.expect(lookupReferencesManager.get("lookupId1")).andReturn( + new LookupExtractorFactoryContainer( + "v0", + lookupExtractorFactory1 + ) + ).anyTimes(); EasyMock.replay(lookupReferencesManager); } diff --git a/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceTest.java b/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceTest.java index 7974318b1506..9bd895640e77 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceTest.java @@ -44,7 +44,12 @@ public class LookupIntrospectionResourceTest @Before public void setUp() { - EasyMock.expect(lookupReferencesManager.get("lookupId")).andReturn(lookupExtractorFactory).anyTimes(); + EasyMock.expect(lookupReferencesManager.get("lookupId")).andReturn( + new LookupExtractorFactoryContainer( + "v0", + lookupExtractorFactory + ) + ).anyTimes(); EasyMock.expect(lookupReferencesManager.get(EasyMock.anyString())).andReturn(null).anyTimes(); EasyMock.replay(lookupReferencesManager); } @@ -121,7 +126,12 @@ public LookupExtractor get() }; LookupIntrospectionResource lookupIntrospectionResource = new LookupIntrospectionResource(lookupReferencesManager); - EasyMock.expect(lookupReferencesManager.get("lookupId1")).andReturn(lookupExtractorFactory1).anyTimes(); + EasyMock.expect(lookupReferencesManager.get("lookupId1")).andReturn( + new LookupExtractorFactoryContainer( + "v0", + lookupExtractorFactory1 + ) + ).anyTimes(); EasyMock.replay(lookupReferencesManager); } diff --git a/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java b/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java index b8d2acbd0ef5..8aaf8b5c86ed 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupReferencesManagerTest.java @@ -20,628 +20,308 @@ package io.druid.query.lookup; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; -import com.google.common.io.Files; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; - -import io.druid.concurrent.Execs; +import com.metamx.emitter.EmittingLogger; import io.druid.jackson.DefaultObjectMapper; -import io.druid.java.util.common.ISE; -import io.druid.java.util.common.StringUtils; - +import io.druid.server.metrics.NoopServiceEmitter; import org.easymock.EasyMock; -import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nullable; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.BrokenBarrierException; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.CyclicBarrier; import java.util.concurrent.TimeUnit; public class LookupReferencesManagerTest { - private static final int CONCURRENT_THREADS = 16; LookupReferencesManager lookupReferencesManager; @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); ObjectMapper mapper = new DefaultObjectMapper(); - private final ListeningExecutorService executorService = MoreExecutors.listeningDecorator(Execs.multiThreaded( - CONCURRENT_THREADS, - "hammer-time-%s" - )); @Before public void setUp() throws IOException { + EmittingLogger.registerEmitter(new NoopServiceEmitter()); + mapper.registerSubtypes(MapLookupExtractorFactory.class); lookupReferencesManager = new LookupReferencesManager( - new LookupConfig(Files.createTempDir().getAbsolutePath()), - mapper + new LookupConfig(temporaryFolder.newFolder().getAbsolutePath()), + mapper, + true ); - Assert.assertTrue("must be closed before start call", lookupReferencesManager.isClosed()); - lookupReferencesManager.start(); - Assert.assertFalse("must start after start call", lookupReferencesManager.isClosed()); } - @After - public void tearDown() + @Test + public void testStartStop() { + lookupReferencesManager = new LookupReferencesManager( + new LookupConfig(null), + mapper + ); + + Assert.assertFalse(lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); + Assert.assertNull(lookupReferencesManager.mainThread); + Assert.assertNull(lookupReferencesManager.stateRef.get()); + + lookupReferencesManager.start(); + Assert.assertTrue(lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); + Assert.assertTrue(lookupReferencesManager.mainThread.isAlive()); + Assert.assertNotNull(lookupReferencesManager.stateRef.get()); + lookupReferencesManager.stop(); - Assert.assertTrue("stop call should close it", lookupReferencesManager.isClosed()); - executorService.shutdownNow(); + Assert.assertFalse(lookupReferencesManager.lifecycleLock.awaitStarted(1, TimeUnit.MICROSECONDS)); + Assert.assertFalse(lookupReferencesManager.mainThread.isAlive()); } - @Test(expected = ISE.class) + @Test(expected = IllegalStateException.class) public void testGetExceptionWhenClosed() { - lookupReferencesManager.stop(); lookupReferencesManager.get("test"); } - @Test(expected = ISE.class) + @Test(expected = IllegalStateException.class) public void testAddExceptionWhenClosed() { - lookupReferencesManager.stop(); - lookupReferencesManager.put("test", EasyMock.createMock(LookupExtractorFactory.class)); + lookupReferencesManager.add("test", EasyMock.createMock(LookupExtractorFactoryContainer.class)); + } + + @Test(expected = IllegalStateException.class) + public void testRemoveExceptionWhenClosed() + { + lookupReferencesManager.remove("test"); + } + + @Test(expected = IllegalStateException.class) + public void testGetAllLookupsStateExceptionWhenClosed() + { + lookupReferencesManager.getAllLookupsState(); } @Test - public void testPutGetRemove() + public void testAddGetRemove() throws Exception { LookupExtractorFactory lookupExtractorFactory = EasyMock.createMock(LookupExtractorFactory.class); EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); EasyMock.expect(lookupExtractorFactory.close()).andReturn(true).once(); EasyMock.replay(lookupExtractorFactory); + lookupReferencesManager.start(); Assert.assertNull(lookupReferencesManager.get("test")); - lookupReferencesManager.put("test", lookupExtractorFactory); - Assert.assertEquals(lookupExtractorFactory, lookupReferencesManager.get("test")); - Assert.assertTrue(lookupReferencesManager.remove("test")); + + LookupExtractorFactoryContainer testContainer = new LookupExtractorFactoryContainer("0", lookupExtractorFactory); + + lookupReferencesManager.add("test", testContainer); + lookupReferencesManager.handlePendingNotices(); + + Assert.assertEquals(testContainer, lookupReferencesManager.get("test")); + + lookupReferencesManager.remove("test"); + lookupReferencesManager.handlePendingNotices(); + Assert.assertNull(lookupReferencesManager.get("test")); } @Test - public void testCloseIsCalledAfterStopping() throws IOException + public void testCloseIsCalledAfterStopping() throws Exception { LookupExtractorFactory lookupExtractorFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); EasyMock.expect(lookupExtractorFactory.close()).andReturn(true).once(); EasyMock.replay(lookupExtractorFactory); - lookupReferencesManager.put("testMock", lookupExtractorFactory); + lookupReferencesManager.start(); + lookupReferencesManager.add("testMock", new LookupExtractorFactoryContainer("0", lookupExtractorFactory)); + lookupReferencesManager.handlePendingNotices(); + lookupReferencesManager.stop(); EasyMock.verify(lookupExtractorFactory); } @Test - public void testCloseIsCalledAfterRemove() throws IOException + public void testCloseIsCalledAfterRemove() throws Exception { LookupExtractorFactory lookupExtractorFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); EasyMock.expect(lookupExtractorFactory.close()).andReturn(true).once(); EasyMock.replay(lookupExtractorFactory); - lookupReferencesManager.put("testMock", lookupExtractorFactory); + + lookupReferencesManager.start(); + lookupReferencesManager.add("testMock", new LookupExtractorFactoryContainer("0", lookupExtractorFactory)); + lookupReferencesManager.handlePendingNotices(); + lookupReferencesManager.remove("testMock"); - EasyMock.verify(lookupExtractorFactory); - } + lookupReferencesManager.handlePendingNotices(); - @Test - public void testRemoveInExisting() - { - Assert.assertFalse(lookupReferencesManager.remove("notThere")); + EasyMock.verify(lookupExtractorFactory); } @Test public void testGetNotThere() { + lookupReferencesManager.start(); Assert.assertNull(lookupReferencesManager.get("notThere")); } @Test - public void testAddingWithSameLookupName() + public void testUpdateWithHigherVersion() throws Exception { - LookupExtractorFactory lookupExtractorFactory = EasyMock.createNiceMock(LookupExtractorFactory.class); - EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); - LookupExtractorFactory lookupExtractorFactory2 = EasyMock.createNiceMock(LookupExtractorFactory.class); - EasyMock.expect(lookupExtractorFactory2.start()).andReturn(true).times(2); - EasyMock.replay(lookupExtractorFactory, lookupExtractorFactory2); - Assert.assertTrue(lookupReferencesManager.put("testName", lookupExtractorFactory)); - Assert.assertFalse(lookupReferencesManager.put("testName", lookupExtractorFactory2)); - ImmutableMap extractorImmutableMap = ImmutableMap.of( - "testName", - lookupExtractorFactory2 - ); - lookupReferencesManager.put(extractorImmutableMap); - Assert.assertEquals(lookupExtractorFactory, lookupReferencesManager.get("testName")); - } + LookupExtractorFactory lookupExtractorFactory1 = EasyMock.createNiceMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory1.start()).andReturn(true).once(); + EasyMock.expect(lookupExtractorFactory1.close()).andReturn(true).once(); - @Test - public void testAddLookupsThenGetAll() - { - LookupExtractorFactory lookupExtractorFactory = EasyMock.createNiceMock(LookupExtractorFactory.class); - EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); LookupExtractorFactory lookupExtractorFactory2 = EasyMock.createNiceMock(LookupExtractorFactory.class); EasyMock.expect(lookupExtractorFactory2.start()).andReturn(true).once(); - EasyMock.replay(lookupExtractorFactory, lookupExtractorFactory2); - ImmutableMap extractorImmutableMap = ImmutableMap.of( - "name1", - lookupExtractorFactory, - "name2", - lookupExtractorFactory2 - ); - lookupReferencesManager.put(extractorImmutableMap); - Assert.assertEquals(extractorImmutableMap, lookupReferencesManager.getAll()); - } - - @Test(expected = ISE.class) - public void testExceptionWhenStartFail() - { - LookupExtractorFactory lookupExtractorFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); - EasyMock.expect(lookupExtractorFactory.start()).andReturn(false).once(); - EasyMock.replay(lookupExtractorFactory); - lookupReferencesManager.put("testMock", lookupExtractorFactory); - } - - @Test(expected = ISE.class) - public void testputAllExceptionWhenStartFail() - { - LookupExtractorFactory lookupExtractorFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); - EasyMock.expect(lookupExtractorFactory.start()).andReturn(false).once(); - ImmutableMap extractorImmutableMap = ImmutableMap.of( - "name1", - lookupExtractorFactory - ); - lookupReferencesManager.put(extractorImmutableMap); - } - - @Test - public void testUpdateIfNewOnlyIfIsNew() - { - final String lookupName = "some lookup"; - LookupExtractorFactory oldFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); - LookupExtractorFactory newFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); - EasyMock.expect(oldFactory.replaces(EasyMock.isNull())).andReturn(true).once(); - EasyMock.expect(oldFactory.start()).andReturn(true).once(); - EasyMock.expect(oldFactory.replaces(EasyMock.eq(oldFactory))).andReturn(false).once(); - // Add new + EasyMock.replay(lookupExtractorFactory1, lookupExtractorFactory2); - EasyMock.expect(newFactory.replaces(EasyMock.eq(oldFactory))).andReturn(true).once(); - EasyMock.expect(newFactory.start()).andReturn(true).once(); - EasyMock.expect(oldFactory.close()).andReturn(true).once(); - EasyMock.expect(newFactory.close()).andReturn(true).once(); - - EasyMock.replay(oldFactory, newFactory); - - Assert.assertTrue(lookupReferencesManager.updateIfNew(lookupName, oldFactory)); - Assert.assertFalse(lookupReferencesManager.updateIfNew(lookupName, oldFactory)); - Assert.assertTrue(lookupReferencesManager.updateIfNew(lookupName, newFactory)); - - // Remove now or else EasyMock gets confused on lazy lookup manager stop handling - lookupReferencesManager.remove(lookupName); + lookupReferencesManager.start(); + lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("1", lookupExtractorFactory1)); + lookupReferencesManager.handlePendingNotices(); - EasyMock.verify(oldFactory, newFactory); - } + lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("2", lookupExtractorFactory2)); + lookupReferencesManager.handlePendingNotices(); - @Test(expected = ISE.class) - public void testUpdateIfNewExceptional() - { - final String lookupName = "some lookup"; - LookupExtractorFactory newFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); - EasyMock.expect(newFactory.replaces(EasyMock.isNull())).andReturn(true).once(); - EasyMock.expect(newFactory.start()).andReturn(false).once(); - EasyMock.replay(newFactory); - try { - lookupReferencesManager.updateIfNew(lookupName, newFactory); - } - finally { - EasyMock.verify(newFactory); - } + EasyMock.verify(lookupExtractorFactory1, lookupExtractorFactory2); } @Test - public void testUpdateIfNewSuppressOldCloseProblem() + public void testUpdateWithLowerVersion() throws Exception { - final String lookupName = "some lookup"; - LookupExtractorFactory oldFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); - LookupExtractorFactory newFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); + LookupExtractorFactory lookupExtractorFactory1 = EasyMock.createNiceMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory1.start()).andReturn(true).once(); - EasyMock.expect(oldFactory.replaces(EasyMock.isNull())).andReturn(true).once(); - EasyMock.expect(oldFactory.start()).andReturn(true).once(); - // Add new - EasyMock.expect(newFactory.replaces(EasyMock.eq(oldFactory))).andReturn(true).once(); - EasyMock.expect(newFactory.start()).andReturn(true).once(); - EasyMock.expect(oldFactory.close()).andReturn(false).once(); - EasyMock.expect(newFactory.close()).andReturn(true).once(); + LookupExtractorFactory lookupExtractorFactory2 = EasyMock.createNiceMock(LookupExtractorFactory.class); - EasyMock.replay(oldFactory, newFactory); + EasyMock.replay(lookupExtractorFactory1, lookupExtractorFactory2); - lookupReferencesManager.updateIfNew(lookupName, oldFactory); - lookupReferencesManager.updateIfNew(lookupName, newFactory); + lookupReferencesManager.start(); + lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("1", lookupExtractorFactory1)); + lookupReferencesManager.handlePendingNotices(); - // Remove now or else EasyMock gets confused on lazy lookup manager stop handling - lookupReferencesManager.remove(lookupName); + lookupReferencesManager.add("testName", new LookupExtractorFactoryContainer("0", lookupExtractorFactory2)); + lookupReferencesManager.handlePendingNotices(); - EasyMock.verify(oldFactory, newFactory); + EasyMock.verify(lookupExtractorFactory1, lookupExtractorFactory2); } @Test - public void testBootstrapFromFile() throws IOException + public void testRemoveNonExisting() throws Exception { - LookupExtractorFactory lookupExtractorFactory = new MapLookupExtractorFactory(ImmutableMap.of( - "key", - "value" - ), true); - lookupReferencesManager.put("testMockForBootstrap", lookupExtractorFactory); - lookupReferencesManager.stop(); lookupReferencesManager.start(); - Assert.assertEquals(lookupExtractorFactory, lookupReferencesManager.get("testMockForBootstrap")); - + lookupReferencesManager.remove("test"); + lookupReferencesManager.handlePendingNotices(); } @Test - public void testConcurrencyStaaaaaaaaaaartStop() throws Exception + public void testBootstrapFromFile() throws Exception { + LookupExtractorFactory lookupExtractorFactory = new MapLookupExtractorFactory( + ImmutableMap.of( + "key", + "value" + ), true + ); + LookupExtractorFactoryContainer container = new LookupExtractorFactoryContainer("v0", lookupExtractorFactory); + lookupReferencesManager.start(); + lookupReferencesManager.add("testMockForBootstrap", container); + lookupReferencesManager.handlePendingNotices(); lookupReferencesManager.stop(); - final CyclicBarrier cyclicBarrier = new CyclicBarrier(CONCURRENT_THREADS); - final Runnable start = new Runnable() - { - @Override - public void run() - { - try { - cyclicBarrier.await(); - } - catch (InterruptedException | BrokenBarrierException e) { - throw Throwables.propagate(e); - } - lookupReferencesManager.start(); - } - }; - final Collection> futures = new ArrayList<>(CONCURRENT_THREADS); - for (int i = 0; i < CONCURRENT_THREADS; ++i) { - futures.add(executorService.submit(start)); - } - lookupReferencesManager.stop(); - Futures.allAsList(futures).get(100, TimeUnit.MILLISECONDS); - for (ListenableFuture future : futures) { - Assert.assertNull(future.get()); - } - } - @Test - public void testConcurrencyStartStoooooooooop() throws Exception - { - lookupReferencesManager.stop(); + lookupReferencesManager = new LookupReferencesManager( + new LookupConfig(lookupReferencesManager.lookupSnapshotTaker.getPersistFile().getParent()), + mapper, + true + ); lookupReferencesManager.start(); - final CyclicBarrier cyclicBarrier = new CyclicBarrier(CONCURRENT_THREADS); - final Runnable start = new Runnable() - { - @Override - public void run() - { - try { - cyclicBarrier.await(); - } - catch (InterruptedException | BrokenBarrierException e) { - throw Throwables.propagate(e); - } - lookupReferencesManager.stop(); - } - }; - final Collection> futures = new ArrayList<>(CONCURRENT_THREADS); - for (int i = 0; i < CONCURRENT_THREADS; ++i) { - futures.add(executorService.submit(start)); - } - Futures.allAsList(futures).get(100, TimeUnit.MILLISECONDS); - for (ListenableFuture future : futures) { - Assert.assertNull(future.get()); - } + Assert.assertEquals(container, lookupReferencesManager.get("testMockForBootstrap")); } - @Test(timeout = 10000L) - public void testConcurrencySequentialChaos() throws Exception + @Test + public void testGetAllLookupsState() throws Exception { - final CountDownLatch runnableStartBarrier = new CountDownLatch(1); - final Random random = new Random(478137498L); - final int numUpdates = 100000; - final int numNamespaces = 100; - final CountDownLatch runnablesFinishedBarrier = new CountDownLatch(numUpdates); - final List runnables = new ArrayList<>(numUpdates); - final Map maxNumber = new HashMap<>(); - for (int i = 1; i <= numUpdates; ++i) { - final boolean shouldStart = random.nextInt(10) == 1; - final boolean shouldClose = random.nextInt(10) == 1; - final String name = Integer.toString(random.nextInt(numNamespaces)); - final int position = i; - - final LookupExtractorFactory lookupExtractorFactory = new LookupExtractorFactory() - { - @Override - public boolean start() - { - return shouldStart; - } - - @Override - public boolean close() - { - return shouldClose; - } - - @Override - public boolean replaces(@Nullable LookupExtractorFactory other) - { - if (other == null) { - return true; - } - final NamedIntrospectionHandler introspectionHandler = (NamedIntrospectionHandler) other.getIntrospectHandler(); - return position > introspectionHandler.position; - } - - @Nullable - @Override - public LookupIntrospectHandler getIntrospectHandler() - { - return new NamedIntrospectionHandler(position); - } - - @Override - public String toString() - { - return String.format("TestFactroy position %d", position); - } - - @Override - public LookupExtractor get() - { - return null; - } - }; - - if (shouldStart && (!maxNumber.containsKey(name) || maxNumber.get(name) < position)) { - maxNumber.put(name, position); - } - runnables.add(new LookupUpdatingRunnable( - name, - lookupExtractorFactory, - runnableStartBarrier, - lookupReferencesManager - )); - } - ////// Add some CHAOS! - Collections.shuffle(runnables, random); - final Runnable decrementFinished = new Runnable() - { - @Override - public void run() - { - runnablesFinishedBarrier.countDown(); - } - }; - for (Runnable runnable : runnables) { - executorService.submit(runnable).addListener(decrementFinished, MoreExecutors.sameThreadExecutor()); - } + LookupExtractorFactoryContainer container1 = new LookupExtractorFactoryContainer( + "0", + new MapLookupExtractorFactory( + ImmutableMap.of( + "key1", + "value1" + ), true + ) + ); + + LookupExtractorFactoryContainer container2 = new LookupExtractorFactoryContainer( + "0", + new MapLookupExtractorFactory( + ImmutableMap.of( + "key2", + "value2" + ), true + ) + ); - runnableStartBarrier.countDown(); - do { - for (String name : maxNumber.keySet()) { - final LookupExtractorFactory factory; - try { - factory = lookupReferencesManager.get(name); - } - catch (ISE e) { - continue; - } - if (null == factory) { - continue; - } - final NamedIntrospectionHandler introspectionHandler = (NamedIntrospectionHandler) factory.getIntrospectHandler(); - Assert.assertTrue(introspectionHandler.position >= 0); - } - } while (runnablesFinishedBarrier.getCount() > 0); + LookupExtractorFactoryContainer container3 = new LookupExtractorFactoryContainer( + "0", + new MapLookupExtractorFactory( + ImmutableMap.of( + "key3", + "value3" + ), true + ) + ); lookupReferencesManager.start(); + lookupReferencesManager.add("one", container1); + lookupReferencesManager.add("two", container2); + lookupReferencesManager.handlePendingNotices(); + lookupReferencesManager.remove("one"); + lookupReferencesManager.add("three", container3); - for (String name : maxNumber.keySet()) { - final LookupExtractorFactory factory = lookupReferencesManager.get(name); - if (null == factory) { - continue; - } - final NamedIntrospectionHandler introspectionHandler = (NamedIntrospectionHandler) factory.getIntrospectHandler(); - Assert.assertNotNull(introspectionHandler); - Assert.assertEquals( - StringUtils.safeFormat("Named position %s failed", name), - maxNumber.get(name), - Integer.valueOf(introspectionHandler.position) - ); - } - Assert.assertEquals(maxNumber.size(), lookupReferencesManager.getAll().size()); - } + LookupsState state = lookupReferencesManager.getAllLookupsState(); - @Test(timeout = 10000L) - public void testConcurrencyStartStopChaos() throws Exception - { - // Don't want to exercise snapshot here - final LookupReferencesManager manager = new LookupReferencesManager(new LookupConfig(null), mapper); - final Runnable chaosStart = new Runnable() - { - @Override - public void run() - { - manager.start(); - } - }; - final Runnable chaosStop = new Runnable() - { - @Override - public void run() - { - manager.stop(); - } - }; - final CountDownLatch runnableStartBarrier = new CountDownLatch(1); - final Random random = new Random(478137498L); - final int numUpdates = 100000; - final int numNamespaces = 100; - final CountDownLatch runnablesFinishedBarrier = new CountDownLatch(numUpdates); - final List runnables = new ArrayList<>(numUpdates); - final Map maxNumber = new HashMap<>(); - for (int i = 1; i <= numUpdates; ++i) { - final boolean shouldStart = random.nextInt(10) == 1; - final boolean shouldClose = random.nextInt(10) == 1; - final String name = Integer.toString(random.nextInt(numNamespaces)); - final int position = i; - - final LookupExtractorFactory lookupExtractorFactory = new LookupExtractorFactory() - { - @Override - public boolean start() - { - return shouldStart; - } - - @Override - public boolean close() - { - return shouldClose; - } - - @Override - public boolean replaces(@Nullable LookupExtractorFactory other) - { - if (other == null) { - return true; - } - final NamedIntrospectionHandler introspectionHandler = (NamedIntrospectionHandler) other.getIntrospectHandler(); - return position > introspectionHandler.position; - } - - @Nullable - @Override - public LookupIntrospectHandler getIntrospectHandler() - { - return new NamedIntrospectionHandler(position); - } - - @Override - public String toString() - { - return String.format("TestFactroy position %d", position); - } - - @Override - public LookupExtractor get() - { - return null; - } - }; - if (random.nextFloat() < 0.001) { - if (random.nextBoolean()) { - runnables.add(chaosStart); - } else { - runnables.add(chaosStop); - } - } else { - if (shouldStart && (!maxNumber.containsKey(name) || maxNumber.get(name) < position)) { - maxNumber.put(name, position); - } - runnables.add(new LookupUpdatingRunnable( - name, - lookupExtractorFactory, - runnableStartBarrier, - manager - )); - } - } - ////// Add some CHAOS! - Collections.shuffle(runnables, random); - final Runnable decrementFinished = new Runnable() - { - @Override - public void run() - { - runnablesFinishedBarrier.countDown(); - } - }; - for (Runnable runnable : runnables) { - executorService.submit(runnable).addListener(decrementFinished, MoreExecutors.sameThreadExecutor()); - } + Assert.assertEquals(2, state.getCurrent().size()); + Assert.assertEquals(container1, state.getCurrent().get("one")); + Assert.assertEquals(container2, state.getCurrent().get("two")); - runnableStartBarrier.countDown(); - do { - for (String name : maxNumber.keySet()) { - final LookupExtractorFactory factory; - try { - factory = manager.get(name); - } - catch (ISE e) { - continue; - } - if (null == factory) { - continue; - } - final NamedIntrospectionHandler introspectionHandler = (NamedIntrospectionHandler) factory.getIntrospectHandler(); - Assert.assertTrue(introspectionHandler.position >= 0); - } - } while (runnablesFinishedBarrier.getCount() > 0); - } -} + Assert.assertEquals(1, state.getToLoad().size()); + Assert.assertEquals(container3, state.getToLoad().get("three")); -class LookupUpdatingRunnable implements Runnable -{ - final String name; - final LookupExtractorFactory factory; - final CountDownLatch startLatch; - final LookupReferencesManager lookupReferencesManager; - - LookupUpdatingRunnable( - String name, - LookupExtractorFactory factory, - CountDownLatch startLatch, - LookupReferencesManager lookupReferencesManager - ) - { - this.name = name; - this.factory = factory; - this.startLatch = startLatch; - this.lookupReferencesManager = lookupReferencesManager; + Assert.assertEquals(1, state.getToDrop().size()); + Assert.assertTrue(state.getToDrop().contains("one")); } - @Override - public void run() + @Test (timeout = 20000) + public void testRealModeWithMainThread() throws Exception { - try { - startLatch.await(); + LookupReferencesManager lookupReferencesManager = new LookupReferencesManager( + new LookupConfig(temporaryFolder.newFolder().getAbsolutePath()), + mapper + ); + + lookupReferencesManager.start(); + Assert.assertTrue(lookupReferencesManager.mainThread.isAlive()); + + LookupExtractorFactory lookupExtractorFactory = EasyMock.createMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); + EasyMock.expect(lookupExtractorFactory.close()).andReturn(true).once(); + EasyMock.replay(lookupExtractorFactory); + Assert.assertNull(lookupReferencesManager.get("test")); + + LookupExtractorFactoryContainer testContainer = new LookupExtractorFactoryContainer("0", lookupExtractorFactory); + lookupReferencesManager.add("test", testContainer); + + while (!testContainer.equals(lookupReferencesManager.get("test"))) { + Thread.sleep(100); } - catch (InterruptedException e) { - throw Throwables.propagate(e); + + lookupReferencesManager.remove("test"); + + while (lookupReferencesManager.get("test") != null) { + Thread.sleep(100); } - lookupReferencesManager.updateIfNew(name, factory); - } -} -class NamedIntrospectionHandler implements LookupIntrospectHandler -{ - final int position; + lookupReferencesManager.stop(); - NamedIntrospectionHandler(final int position) - { - this.position = position; + Assert.assertFalse(lookupReferencesManager.mainThread.isAlive()); } } diff --git a/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java b/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java index 676fe6b2e5e1..6f69ccc7f240 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java @@ -23,10 +23,10 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.io.Files; - import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.segment.TestHelper; +import org.apache.commons.io.FileUtils; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -61,15 +61,57 @@ public void setUp() throws IOException public void testTakeSnapshotAndPullExisting() throws IOException { - LookupBean lookupBean = new LookupBean(); - lookupBean.name = "name"; - lookupBean.factory = new MapLookupExtractorFactory(ImmutableMap.of("key", "value"), true); + LookupBean lookupBean = new LookupBean( + "name", + null, + new LookupExtractorFactoryContainer( + "v1", + new MapLookupExtractorFactory( + ImmutableMap.of( + "key", + "value" + ), true + ) + ) + ); List lookupBeanList = Lists.newArrayList(lookupBean); lookupSnapshotTaker.takeSnapshot(lookupBeanList); List actualList = lookupSnapshotTaker.pullExistingSnapshot(); Assert.assertEquals(lookupBeanList, actualList); } + //test backward compatibility with snapshots stored using 0.9.x code + @Test + public void testBackwardCompatibility() throws IOException + { + File directory = temporaryFolder.newFolder(); + File snapshotFile = new File(directory, LookupSnapshotTaker.PERSIST_FILE_NAME); + Assert.assertFalse(snapshotFile.exists()); + FileUtils.write( + snapshotFile, + "[{\"factory\":{\"type\":\"map\",\"map\":{\"key\":\"value\"},\"isOneToOne\":true},\"name\":\"name\"}]" + ); + Assert.assertTrue(snapshotFile.exists()); + List actualList = new LookupSnapshotTaker(mapper, directory.getAbsolutePath()).pullExistingSnapshot(); + + LookupBean lookupBean = new LookupBean( + "name", + null, + new LookupExtractorFactoryContainer( + null, + new MapLookupExtractorFactory( + ImmutableMap.of( + "key", + "value" + ), true + ) + ) + ); + List lookupBeanList = Lists.newArrayList(lookupBean); + + Assert.assertEquals(lookupBeanList, actualList); + } + @Test(expected = ISE.class) public void testIOExceptionDuringLookupPersist() throws IOException { @@ -79,9 +121,19 @@ public void testIOExceptionDuringLookupPersist() throws IOException Assert.assertTrue(snapshotFile.createNewFile()); Assert.assertTrue(snapshotFile.setReadOnly()); LookupSnapshotTaker lookupSnapshotTaker = new LookupSnapshotTaker(mapper, directory.getAbsolutePath()); - LookupBean lookupBean = new LookupBean(); - lookupBean.name = "name"; - lookupBean.factory = new MapLookupExtractorFactory(ImmutableMap.of("key", "value"), true); + LookupBean lookupBean = new LookupBean( + "name", + null, + new LookupExtractorFactoryContainer( + "v1", + new MapLookupExtractorFactory( + ImmutableMap.of( + "key", + "value" + ), true + ) + ) + ); List lookupBeanList = Lists.newArrayList(lookupBean); lookupSnapshotTaker.takeSnapshot(lookupBeanList); } diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java index d48420d6756c..fd81aa4fed47 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java @@ -47,6 +47,7 @@ import io.druid.server.coordinator.rules.ForeverLoadRule; import io.druid.server.coordinator.rules.Rule; import io.druid.server.initialization.ZkPathsConfig; +import io.druid.server.lookup.cache.LookupCoordinatorManager; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; @@ -192,7 +193,8 @@ public void unannounce(DruidNode node) druidNode, loadManagementPeons, null, - new CostBalancerStrategyFactory() + new CostBalancerStrategyFactory(), + EasyMock.createNiceMock(LookupCoordinatorManager.class) ); } diff --git a/server/src/test/java/io/druid/server/http/LookupCoordinatorResourceTest.java b/server/src/test/java/io/druid/server/http/LookupCoordinatorResourceTest.java index 1d428d68d498..394c273b17a5 100644 --- a/server/src/test/java/io/druid/server/http/LookupCoordinatorResourceTest.java +++ b/server/src/test/java/io/druid/server/http/LookupCoordinatorResourceTest.java @@ -23,11 +23,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteSource; - import io.druid.audit.AuditInfo; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.StringUtils; import io.druid.server.lookup.cache.LookupCoordinatorManager; +import io.druid.server.lookup.cache.LookupExtractorFactoryMapContainer; import org.easymock.Capture; import org.easymock.EasyMock; import org.junit.Assert; @@ -48,11 +48,15 @@ public class LookupCoordinatorResourceTest private static final ObjectMapper mapper = new DefaultObjectMapper(); private static final String LOOKUP_TIER = "lookupTier"; private static final String LOOKUP_NAME = "lookupName"; - private static final Map> SINGLE_LOOKUP_MAP = ImmutableMap.>of( - LOOKUP_NAME, + private static final LookupExtractorFactoryMapContainer SINGLE_LOOKUP = new LookupExtractorFactoryMapContainer( + "v0", ImmutableMap.of() ); - private static final Map>> SINGLE_TIER_MAP = ImmutableMap.>>of( + private static final Map SINGLE_LOOKUP_MAP = ImmutableMap.of( + LOOKUP_NAME, + SINGLE_LOOKUP + ); + private static final Map> SINGLE_TIER_MAP = ImmutableMap.of( LOOKUP_TIER, SINGLE_LOOKUP_MAP ); @@ -69,7 +73,7 @@ public InputStream openStream() throws IOException @Override public InputStream openStream() throws IOException { - return new ByteArrayInputStream(StringUtils.toUtf8(mapper.writeValueAsString(ImmutableMap.of()))); + return new ByteArrayInputStream(StringUtils.toUtf8(mapper.writeValueAsString(SINGLE_LOOKUP))); } }; @@ -78,7 +82,7 @@ public void testSimpleGet() { final LookupCoordinatorManager lookupCoordinatorManager = EasyMock.createStrictMock( LookupCoordinatorManager.class); - final Map>> retVal = new HashMap<>(); + final Map> retVal = new HashMap<>(); EasyMock.expect(lookupCoordinatorManager.getKnownLookups()).andReturn(retVal).once(); EasyMock.replay(lookupCoordinatorManager); final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( @@ -150,7 +154,6 @@ public void testDiscoveryGet() @Test public void testDiscoveryExceptionalGet() { - final List tiers = ImmutableList.of(); final String errMsg = "some error"; final RuntimeException ex = new RuntimeException(errMsg); final LookupCoordinatorManager lookupCoordinatorManager = EasyMock.createStrictMock( @@ -171,11 +174,14 @@ public void testDiscoveryExceptionalGet() @Test public void testSimpleGetLookup() { - final Map map = new HashMap<>(); + final LookupExtractorFactoryMapContainer container = new LookupExtractorFactoryMapContainer( + "v0", + new HashMap() + ); final LookupCoordinatorManager lookupCoordinatorManager = EasyMock.createStrictMock( LookupCoordinatorManager.class); EasyMock.expect(lookupCoordinatorManager.getLookup(EasyMock.eq(LOOKUP_TIER), EasyMock.eq(LOOKUP_NAME))) - .andReturn(map) + .andReturn(container) .once(); EasyMock.replay(lookupCoordinatorManager); final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( @@ -185,7 +191,7 @@ public void testSimpleGetLookup() ); final Response response = lookupCoordinatorResource.getSpecificLookup(LOOKUP_TIER, LOOKUP_NAME); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(map, response.getEntity()); + Assert.assertEquals(container, response.getEntity()); EasyMock.verify(lookupCoordinatorManager); } @@ -549,7 +555,7 @@ public void testSimpleNewLookup() throws Exception EasyMock.expect(lookupCoordinatorManager.updateLookup( EasyMock.eq(LOOKUP_TIER), EasyMock.eq(LOOKUP_NAME), - EasyMock.eq(ImmutableMap.of()), + EasyMock.eq(SINGLE_LOOKUP), EasyMock.capture(auditInfoCapture) )).andReturn(true).once(); @@ -597,7 +603,7 @@ public void testDBErrNewLookup() throws Exception EasyMock.expect(lookupCoordinatorManager.updateLookup( EasyMock.eq(LOOKUP_TIER), EasyMock.eq(LOOKUP_NAME), - EasyMock.eq(ImmutableMap.of()), + EasyMock.eq(SINGLE_LOOKUP), EasyMock.capture(auditInfoCapture) )).andReturn(false).once(); @@ -646,7 +652,7 @@ public void testExceptionalNewLookup() throws Exception EasyMock.expect(lookupCoordinatorManager.updateLookup( EasyMock.eq(LOOKUP_TIER), EasyMock.eq(LOOKUP_NAME), - EasyMock.eq(ImmutableMap.of()), + EasyMock.eq(SINGLE_LOOKUP), EasyMock.capture(auditInfoCapture) )).andThrow(new RuntimeException(errMsg)).once(); @@ -738,23 +744,17 @@ public void testNullValsNewLookup() throws Exception @Test public void testSimpleGetTier() { - final String tier = "some tier"; - final String lookup = "some lookup"; final LookupCoordinatorManager lookupCoordinatorManager = EasyMock.createStrictMock(LookupCoordinatorManager.class); - final Map>> retVal = - ImmutableMap.>>of( - tier, ImmutableMap.>of(lookup, ImmutableMap.of()) - ); - EasyMock.expect(lookupCoordinatorManager.getKnownLookups()).andReturn(retVal).once(); + EasyMock.expect(lookupCoordinatorManager.getKnownLookups()).andReturn(SINGLE_TIER_MAP).once(); EasyMock.replay(lookupCoordinatorManager); final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( lookupCoordinatorManager, mapper, mapper ); - final Response response = lookupCoordinatorResource.getSpecificTier(tier); + final Response response = lookupCoordinatorResource.getSpecificTier(LOOKUP_TIER); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(retVal.get(tier).keySet(), response.getEntity()); + Assert.assertEquals(SINGLE_TIER_MAP.get(LOOKUP_TIER).keySet(), response.getEntity()); EasyMock.verify(lookupCoordinatorManager); } @@ -765,7 +765,7 @@ public void testMissingGetTier() final LookupCoordinatorManager lookupCoordinatorManager = EasyMock.createStrictMock(LookupCoordinatorManager.class); final Map>> retVal = ImmutableMap.>>of(); - EasyMock.expect(lookupCoordinatorManager.getKnownLookups()).andReturn(retVal).once(); + EasyMock.expect(lookupCoordinatorManager.getKnownLookups()).andReturn(SINGLE_TIER_MAP).once(); EasyMock.replay(lookupCoordinatorManager); final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( lookupCoordinatorManager, diff --git a/server/src/test/java/io/druid/server/listener/resource/AbstractListenerHandlerTest.java b/server/src/test/java/io/druid/server/listener/resource/AbstractListenerHandlerTest.java index 4f7e29a9007e..cb6a543354cb 100644 --- a/server/src/test/java/io/druid/server/listener/resource/AbstractListenerHandlerTest.java +++ b/server/src/test/java/io/druid/server/listener/resource/AbstractListenerHandlerTest.java @@ -61,6 +61,12 @@ public class AbstractListenerHandlerTest final AbstractListenerHandler abstractListenerHandler = new AbstractListenerHandler(SomeBeanClass.TYPE_REFERENCE) { + @Override + public Response handleUpdates(InputStream inputStream, ObjectMapper mapper) + { + return null; + } + @Nullable @Override public Object post(@NotNull Map inputObject) throws Exception diff --git a/server/src/test/java/io/druid/server/listener/resource/ListenerResourceTest.java b/server/src/test/java/io/druid/server/listener/resource/ListenerResourceTest.java index 4456645f905d..3a8c9f3353e8 100644 --- a/server/src/test/java/io/druid/server/listener/resource/ListenerResourceTest.java +++ b/server/src/test/java/io/druid/server/listener/resource/ListenerResourceTest.java @@ -495,4 +495,10 @@ public Object post(@NotNull Map inputObject) throws Excep { throw new UnsupportedOperationException("should not have called post"); } + + @Override + public Response handleUpdates(InputStream inputStream, ObjectMapper mapper) + { + throw new UnsupportedOperationException("should not have called handleUpdates"); + } } diff --git a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerConfigTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerConfigTest.java index 0d9ec282bb9b..a4858f709d61 100644 --- a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerConfigTest.java +++ b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerConfigTest.java @@ -30,17 +30,13 @@ public void testConfigsTakeOverrides() { final Duration funnyDuration = Duration.standardDays(100); final LookupCoordinatorManagerConfig config = new LookupCoordinatorManagerConfig(); - config.setDeleteAllTimeout(funnyDuration); - config.setHostDeleteTimeout(funnyDuration); - config.setHostUpdateTimeout(funnyDuration); - config.setUpdateAllTimeout(funnyDuration); + config.setHostTimeout(funnyDuration); + config.setAllHostTimeout(funnyDuration); config.setPeriod(funnyDuration.getMillis()); config.setThreadPoolSize(1200); - Assert.assertEquals(funnyDuration, config.getDeleteAllTimeout()); - Assert.assertEquals(funnyDuration, config.getHostDeleteTimeout()); - Assert.assertEquals(funnyDuration, config.getHostUpdateTimeout()); - Assert.assertEquals(funnyDuration, config.getUpdateAllTimeout()); + Assert.assertEquals(funnyDuration, config.getHostTimeout()); + Assert.assertEquals(funnyDuration, config.getAllHostTimeout()); Assert.assertEquals(funnyDuration.getMillis(), config.getPeriod()); Assert.assertEquals(1200, config.getThreadPoolSize()); } @@ -49,11 +45,9 @@ public void testConfigsTakeOverrides() public void testSimpleConfigDefaults() { final LookupCoordinatorManagerConfig config = new LookupCoordinatorManagerConfig(); - Assert.assertEquals(LookupCoordinatorManagerConfig.DEFAULT_DELETE_ALL_TIMEOUT, config.getDeleteAllTimeout()); - Assert.assertEquals(LookupCoordinatorManagerConfig.DEFAULT_HOST_DELETE_TIMEOUT, config.getHostDeleteTimeout()); - Assert.assertEquals(LookupCoordinatorManagerConfig.DEFAULT_HOST_UPDATE_TIMEOUT, config.getHostUpdateTimeout()); - Assert.assertEquals(LookupCoordinatorManagerConfig.DEFAULT_UPDATE_ALL_TIMEOUT, config.getUpdateAllTimeout()); + Assert.assertEquals(LookupCoordinatorManagerConfig.DEFAULT_HOST_TIMEOUT, config.getHostTimeout()); + Assert.assertEquals(LookupCoordinatorManagerConfig.DEFAULT_ALL_HOST_TIMEOUT, config.getAllHostTimeout()); Assert.assertEquals(10, config.getThreadPoolSize()); - Assert.assertEquals(30_000, config.getPeriod()); + Assert.assertEquals(120_000, config.getPeriod()); } } diff --git a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index aa0b2fc91f56..c7c0247bd14a 100644 --- a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java +++ b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java @@ -19,11 +19,11 @@ package io.druid.server.lookup.cache; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.net.HostAndPort; import com.google.common.util.concurrent.SettableFuture; import com.metamx.emitter.core.Event; @@ -35,12 +35,13 @@ import com.metamx.http.client.response.SequenceInputStreamResponseHandler; import io.druid.audit.AuditInfo; import io.druid.common.config.JacksonConfigManager; +import io.druid.common.utils.StringUtils; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.StringUtils; import io.druid.query.lookup.LookupModule; +import io.druid.query.lookup.LookupsState; import io.druid.server.listener.announcer.ListenerDiscoverer; -import io.druid.server.listener.resource.ListenerResource; import org.easymock.EasyMock; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; @@ -53,15 +54,14 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import javax.ws.rs.core.Response; import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; -import java.net.URL; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -78,19 +78,39 @@ public class LookupCoordinatorManagerTest private static final String LOOKUP_TIER = "lookup_tier"; private static final String SINGLE_LOOKUP_NAME = "lookupName"; - private static final Map SINGLE_LOOKUP_SPEC = ImmutableMap.of( - "some property", - "some value" + private static final LookupExtractorFactoryMapContainer SINGLE_LOOKUP_SPEC_V0 = + new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("k0", "v0") ); - private static final Map> SINGLE_LOOKUP_MAP = ImmutableMap.>of( + private static final LookupExtractorFactoryMapContainer SINGLE_LOOKUP_SPEC_V1 = + new LookupExtractorFactoryMapContainer( + "v1", + ImmutableMap.of("k1", "v1") + ); + private static final Map SINGLE_LOOKUP_MAP_V0 = ImmutableMap.of( + SINGLE_LOOKUP_NAME, + SINGLE_LOOKUP_SPEC_V0 + ); + private static final Map SINGLE_LOOKUP_MAP_V1 = ImmutableMap.of( SINGLE_LOOKUP_NAME, - SINGLE_LOOKUP_SPEC + SINGLE_LOOKUP_SPEC_V1 ); - private static final Map>> TIERED_LOOKUP_MAP = (Map>>) ImmutableMap.>>of( + private static final Map> TIERED_LOOKUP_MAP_V0 = ImmutableMap.of( LOOKUP_TIER, - SINGLE_LOOKUP_MAP + SINGLE_LOOKUP_MAP_V0 + ); + private static final Map> TIERED_LOOKUP_MAP_V1 = ImmutableMap.of( + LOOKUP_TIER, + SINGLE_LOOKUP_MAP_V1 + ); + private static final Map> EMPTY_TIERED_LOOKUP = ImmutableMap.of(); + private static final LookupsState LOOKUPS_STATE = new LookupsState<>( + SINGLE_LOOKUP_MAP_V0, + SINGLE_LOOKUP_MAP_V1, + Collections.EMPTY_SET ); - private static final Map>> EMPTY_TIERED_LOOKUP = (Map>>) ImmutableMap.>>of(); + private static final AtomicLong EVENT_EMITS = new AtomicLong(0L); private static ServiceEmitter SERVICE_EMITTER; @@ -116,6 +136,27 @@ public void setUp() throws IOException { SERVICE_EMITTER.flush(); EVENT_EMITS.set(0L); + + EasyMock.reset(configManager); + EasyMock.expect( + configManager.watch( + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.anyObject(), + EasyMock.isNull() + ) + ).andReturn( + new AtomicReference<>(null) + ).anyTimes(); + EasyMock.expect( + configManager.watch( + EasyMock.eq(LookupCoordinatorManager.OLD_LOOKUP_CONFIG_KEY), + EasyMock.anyObject(), + EasyMock.isNull() + ) + ).andReturn( + new AtomicReference<>(null) + ).anyTimes(); + EasyMock.replay(configManager); } @After @@ -126,18 +167,20 @@ public void tearDown() throws IOException } @Test - public void testUpdateAllOnHost() throws Exception + public void testUpdateNodeWithSuccess() throws Exception { final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - final URL url = LookupCoordinatorManager.getLookupsURL(HostAndPort.fromString("localhost")); final SettableFuture future = SettableFuture.create(); - future.set(new ByteArrayInputStream(StringUtils.toUtf8(mapper.writeValueAsString(ImmutableMap.of( - "status", - "accepted", - LookupModule.FAILED_UPDATES_KEY, - ImmutableMap.of() - ))))); + future.set( + new ByteArrayInputStream( + StringUtils.toUtf8( + mapper.writeValueAsString( + LOOKUPS_STATE + ) + ) + ) + ); EasyMock.expect(client.go( EasyMock.anyObject(), EasyMock.anyObject(), @@ -146,473 +189,344 @@ public void testUpdateAllOnHost() throws Exception EasyMock.replay(client, responseHandler); - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - HttpResponseHandler makeResponseHandler( - final AtomicInteger returnCode, - final AtomicReference reasonString - ) - { - returnCode.set(200); - reasonString.set(""); - return responseHandler; - } - }; - manager.updateAllOnHost( - url, - SINGLE_LOOKUP_MAP + final LookupCoordinatorManager.LookupsCommunicator lookupsCommunicator = + new LookupCoordinatorManager.LookupsCommunicator(client, lookupCoordinatorManagerConfig, mapper) + { + @Override + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) + { + returnCode.set(Response.Status.ACCEPTED.getStatusCode()); + reasonString.set(""); + return responseHandler; + } + }; + + LookupsState resp = lookupsCommunicator.updateNode( + HostAndPort.fromString("localhost"), + LOOKUPS_STATE ); EasyMock.verify(client, responseHandler); + Assert.assertEquals(resp, LOOKUPS_STATE); } @Test - public void testUpdateAllOnHostFailsWithFailedThings() throws Exception + public void testUpdateNodeRespondedWithNotOkErrorCode() throws Exception { final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - final String failedLookup = "failedLookup"; - final URL url = LookupCoordinatorManager.getLookupsURL(HostAndPort.fromString("localhost")); final SettableFuture future = SettableFuture.create(); - future.set(new ByteArrayInputStream(StringUtils.toUtf8(mapper.writeValueAsString(ImmutableMap.of( - "status", - "accepted", - LookupModule.FAILED_UPDATES_KEY, - ImmutableMap.of( - failedLookup, - ImmutableMap.of() - ) - ))))); + future.set(new ByteArrayInputStream("server failed".getBytes())); EasyMock.expect(client.go( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - )).andReturn(future).once(); + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + )).andReturn(future).once(); EasyMock.replay(client, responseHandler); - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - HttpResponseHandler makeResponseHandler( - final AtomicInteger returnCode, - final AtomicReference reasonString - ) - { - returnCode.set(200); - reasonString.set(""); - return responseHandler; - } - }; - expectedException.expectMessage("Lookups failed to update: [\"" + failedLookup + "\"]"); + final LookupCoordinatorManager.LookupsCommunicator lookupsCommunicator = + new LookupCoordinatorManager.LookupsCommunicator(client, lookupCoordinatorManagerConfig, mapper) + { + @Override + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) + { + returnCode.set(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode()); + reasonString.set(""); + return responseHandler; + } + }; + try { - manager.updateAllOnHost( - url, - SINGLE_LOOKUP_MAP + lookupsCommunicator.updateNode( + HostAndPort.fromString("localhost"), + LOOKUPS_STATE ); + Assert.fail(); } - finally { - - EasyMock.verify(client, responseHandler); + catch (IOException ex) { } + + EasyMock.verify(client, responseHandler); } @Test - public void testUpdateAllOnHostFailsWhenServerReturnsWeird() throws Exception + public void testUpdateNodeReturnsWeird() throws Exception { final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - final String failedLookup = "failedLookup"; - final URL url = LookupCoordinatorManager.getLookupsURL(HostAndPort.fromString("localhost")); final SettableFuture future = SettableFuture.create(); - future.set(new ByteArrayInputStream(StringUtils.toUtf8(mapper.writeValueAsString(ImmutableMap.of( - "status", - "accepted" - ))))); + future.set(new ByteArrayInputStream("weird".getBytes())); EasyMock.expect(client.go( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - )).andReturn(future).once(); + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + )).andReturn(future).once(); EasyMock.replay(client, responseHandler); - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - HttpResponseHandler makeResponseHandler( - final AtomicInteger returnCode, - final AtomicReference reasonString - ) - { - returnCode.set(200); - reasonString.set(""); - return responseHandler; - } - }; - expectedException.expectMessage(String.format( - "Update result did not have field for [%s]", - LookupModule.FAILED_UPDATES_KEY - )); + final LookupCoordinatorManager.LookupsCommunicator lookupsCommunicator = + new LookupCoordinatorManager.LookupsCommunicator(client, lookupCoordinatorManagerConfig, mapper) + { + @Override + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) + { + returnCode.set(Response.Status.ACCEPTED.getStatusCode()); + reasonString.set(""); + return responseHandler; + } + }; + try { - manager.updateAllOnHost( - url, - SINGLE_LOOKUP_MAP + lookupsCommunicator.updateNode( + HostAndPort.fromString("localhost"), + LOOKUPS_STATE ); + Assert.fail(); } - finally { - - EasyMock.verify(client, responseHandler); + catch (IOException ex) { } - } + EasyMock.verify(client, responseHandler); + } @Test - public void testUpdateAllOnHostException() throws Exception + public void testUpdateNodeInterrupted() throws Exception { final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - final URL url = LookupCoordinatorManager.getLookupsURL(HostAndPort.fromString("localhost")); final SettableFuture future = SettableFuture.create(); - future.set(new ByteArrayInputStream(new byte[0])); EasyMock.expect(client.go( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - )).andReturn(future).once(); + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + )).andReturn(future).once(); EasyMock.replay(client, responseHandler); - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - HttpResponseHandler makeResponseHandler( - final AtomicInteger returnCode, - final AtomicReference reasonString - ) - { - returnCode.set(500); - reasonString.set(""); - return responseHandler; - } - }; - expectedException.expect(new BaseMatcher() - { - @Override - public boolean matches(Object o) - { - return o instanceof IOException && ((IOException) o).getMessage().startsWith("Bad update request"); - } - - @Override - public void describeTo(Description description) - { + final LookupCoordinatorManager.LookupsCommunicator lookupsCommunicator = + new LookupCoordinatorManager.LookupsCommunicator(client, lookupCoordinatorManagerConfig, mapper) + { + @Override + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) + { + returnCode.set(Response.Status.ACCEPTED.getStatusCode()); + reasonString.set(""); + return responseHandler; + } + }; - } - }); + Thread.currentThread().interrupt(); try { - manager.updateAllOnHost( - url, - SINGLE_LOOKUP_MAP + lookupsCommunicator.updateNode( + HostAndPort.fromString("localhost"), + LOOKUPS_STATE ); + Assert.fail(); + } + catch (InterruptedException ex) { } finally { - EasyMock.verify(client, responseHandler); + //clear the interrupt + Thread.interrupted(); } + + EasyMock.verify(client, responseHandler); } + @Test - public void testParseErrorUpdateAllOnHost() throws Exception + public void testGetLookupsStateNodeWithSuccess() throws Exception { + final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - final AtomicReference>> configVal = new AtomicReference<>(null); + final SettableFuture future = SettableFuture.create(); + future.set( + new ByteArrayInputStream( + StringUtils.toUtf8( + mapper.writeValueAsString( + LOOKUPS_STATE + ) + ) + ) + ); + EasyMock.expect(client.go( + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + )).andReturn(future).once(); - final URL url = LookupCoordinatorManager.getLookupsURL(HostAndPort.fromString("localhost")); + EasyMock.replay(client, responseHandler); - EasyMock.reset(configManager); - EasyMock.expect(configManager.watch(EasyMock.anyString(), EasyMock.anyObject())) - .andReturn(configVal); + final LookupCoordinatorManager.LookupsCommunicator lookupsCommunicator = + new LookupCoordinatorManager.LookupsCommunicator(client, lookupCoordinatorManagerConfig, mapper) + { + @Override + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) + { + returnCode.set(Response.Status.OK.getStatusCode()); + reasonString.set(""); + return responseHandler; + } + }; - final JsonProcessingException ex = EasyMock.createStrictMock(JsonProcessingException.class); + LookupsState resp = lookupsCommunicator.getLookupStateForNode( + HostAndPort.fromString("localhost") + ); - final ObjectMapper mapper = EasyMock.createStrictMock(ObjectMapper.class); - EasyMock.expect(mapper.writeValueAsBytes(EasyMock.eq(SINGLE_LOOKUP_MAP))).andThrow(ex); + EasyMock.verify(client, responseHandler); + Assert.assertEquals(resp, LOOKUPS_STATE); + } - expectedException.expectCause(new BaseMatcher() - { - @Override - public boolean matches(Object o) - { - return ex == o; - } - @Override - public void describeTo(Description description) - { + @Test + public void testGetLookupsStateNodeRespondedWithNotOkErrorCode() throws Exception + { + final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - } - }); + final SettableFuture future = SettableFuture.create(); + future.set(new ByteArrayInputStream("server failed".getBytes())); + EasyMock.expect(client.go( + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + )).andReturn(future).once(); - EasyMock.replay(mapper); + EasyMock.replay(client, responseHandler); + + final LookupCoordinatorManager.LookupsCommunicator lookupsCommunicator = + new LookupCoordinatorManager.LookupsCommunicator(client, lookupCoordinatorManagerConfig, mapper) + { + @Override + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) + { + returnCode.set(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode()); + reasonString.set(""); + return responseHandler; + } + }; - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ); try { - manager.updateAllOnHost( - url, - SINGLE_LOOKUP_MAP + lookupsCommunicator.getLookupStateForNode( + HostAndPort.fromString("localhost") ); + Assert.fail(); } - finally { - EasyMock.verify(mapper); + catch (IOException ex) { } - } - - - @Test - public void testUpdateAll() throws Exception - { - final List urls = ImmutableList.of(new URL("http://foo.bar")); - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - Collection getAllHostsAnnounceEndpoint(String tier) - { - return urls; - } - @Override - void updateAllOnHost(final URL url, Map> updatedLookups) - throws IOException, InterruptedException, ExecutionException - { - if (!urls.get(0).equals(url) || updatedLookups != SINGLE_LOOKUP_MAP) { - throw new RuntimeException("Not matched"); - } - } - }; - // Should be no-ops - manager.updateAllOnTier(null, null); + EasyMock.verify(client, responseHandler); } - @Test - public void testUpdateAllIOException() throws Exception + public void testGetLookupsStateNodeReturnsWeird() throws Exception { - final IOException ex = new IOException("test exception"); - final List urls = ImmutableList.of(new URL("http://foo.bar")); - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - Collection getAllHostsAnnounceEndpoint(String Tier) - { - return urls; - } - - @Override - void updateAllOnHost(final URL url, Map> updatedLookups) - throws IOException, InterruptedException, ExecutionException - { - throw ex; - } - }; - // Should log and pass io exception - manager.updateAllOnTier(LOOKUP_TIER, SINGLE_LOOKUP_MAP); - SERVICE_EMITTER.flush(); - Assert.assertEquals(1, EVENT_EMITS.get()); - EVENT_EMITS.set(0); - } + final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - @Test - public void testUpdateAllInterrupted() throws Exception - { - final InterruptedException ex = new InterruptedException("interruption test"); - final List urls = ImmutableList.of(new URL("http://foo.bar")); - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - Collection getAllHostsAnnounceEndpoint(String tier) - { - return urls; - } + final SettableFuture future = SettableFuture.create(); + future.set(new ByteArrayInputStream("weird".getBytes())); + EasyMock.expect(client.go( + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + )).andReturn(future).once(); - @Override - void updateAllOnHost(final URL url, Map> knownLookups) - throws IOException, InterruptedException, ExecutionException - { - throw ex; - } - }; - expectedException.expectCause(new BaseMatcher() - { - @Override - public boolean matches(Object o) - { - if (!(o instanceof RuntimeException)) { - return false; - } - final Throwable e = (Throwable) o; - return e.getCause() == ex; - } + EasyMock.replay(client, responseHandler); - @Override - public void describeTo(Description description) - { + final LookupCoordinatorManager.LookupsCommunicator lookupsCommunicator = + new LookupCoordinatorManager.LookupsCommunicator(client, lookupCoordinatorManagerConfig, mapper) + { + @Override + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) + { + returnCode.set(Response.Status.ACCEPTED.getStatusCode()); + reasonString.set(""); + return responseHandler; + } + }; - } - }); try { - manager.updateAllOnTier(LOOKUP_TIER, SINGLE_LOOKUP_MAP); + lookupsCommunicator.getLookupStateForNode( + HostAndPort.fromString("localhost") + ); + Assert.fail(); } - finally { - // Clear status - Thread.interrupted(); + catch (IOException ex) { } + + EasyMock.verify(client, responseHandler); } @Test - public void testGetAllHostsAnnounceEndpoint() throws Exception + public void testGetLookupsStateNodeInterrupted() throws Exception { - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ); + final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - EasyMock.expect(discoverer.getNodes(EasyMock.eq(LookupModule.getTierListenerPath(LOOKUP_TIER)))) - .andReturn(ImmutableList.of()) - .once(); - EasyMock.replay(discoverer); - Assert.assertEquals(ImmutableList.of(), manager.getAllHostsAnnounceEndpoint(LOOKUP_TIER)); - EasyMock.verify(discoverer); - EasyMock.reset(discoverer); + final SettableFuture future = SettableFuture.create(); + EasyMock.expect( + client.go( + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + )).andReturn(future).once(); - EasyMock.expect(discoverer.getNodes(EasyMock.eq(LookupModule.getTierListenerPath(LOOKUP_TIER)))) - .andReturn(Collections.singletonList(null)) - .once(); - EasyMock.replay(discoverer); - Assert.assertEquals(ImmutableList.of(), manager.getAllHostsAnnounceEndpoint(LOOKUP_TIER)); - EasyMock.verify(discoverer); - } + EasyMock.replay(client, responseHandler); - @Test - public void testGetLookupURL() throws Exception - { - final String path = ListenerResource.BASE_PATH + "/" + LookupCoordinatorManager.LOOKUP_LISTEN_ANNOUNCE_KEY; - Assert.assertEquals( - new URL("http", "someHost", 1, path), - LookupCoordinatorManager.getLookupsURL( - HostAndPort.fromParts("someHost", 1) - ) - ); - Assert.assertEquals( - new URL("http", "someHost", -1, path), - LookupCoordinatorManager.getLookupsURL( - HostAndPort.fromString("someHost") - ) - ); + final LookupCoordinatorManager.LookupsCommunicator lookupsCommunicator = + new LookupCoordinatorManager.LookupsCommunicator(client, lookupCoordinatorManagerConfig, mapper) + { + @Override + HttpResponseHandler makeResponseHandler( + final AtomicInteger returnCode, + final AtomicReference reasonString + ) + { + returnCode.set(Response.Status.ACCEPTED.getStatusCode()); + reasonString.set(""); + return responseHandler; + } + }; - Assert.assertEquals( - new URL("http", "::1", -1, path), - LookupCoordinatorManager.getLookupsURL( - HostAndPort.fromString("::1") - ) - ); - Assert.assertEquals( - new URL("http", "[::1]", -1, path), - LookupCoordinatorManager.getLookupsURL( - HostAndPort.fromString("[::1]") - ) - ); - Assert.assertEquals( - new URL("http", "::1", -1, path), - LookupCoordinatorManager.getLookupsURL( - HostAndPort.fromString("::1") - ) - ); - } + Thread.currentThread().interrupt(); + try { + lookupsCommunicator.getLookupStateForNode( + HostAndPort.fromString("localhost") + ); + Assert.fail(); + } + catch (InterruptedException ex) { + } + finally { + //clear the interrupt + Thread.interrupted(); + } - @Test - public void testUpdateLookupAdds() throws Exception - { - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - public Map>> getKnownLookups() - { - return EMPTY_TIERED_LOOKUP; - } - }; - final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); - EasyMock.reset(configManager); - EasyMock.expect(configManager.set( - EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), - EasyMock.eq(TIERED_LOOKUP_MAP), - EasyMock.eq(auditInfo) - )).andReturn(true).once(); - EasyMock.replay(configManager); - manager.updateLookup(LOOKUP_TIER, SINGLE_LOOKUP_NAME, SINGLE_LOOKUP_SPEC, auditInfo); - EasyMock.verify(configManager); + EasyMock.verify(client, responseHandler); } - @Test - public void testUpdateLookupFailsUnitialized() throws Exception + public void testUpdateLookupsFailsUnitialized() throws Exception { final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, @@ -623,20 +537,20 @@ public void testUpdateLookupFailsUnitialized() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { return null; } }; + manager.start(); final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); expectedException.expect(ISE.class); - manager.updateLookups(TIERED_LOOKUP_MAP, auditInfo); + manager.updateLookups(TIERED_LOOKUP_MAP_V0, auditInfo); } @Test - public void testUpdateLookupUpdates() throws Exception + public void testUpdateLookupsInitialization() throws Exception { - final Map ignore = ImmutableMap.of("prop", "old"); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -646,46 +560,29 @@ public void testUpdateLookupUpdates() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return ImmutableMap.>>of(LOOKUP_TIER, ImmutableMap.of( - "foo", ImmutableMap.of("prop", "old"), - "ignore", ignore - )); + return null; } }; - final Map newSpec = ImmutableMap.of( - "prop", - "new" - ); - final Map> lookup = ImmutableMap.>of( - "foo", newSpec, - "ignore", ignore - ); - final Map>> tier = ImmutableMap.of( - LOOKUP_TIER, - lookup - ); + manager.start(); + final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); EasyMock.reset(configManager); - EasyMock.expect(configManager.set( - EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), - EasyMock.eq(ImmutableMap.of(LOOKUP_TIER, ImmutableMap.of( - "foo", newSpec, - "ignore", ignore - ))), - EasyMock.eq(auditInfo) - )).andReturn(true).once(); + EasyMock.expect( + configManager.set( + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.eq(EMPTY_TIERED_LOOKUP), + EasyMock.eq(auditInfo) + )).andReturn(true).once(); EasyMock.replay(configManager); - manager.updateLookups(tier, auditInfo); + manager.updateLookups(EMPTY_TIERED_LOOKUP, auditInfo); EasyMock.verify(configManager); } - @Test - public void testUpdateLookupFailsBadUpdates() throws Exception + public void testUpdateLookupAdds() throws Exception { - final Map ignore = ImmutableMap.of("prop", "old"); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -695,94 +592,33 @@ public void testUpdateLookupFailsBadUpdates() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return ImmutableMap.>>of(LOOKUP_TIER, ImmutableMap.of( - "foo", ImmutableMap.of("prop", "old"), - "ignore", ignore - )); + return EMPTY_TIERED_LOOKUP; } }; - final Map newSpec = ImmutableMap.of( - "prop", - "new" - ); - final Map> lookup = ImmutableMap.>of( - "foo", newSpec, - "ignore", ignore - ); - final Map>> tier = ImmutableMap.of( - LOOKUP_TIER, - lookup - ); + manager.start(); + final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); EasyMock.reset(configManager); EasyMock.expect(configManager.set( - EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), - EasyMock.eq(ImmutableMap.of(LOOKUP_TIER, ImmutableMap.of( - "foo", newSpec, - "ignore", ignore - ))), - EasyMock.eq(auditInfo) - )).andReturn(false).once(); + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.eq(TIERED_LOOKUP_MAP_V0), + EasyMock.eq(auditInfo) + )).andReturn(true).once(); EasyMock.replay(configManager); - Assert.assertFalse(manager.updateLookups(tier, auditInfo)); + manager.updateLookup(LOOKUP_TIER, SINGLE_LOOKUP_NAME, SINGLE_LOOKUP_SPEC_V0, auditInfo); EasyMock.verify(configManager); } @Test - public void testUpdateLookupsOnlyAddsToTier() throws Exception + public void testUpdateLookupsAddsNewLookup() throws Exception { - final Map ignore = ImmutableMap.of("prop", "old"); - final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); - final LookupCoordinatorManager manager = new LookupCoordinatorManager( - client, - discoverer, - mapper, - configManager, - lookupCoordinatorManagerConfig - ) - { - @Override - public Map>> getKnownLookups() - { - return ImmutableMap.>>of( - LOOKUP_TIER + "1", - ImmutableMap.>of("foo", ImmutableMap.of("prop", "old")), - LOOKUP_TIER + "2", - ImmutableMap.of("ignore", ignore) - ); - } - }; - final Map newSpec = ImmutableMap.of( - "prop", - "new" + final LookupExtractorFactoryMapContainer ignore = new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("prop", "old") ); - EasyMock.reset(configManager); - EasyMock.expect( - configManager.set( - EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), - EasyMock.eq(ImmutableMap.>>of( - LOOKUP_TIER + "1", ImmutableMap.of("foo", newSpec), - LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) - )), - EasyMock.eq(auditInfo) - ) - ).andReturn(true).once(); - EasyMock.replay(configManager); - Assert.assertTrue(manager.updateLookups(ImmutableMap.>>of( - LOOKUP_TIER + "1", ImmutableMap.>of( - "foo", - newSpec - ) - ), auditInfo)); - EasyMock.verify(configManager); - } - @Test - public void testUpdateLookupsAddsNewTier() throws Exception - { - final Map ignore = ImmutableMap.of("prop", "old"); final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, @@ -793,43 +629,58 @@ public void testUpdateLookupsAddsNewTier() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return ImmutableMap.>>of( + return ImmutableMap.>of( + LOOKUP_TIER + "1", + ImmutableMap.of( + "foo1", new LookupExtractorFactoryMapContainer( + "v0", ImmutableMap.of("prop", "old") + ) + ), LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) ); } }; - final Map newSpec = ImmutableMap.of( - "prop", - "new" + manager.start(); + final LookupExtractorFactoryMapContainer newSpec = new LookupExtractorFactoryMapContainer( + "v1", + ImmutableMap.of("prop", "new") ); EasyMock.reset(configManager); EasyMock.expect( configManager.set( EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), - EasyMock.eq(ImmutableMap.>>of( - LOOKUP_TIER + "1", ImmutableMap.of("foo", newSpec), - LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) - )), + EasyMock.eq(ImmutableMap.>of( + LOOKUP_TIER + "1", ImmutableMap.of( + "foo1", ignore, + "foo2", newSpec + ), + LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) + )), EasyMock.eq(auditInfo) ) ).andReturn(true).once(); EasyMock.replay(configManager); - Assert.assertTrue(manager.updateLookups(ImmutableMap.>>of( - LOOKUP_TIER + "1", ImmutableMap.>of( - "foo", - newSpec - ) - ), auditInfo)); + Assert.assertTrue( + manager.updateLookups( + ImmutableMap.>of( + LOOKUP_TIER + "1", ImmutableMap.of( + "foo2", + newSpec + ) + ), auditInfo)); EasyMock.verify(configManager); } @Test - public void testUpdateLookupsAddsNewLookup() throws Exception + public void testUpdateLookupsOnlyUpdatesToTier() throws Exception { - final Map ignore = ImmutableMap.of("prop", "old"); + final LookupExtractorFactoryMapContainer ignore = new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("prop", "old") + ); final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, @@ -840,49 +691,52 @@ public void testUpdateLookupsAddsNewLookup() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return ImmutableMap.>>of( + return ImmutableMap.>of( LOOKUP_TIER + "1", - ImmutableMap.>of("foo1", ImmutableMap.of("prop", "old")), + ImmutableMap.of( + "foo", new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("prop", "new") + ) + ), LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) ); } }; - final Map newSpec = ImmutableMap.of( - "prop", - "new" + manager.start(); + final LookupExtractorFactoryMapContainer newSpec = new LookupExtractorFactoryMapContainer( + "v1", + ImmutableMap.of("prop", "new") ); EasyMock.reset(configManager); EasyMock.expect( configManager.set( EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), - EasyMock.eq(ImmutableMap.>>of( - LOOKUP_TIER + "1", ImmutableMap.of( - "foo1", ImmutableMap.of("prop", "old"), - "foo2", newSpec - ), - LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) - )), + EasyMock.eq(ImmutableMap.>of( + LOOKUP_TIER + "1", ImmutableMap.of("foo", newSpec), + LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) + )), EasyMock.eq(auditInfo) ) ).andReturn(true).once(); EasyMock.replay(configManager); - Assert.assertTrue(manager.updateLookups(ImmutableMap.>>of( - LOOKUP_TIER + "1", ImmutableMap.>of( - "foo2", - newSpec - ) - ), auditInfo)); + Assert.assertTrue( + manager.updateLookups( + ImmutableMap.>of( + LOOKUP_TIER + "1", ImmutableMap.of( + "foo", + newSpec + ) + ), auditInfo)); EasyMock.verify(configManager); } @Test - public void testDeleteLookup() throws Exception + public void testUpdateLookupsUpdates() throws Exception { - final Map ignore = ImmutableMap.of("lookup", "ignore"); - final Map lookup = ImmutableMap.of("lookup", "foo"); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -892,32 +746,27 @@ public void testDeleteLookup() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return ImmutableMap.>>of(LOOKUP_TIER, ImmutableMap.of( - "foo", lookup, - "ignore", ignore - )); + return TIERED_LOOKUP_MAP_V0; } }; + manager.start(); final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); EasyMock.reset(configManager); EasyMock.expect(configManager.set( - EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), - EasyMock.eq(ImmutableMap.of(LOOKUP_TIER, ImmutableMap.of( - "ignore", ignore - ))), - EasyMock.eq(auditInfo) - )).andReturn(true).once(); + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.eq(TIERED_LOOKUP_MAP_V1), + EasyMock.eq(auditInfo) + )).andReturn(true).once(); EasyMock.replay(configManager); - Assert.assertTrue(manager.deleteLookup(LOOKUP_TIER, "foo", auditInfo)); + manager.updateLookups(TIERED_LOOKUP_MAP_V1, auditInfo); EasyMock.verify(configManager); } @Test - public void testDeleteLookupIgnoresMissing() throws Exception + public void testUpdateLookupFailsSameVersionUpdates() throws Exception { - final Map ignore = ImmutableMap.of("lookup", "ignore"); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -927,20 +776,30 @@ public void testDeleteLookupIgnoresMissing() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return ImmutableMap.>>of(LOOKUP_TIER, ImmutableMap.of( - "ignore", ignore - )); + return TIERED_LOOKUP_MAP_V0; } }; + manager.start(); final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); - Assert.assertFalse(manager.deleteLookup(LOOKUP_TIER, "foo", auditInfo)); + + try { + manager.updateLookups(TIERED_LOOKUP_MAP_V0, auditInfo); + Assert.fail(); + } catch (IAE ex) { + } } @Test - public void testDeleteLookupIgnoresNotReady() throws Exception + public void testUpdateLookupsAddsNewTier() throws Exception { + final LookupExtractorFactoryMapContainer ignore = new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("prop", "old") + ); + + final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -950,19 +809,52 @@ public void testDeleteLookupIgnoresNotReady() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return null; + return ImmutableMap.>of( + LOOKUP_TIER + "2", + ImmutableMap.of("ignore", ignore) + ); } }; - final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); - Assert.assertFalse(manager.deleteLookup(LOOKUP_TIER, "foo", auditInfo)); + manager.start(); + final LookupExtractorFactoryMapContainer newSpec = new LookupExtractorFactoryMapContainer( + "v1", + ImmutableMap.of("prop", "new") + ); + EasyMock.reset(configManager); + EasyMock.expect( + configManager.set( + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.eq(ImmutableMap.>of( + LOOKUP_TIER + "1", ImmutableMap.of("foo", newSpec), + LOOKUP_TIER + "2", ImmutableMap.of("ignore", ignore) + )), + EasyMock.eq(auditInfo) + ) + ).andReturn(true).once(); + EasyMock.replay(configManager); + Assert.assertTrue(manager.updateLookups(ImmutableMap.>of( + LOOKUP_TIER + "1", ImmutableMap.of( + "foo", + newSpec + ) + ), auditInfo)); + EasyMock.verify(configManager); } @Test - public void testDeleteAllTier() throws Exception + public void testDeleteLookup() throws Exception { - final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); + final LookupExtractorFactoryMapContainer ignore = new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("lookup", "ignore") + ); + + final LookupExtractorFactoryMapContainer lookup = new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("lookup", "foo") + ); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -971,43 +863,43 @@ public void testDeleteAllTier() throws Exception lookupCoordinatorManagerConfig ) { - @Override - HttpResponseHandler makeResponseHandler( - final AtomicInteger returnCode, - final AtomicReference reasonString - ) + public Map> getKnownLookups() { - returnCode.set(200); - reasonString.set(""); - return responseHandler; + return ImmutableMap.>of(LOOKUP_TIER, ImmutableMap.of( + "foo", lookup, + "ignore", ignore + )); } }; - final HostAndPort hostAndPort = HostAndPort.fromParts("someHost", 8080); - final Collection drop = ImmutableList.of("lookup1"); - EasyMock - .expect(discoverer.getNodes(LookupModule.getTierListenerPath(LOOKUP_TIER))) - .andReturn(ImmutableList.of(hostAndPort)) - .once(); - final SettableFuture future = SettableFuture.create(); - future.set(new ByteArrayInputStream(new byte[0])); - EasyMock - .expect(client.go( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - )) - .andReturn(future) - .once(); - EasyMock.replay(client, discoverer, responseHandler); - manager.deleteAllOnTier(LOOKUP_TIER, drop); - EasyMock.verify(client, discoverer, responseHandler); + manager.start(); + final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); + EasyMock.reset(configManager); + EasyMock.expect( + configManager.set( + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.eq( + ImmutableMap.of( + LOOKUP_TIER, ImmutableMap.of( + "ignore", ignore + ) + ) + ), + EasyMock.eq(auditInfo) + ) + ).andReturn(true).once(); + EasyMock.replay(configManager); + Assert.assertTrue(manager.deleteLookup(LOOKUP_TIER, "foo", auditInfo)); + EasyMock.verify(configManager); } @Test - public void testDeleteAllTierMissing() throws Exception + public void testDeleteLookupIgnoresMissing() throws Exception { - final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); + final LookupExtractorFactoryMapContainer ignore = new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("lookup", "ignore") + ); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -1017,42 +909,22 @@ public void testDeleteAllTierMissing() throws Exception ) { @Override - HttpResponseHandler makeResponseHandler( - final AtomicInteger returnCode, - final AtomicReference reasonString - ) + public Map> getKnownLookups() { - returnCode.set(404); - reasonString.set(""); - return responseHandler; + return ImmutableMap.>of( + LOOKUP_TIER, + ImmutableMap.of("ignore", ignore) + ); } }; - final HostAndPort hostAndPort = HostAndPort.fromParts("someHost", 8080); - final Collection drop = ImmutableList.of("lookup1"); - EasyMock - .expect(discoverer.getNodes(LookupModule.getTierListenerPath(LOOKUP_TIER))) - .andReturn(ImmutableList.of(hostAndPort)) - .once(); - final SettableFuture future = SettableFuture.create(); - future.set(new ByteArrayInputStream(new byte[0])); - EasyMock - .expect(client.go( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - )) - .andReturn(future) - .once(); - EasyMock.replay(client, discoverer, responseHandler); - manager.deleteAllOnTier(LOOKUP_TIER, drop); - EasyMock.verify(client, discoverer, responseHandler); + manager.start(); + final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); + Assert.assertFalse(manager.deleteLookup(LOOKUP_TIER, "foo", auditInfo)); } @Test - public void testDeleteAllTierContinuesOnMissing() throws Exception + public void testDeleteLookupIgnoresNotReady() throws Exception { - final HttpResponseHandler responseHandler = EasyMock.createStrictMock(HttpResponseHandler.class); - final AtomicInteger responseHandlerCalls = new AtomicInteger(0); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -1062,47 +934,23 @@ public void testDeleteAllTierContinuesOnMissing() throws Exception ) { @Override - HttpResponseHandler makeResponseHandler( - final AtomicInteger returnCode, - final AtomicReference reasonString - ) + public Map> getKnownLookups() { - if (responseHandlerCalls.getAndIncrement() == 0) { - returnCode.set(404); - reasonString.set("Not Found"); - } else { - returnCode.set(202); - reasonString.set(""); - } - return responseHandler; + return null; } }; - final HostAndPort hostAndPort = HostAndPort.fromParts("someHost", 8080); - final Collection drop = ImmutableList.of("lookup1"); - EasyMock - .expect(discoverer.getNodes(LookupModule.getTierListenerPath(LOOKUP_TIER))) - .andReturn(ImmutableList.of(hostAndPort, hostAndPort)) - .once(); - final SettableFuture future = SettableFuture.create(); - future.set(new ByteArrayInputStream(new byte[0])); - EasyMock - .expect(client.go( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - )) - .andReturn(future) - .times(2); - EasyMock.replay(client, discoverer, responseHandler); - manager.deleteAllOnTier(LOOKUP_TIER, drop); - EasyMock.verify(client, discoverer, responseHandler); - Assert.assertEquals(2, responseHandlerCalls.get()); + manager.start(); + final AuditInfo auditInfo = new AuditInfo("author", "comment", "localhost"); + Assert.assertFalse(manager.deleteLookup(LOOKUP_TIER, "foo", auditInfo)); } @Test public void testGetLookup() throws Exception { - final Map lookup = ImmutableMap.of("lookup", "foo"); + final LookupExtractorFactoryMapContainer lookup = new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("lookup", "foo") + ); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -1112,9 +960,9 @@ public void testGetLookup() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return ImmutableMap.>>of(LOOKUP_TIER, ImmutableMap.of( + return ImmutableMap.>of(LOOKUP_TIER, ImmutableMap.of( "foo", lookup )); @@ -1125,11 +973,13 @@ public Map>> getKnownLookups() Assert.assertNull(manager.getLookup("not a tier", "foo")); } - @Test public void testGetLookupIgnoresMalformed() throws Exception { - final Map lookup = ImmutableMap.of("lookup", "foo"); + final LookupExtractorFactoryMapContainer lookup = new LookupExtractorFactoryMapContainer( + "v0", + ImmutableMap.of("lookup", "foo") + ); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, @@ -1139,11 +989,11 @@ public void testGetLookupIgnoresMalformed() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { - return ImmutableMap.>>of(LOOKUP_TIER, ImmutableMap.of( + return ImmutableMap.>of(LOOKUP_TIER, ImmutableMap.of( "foo", lookup, - "bar", ImmutableMap.of() + "bar", new LookupExtractorFactoryMapContainer("v0",ImmutableMap.of()) )); } }; @@ -1164,7 +1014,7 @@ public void testGetLookupIgnoresNotReady() throws Exception ) { @Override - public Map>> getKnownLookups() + public Map> getKnownLookups() { return null; } @@ -1172,50 +1022,221 @@ public Map>> getKnownLookups() Assert.assertNull(manager.getLookup(LOOKUP_TIER, "foo")); } - @Test - public void testStart() throws Exception + + @Test(timeout = 2_000) + public void testLookupManagementLoop() throws Exception { - final AtomicReference>> lookupRef = new AtomicReference<>(null); + Map lookup1 = ImmutableMap.of( + "lookup1", new LookupExtractorFactoryMapContainer( + "v1", ImmutableMap.of("k1", "v1") + ) + ); + Map> configuredLookups = + ImmutableMap.of( + "tier1", + lookup1 + ); EasyMock.reset(configManager); EasyMock.expect(configManager.watch( - EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), - EasyMock.anyObject(), - EasyMock.isNull() - )).andReturn(lookupRef).once(); + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.anyObject(), + EasyMock.isNull() + )).andReturn( + new AtomicReference<>(configuredLookups)).once(); EasyMock.replay(configManager); + HostAndPort host1 = HostAndPort.fromParts("host1", 1234); + HostAndPort host2 = HostAndPort.fromParts("host2", 3456); + + EasyMock.reset(discoverer); + EasyMock.expect( + discoverer.getNodes(LookupModule.getTierListenerPath("tier1")) + ).andReturn(ImmutableList.of(host1, host2)).anyTimes(); + EasyMock.replay(discoverer); + + LookupCoordinatorManager.LookupsCommunicator lookupsCommunicator = EasyMock.createMock(LookupCoordinatorManager.LookupsCommunicator.class); + EasyMock.expect( + lookupsCommunicator.getLookupStateForNode( + host1 + ) + ).andReturn( + new LookupsState<>( + ImmutableMap.of("lookup0", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0"))), null, null + ) + ).once(); + + LookupsState host1UpdatedState = new LookupsState<>( + lookup1, null, null + ); + + EasyMock.expect( + lookupsCommunicator.updateNode( + host1, + new LookupsState<>( + null, + lookup1, + ImmutableSet.of("lookup0") + ) + ) + ).andReturn( + host1UpdatedState + ).once(); + + + EasyMock.expect( + lookupsCommunicator.getLookupStateForNode( + host2 + ) + ).andReturn( + new LookupsState<>( + ImmutableMap.of("lookup3", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0")), + "lookup1", new LookupExtractorFactoryMapContainer("v0", ImmutableMap.of("k0", "v0"))), + null, null + ) + ).once(); + + LookupsState host2UpdatedState = new LookupsState<>( + null, lookup1, null + ); + + EasyMock.expect( + lookupsCommunicator.updateNode( + host2, + new LookupsState<>( + null, + lookup1, + ImmutableSet.of("lookup3") + ) + ) + ).andReturn( + host2UpdatedState + ).once(); + + EasyMock.replay(lookupsCommunicator); + + LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig = new LookupCoordinatorManagerConfig() + { + public long getInitialDelay() + { + return 1; + } + + public int getThreadPoolSize() + { + return 2; + } + }; + final LookupCoordinatorManager manager = new LookupCoordinatorManager( + discoverer, + configManager, + lookupCoordinatorManagerConfig, + lookupsCommunicator + ); + + Assert.assertNull(manager.knownOldState.get()); + + manager.start(); + + Map> expectedKnownState = ImmutableMap.of( + host1, + host1UpdatedState, + host2, + host2UpdatedState + ); + + while (!expectedKnownState.equals(manager.knownOldState.get())) { + Thread.sleep(100); + } + + EasyMock.verify(discoverer, configManager, lookupsCommunicator); + } + + @Test + public void testGetToBeLoadedOnNode() + { + LookupCoordinatorManager manager = new LookupCoordinatorManager( client, discoverer, mapper, configManager, - new LookupCoordinatorManagerConfig(){ - @Override - public long getPeriod(){ - return 1; - } - } + lookupCoordinatorManagerConfig + ); + + LookupsState currNodeState = new LookupsState<>( + ImmutableMap.of("lookup0", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0")), + "lookup1", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k1", "v1")) + ), + ImmutableMap.of("lookup2", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k2", "v2")), + "lookup3", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k3", "v3")) + ), + ImmutableSet.of("lookup2", "lookup4") + ); + + Map stateToBe = ImmutableMap.of( + "lookup0", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0")), + "lookup1", new LookupExtractorFactoryMapContainer("v2", ImmutableMap.of("k1", "v1")), + "lookup2", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k2", "v2")) + ); + + Assert.assertEquals( + ImmutableMap.of( + "lookup1", new LookupExtractorFactoryMapContainer("v2", ImmutableMap.of("k1", "v1")), + "lookup2", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k2", "v2")) + ), + manager.getToBeLoadedOnNode(currNodeState, stateToBe) ); - manager.start(); - manager.start(); - Assert.assertTrue(manager.backgroundManagerIsRunning()); - Assert.assertNull(manager.getKnownLookups()); - Assert.assertFalse(manager.waitForBackgroundTermination(10)); - EasyMock.verify(configManager); } @Test - public void testStop() throws Exception + public void testToBeDropped() { - final AtomicReference>> lookupRef = new AtomicReference<>(null); + LookupCoordinatorManager manager = new LookupCoordinatorManager( + client, + discoverer, + mapper, + configManager, + lookupCoordinatorManagerConfig + ); + + LookupsState currNodeState = new LookupsState<>( + ImmutableMap.of("lookup0", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0")), + "lookup1", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k1", "v1")) + ), + ImmutableMap.of("lookup2", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k2", "v2")), + "lookup3", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k3", "v3")) + ), + ImmutableSet.of("lookup2", "lookup4") + ); + + Map stateToBe = ImmutableMap.of( + "lookup0", new LookupExtractorFactoryMapContainer("v1", ImmutableMap.of("k0", "v0")) + ); + + Assert.assertEquals( + ImmutableSet.of("lookup1", "lookup3"), + manager.getToBeDroppedFromNode(currNodeState, stateToBe) + ); + } + @Test + public void testStartStop() throws Exception + { EasyMock.reset(configManager); + + EasyMock.expect(configManager.watch( + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.anyObject(), + EasyMock.isNull() + )).andReturn(new AtomicReference>(null)).once(); + EasyMock.expect(configManager.watch( - EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), - EasyMock.anyObject(), - EasyMock.isNull() - )).andReturn(lookupRef).once(); + EasyMock.eq(LookupCoordinatorManager.OLD_LOOKUP_CONFIG_KEY), + EasyMock.anyObject(), + EasyMock.isNull() + )).andReturn(new AtomicReference>>(null)).once(); + EasyMock.replay(configManager); final LookupCoordinatorManager manager = new LookupCoordinatorManager( @@ -1225,27 +1246,34 @@ public void testStop() throws Exception configManager, lookupCoordinatorManagerConfig ); + + Assert.assertFalse(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + manager.start(); + Assert.assertTrue(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); Assert.assertTrue(manager.backgroundManagerIsRunning()); Assert.assertFalse(manager.waitForBackgroundTermination(10)); + manager.stop(); - manager.stop(); + Assert.assertFalse(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); Assert.assertTrue(manager.waitForBackgroundTermination(10)); Assert.assertFalse(manager.backgroundManagerIsRunning()); + EasyMock.verify(configManager); } @Test - public void testStartTooMuch() throws Exception + public void testMultipleStartStop() throws Exception { - final AtomicReference>> lookupRef = new AtomicReference<>(null); - EasyMock.reset(configManager); + EasyMock.expect(configManager.watch( - EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), - EasyMock.anyObject(), - EasyMock.isNull() - )).andReturn(lookupRef).once(); + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.anyObject(), + EasyMock.isNull() + )).andReturn( + new AtomicReference<>(Collections.EMPTY_MAP)).anyTimes(); + EasyMock.replay(configManager); final LookupCoordinatorManager manager = new LookupCoordinatorManager( @@ -1255,30 +1283,40 @@ public void testStartTooMuch() throws Exception configManager, lookupCoordinatorManagerConfig ); + + Assert.assertFalse(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + manager.start(); + Assert.assertTrue(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); Assert.assertTrue(manager.backgroundManagerIsRunning()); Assert.assertFalse(manager.waitForBackgroundTermination(10)); + manager.stop(); - expectedException.expect(new BaseMatcher() - { - @Override - public boolean matches(Object o) - { - return o instanceof ISE && ((ISE) o).getMessage().equals("Cannot restart after stop!"); - } + Assert.assertFalse(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + Assert.assertTrue(manager.waitForBackgroundTermination(10)); + Assert.assertFalse(manager.backgroundManagerIsRunning()); - @Override - public void describeTo(Description description) - { + manager.start(); + Assert.assertTrue(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + Assert.assertTrue(manager.backgroundManagerIsRunning()); + Assert.assertFalse(manager.waitForBackgroundTermination(10)); - } - }); - try { - manager.start(); - } - finally { - EasyMock.verify(configManager); - } + manager.stop(); + Assert.assertFalse(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + Assert.assertTrue(manager.waitForBackgroundTermination(10)); + Assert.assertFalse(manager.backgroundManagerIsRunning()); + + manager.start(); + Assert.assertTrue(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + Assert.assertTrue(manager.backgroundManagerIsRunning()); + Assert.assertFalse(manager.waitForBackgroundTermination(10)); + + manager.stop(); + Assert.assertFalse(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + Assert.assertTrue(manager.waitForBackgroundTermination(10)); + Assert.assertFalse(manager.backgroundManagerIsRunning()); + + EasyMock.verify(configManager); } @Test @@ -1301,7 +1339,6 @@ public void testLookupDiscoverAll() throws Exception EasyMock.verify(discoverer); } - @Test public void testLookupDiscoverAllExceptional() throws Exception { @@ -1310,20 +1347,22 @@ public void testLookupDiscoverAllExceptional() throws Exception EasyMock.expect(discoverer.discoverChildren(LookupCoordinatorManager.LOOKUP_LISTEN_ANNOUNCE_KEY)) .andThrow(ex) .once(); - expectedException.expectCause(new BaseMatcher() - { - @Override - public boolean matches(Object o) - { - return o == ex; - } + expectedException.expectCause( + new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o == ex; + } - @Override - public void describeTo(Description description) - { + @Override + public void describeTo(Description description) + { - } - }); + } + } + ); EasyMock.replay(discoverer); final LookupCoordinatorManager manager = new LookupCoordinatorManager( client, @@ -1339,4 +1378,69 @@ public void describeTo(Description description) EasyMock.verify(discoverer); } } + + //tests that lookups stored in db from 0.10.0 are converted and restored. + @Test + public void testBackwardCompatibilityMigration() throws Exception + { + EasyMock.reset(configManager); + + EasyMock.expect(configManager.watch( + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.anyObject(), + EasyMock.isNull() + )).andReturn( + new AtomicReference>>>(null)).once(); + + EasyMock.expect( + configManager.watch( + EasyMock.eq(LookupCoordinatorManager.OLD_LOOKUP_CONFIG_KEY), + EasyMock.anyObject(), + EasyMock.isNull() + ) + ).andReturn( + new AtomicReference>>>( + ImmutableMap.>>of( + "tier1", + ImmutableMap.>of("lookup1", ImmutableMap.of("k", "v")) + ) + ) + ).once(); + + EasyMock.expect( + configManager.set( + EasyMock.eq(LookupCoordinatorManager.LOOKUP_CONFIG_KEY), + EasyMock.eq( + ImmutableMap.>of( + "tier1", + ImmutableMap.of( + "lookup1", + new LookupExtractorFactoryMapContainer( + null, + ImmutableMap.of("k", "v") + ) + ) + ) + ), + EasyMock.anyObject(AuditInfo.class) + ) + ).andReturn(true).once(); + + EasyMock.replay(configManager); + + final LookupCoordinatorManager manager = new LookupCoordinatorManager( + client, + discoverer, + mapper, + configManager, + new LookupCoordinatorManagerConfig(){ + @Override + public long getPeriod(){ + return 1; + } + } + ); + manager.start(); + EasyMock.verify(configManager); + } } diff --git a/server/src/test/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainerTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainerTest.java new file mode 100644 index 000000000000..0e0714d8e186 --- /dev/null +++ b/server/src/test/java/io/druid/server/lookup/cache/LookupExtractorFactoryMapContainerTest.java @@ -0,0 +1,129 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.lookup.cache; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.lookup.LookupExtractorFactoryContainer; +import io.druid.query.lookup.MapLookupExtractorFactory; +import org.junit.Assert; +import org.junit.Test; + +/** + */ +public class LookupExtractorFactoryMapContainerTest +{ + private final ObjectMapper mapper; + private final String jsonStr; + private final LookupExtractorFactoryMapContainer testContainer; + + public LookupExtractorFactoryMapContainerTest() + { + mapper = new DefaultObjectMapper(); + mapper.registerSubtypes(MapLookupExtractorFactory.class); + + jsonStr = "{\n" + + " \"version\": \"v1\",\n" + + " \"lookupExtractorFactory\": {\n" + + " \"type\": \"map\",\n" + + " \"map\": {\"k\": \"v\"},\n" + + " \"isOneToOne\": true\n" + + " }\n" + + "}\n"; + + testContainer = new LookupExtractorFactoryMapContainer( + "v1", + ImmutableMap.of( + "type", "map", + "map", ImmutableMap.of("k", "v"), + "isOneToOne", true + ) + ); + } + + @Test + public void testSerde() throws Exception + { + LookupExtractorFactoryMapContainer actual = mapper.readValue( + mapper.writeValueAsString( + mapper.readValue(jsonStr, LookupExtractorFactoryMapContainer.class) + ), + LookupExtractorFactoryMapContainer.class + ); + Assert.assertEquals("v1", actual.getVersion()); + Assert.assertEquals(testContainer, actual); + } + + @Test + public void testReplaces() throws Exception + { + LookupExtractorFactoryMapContainer l0 = new LookupExtractorFactoryMapContainer(null, ImmutableMap.of()); + LookupExtractorFactoryMapContainer l1 = new LookupExtractorFactoryMapContainer(null, ImmutableMap.of()); + LookupExtractorFactoryMapContainer l2 = new LookupExtractorFactoryMapContainer("V2", ImmutableMap.of()); + LookupExtractorFactoryMapContainer l3 = new LookupExtractorFactoryMapContainer("V3", ImmutableMap.of()); + + Assert.assertFalse(l0.replaces(l1)); + Assert.assertFalse(l1.replaces(l2)); + Assert.assertTrue(l2.replaces(l1)); + Assert.assertFalse(l2.replaces(l3)); + Assert.assertTrue(l3.replaces(l2)); + } + + //test interchangeability with LookupExtractorFactoryContainer + //read and write as LookupExtractorFactoryContainer + //then read as LookupExtractorFactoryMapContainer + @Test + public void testInterchangeability1() throws Exception + { + LookupExtractorFactoryMapContainer actual = mapper.readValue( + mapper.writeValueAsString( + mapper.readValue(jsonStr, LookupExtractorFactoryContainer.class) + ), + LookupExtractorFactoryMapContainer.class + ); + + Assert.assertEquals("v1", actual.getVersion()); + Assert.assertEquals(testContainer, actual); + } + + //test interchangeability with LookupExtractorFactoryContainer + //read and write as LookupExtractorFactoryMapContainer + //then read as LookupExtractorFactoryContainer + @Test + public void testInterchangeability2() throws Exception + { + LookupExtractorFactoryContainer actual = mapper.readValue( + mapper.writeValueAsString( + mapper.readValue(jsonStr, LookupExtractorFactoryMapContainer.class) + ), + LookupExtractorFactoryContainer.class + ); + + Assert.assertEquals("v1", actual.getVersion()); + Assert.assertEquals( + actual, + new LookupExtractorFactoryContainer( + "v1", + new MapLookupExtractorFactory(ImmutableMap.of("k", "v"), true) + ) + ); + } +} diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 9a6b5ac4e878..6ef0feebbe50 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -160,15 +160,14 @@ public void configure(Binder binder) binder.bind(IndexingServiceClient.class).in(LazySingleton.class); binder.bind(CoordinatorServerView.class).in(LazySingleton.class); + binder.bind(LookupCoordinatorManager.class).in(LazySingleton.class); binder.bind(DruidCoordinator.class); - binder.bind(LookupCoordinatorManager.class).in(ManageLifecycle.class); binder.bind(ListenerDiscoverer.class).in(ManageLifecycle.class); LifecycleModule.register(binder, ListenerDiscoverer.class); LifecycleModule.register(binder, MetadataStorage.class); LifecycleModule.register(binder, DruidCoordinator.class); - LifecycleModule.register(binder, LookupCoordinatorManager.class); binder.bind(JettyServerInitializer.class) .to(CoordinatorJettyServerInitializer.class); diff --git a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java index e65bb6b52fa9..1889d8e8f58a 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java @@ -52,6 +52,7 @@ import io.druid.query.groupby.strategy.GroupByStrategySelector; import io.druid.query.lookup.LookupExtractor; import io.druid.query.lookup.LookupExtractorFactory; +import io.druid.query.lookup.LookupExtractorFactoryContainer; import io.druid.query.lookup.LookupIntrospectHandler; import io.druid.query.lookup.LookupReferencesManager; import io.druid.query.metadata.SegmentMetadataQueryConfig; @@ -293,45 +294,48 @@ public void configure(final Binder binder) final LookupReferencesManager mock = EasyMock.createMock(LookupReferencesManager.class); EasyMock.expect(mock.get(EasyMock.eq("lookyloo"))).andReturn( - new LookupExtractorFactory() - { - @Override - public boolean start() - { - throw new UnsupportedOperationException(); - } + new LookupExtractorFactoryContainer( + "v0", + new LookupExtractorFactory() + { + @Override + public boolean start() + { + throw new UnsupportedOperationException(); + } - @Override - public boolean close() - { - throw new UnsupportedOperationException(); - } + @Override + public boolean close() + { + throw new UnsupportedOperationException(); + } - @Override - public boolean replaces(@Nullable final LookupExtractorFactory other) - { - throw new UnsupportedOperationException(); - } + @Override + public boolean replaces(@Nullable final LookupExtractorFactory other) + { + throw new UnsupportedOperationException(); + } - @Nullable - @Override - public LookupIntrospectHandler getIntrospectHandler() - { - throw new UnsupportedOperationException(); - } + @Nullable + @Override + public LookupIntrospectHandler getIntrospectHandler() + { + throw new UnsupportedOperationException(); + } - @Override - public LookupExtractor get() - { - return new MapLookupExtractor( - ImmutableMap.of( - "a", "xa", - "abc", "xabc" - ), - false - ); - } - } + @Override + public LookupExtractor get() + { + return new MapLookupExtractor( + ImmutableMap.of( + "a", "xa", + "abc", "xabc" + ), + false + ); + } + } + ) ).anyTimes(); EasyMock.replay(mock); binder.bind(LookupReferencesManager.class).toInstance(mock);